You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by as...@apache.org on 2016/12/18 02:16:35 UTC

[01/29] hadoop git commit: HADOOP-13871. ITestS3AInputStreamPerformance.testTimeToOpenAndReadWholeFileBlocks performance awful. Contributed by Steve Loughran

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5085 f66f61892 -> fcbe15234


HADOOP-13871. ITestS3AInputStreamPerformance.testTimeToOpenAndReadWholeFileBlocks performance awful. Contributed by Steve Loughran


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c6a39232
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c6a39232
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c6a39232

Branch: refs/heads/YARN-5085
Commit: c6a39232456fa0c98b2b9b6dbeaec762294ca01e
Parents: f66f618
Author: Mingliang Liu <li...@apache.org>
Authored: Mon Dec 12 14:55:34 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Mon Dec 12 14:55:34 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/fs/s3a/S3AInputStream.java    | 33 ++++++++++---
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  3 ++
 .../tools/hadoop-aws/troubleshooting_s3a.md     | 52 ++++++++++++++++++++
 .../scale/ITestS3AInputStreamPerformance.java   | 47 ++++++++++++++++--
 .../hadoop/fs/s3a/scale/S3AScaleTestBase.java   | 20 +++++---
 5 files changed, 140 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6a39232/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index dd6cdd7..3c4093d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -132,7 +132,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
       throws IOException {
 
     if (wrappedStream != null) {
-      closeStream("reopen(" + reason + ")", contentRangeFinish);
+      closeStream("reopen(" + reason + ")", contentRangeFinish, false);
     }
 
     contentRangeFinish = calculateRequestLimit(inputPolicy, targetPos,
@@ -257,7 +257,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
 
     // if the code reaches here, the stream needs to be reopened.
     // close the stream; if read the object will be opened at the new pos
-    closeStream("seekInStream()", this.contentRangeFinish);
+    closeStream("seekInStream()", this.contentRangeFinish, false);
     pos = targetPos;
   }
 
@@ -414,7 +414,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
       closed = true;
       try {
         // close or abort the stream
-        closeStream("close() operation", this.contentRangeFinish);
+        closeStream("close() operation", this.contentRangeFinish, false);
         // this is actually a no-op
         super.close();
       } finally {
@@ -431,17 +431,17 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
    * an abort.
    *
    * This does not set the {@link #closed} flag.
-   *
    * @param reason reason for stream being closed; used in messages
    * @param length length of the stream.
+   * @param forceAbort force an abort; used if explicitly requested.
    */
-  private void closeStream(String reason, long length) {
+  private void closeStream(String reason, long length, boolean forceAbort) {
     if (wrappedStream != null) {
 
       // if the amount of data remaining in the current request is greater
       // than the readahead value: abort.
       long remaining = remainingInCurrentRequest();
-      boolean shouldAbort = remaining > readahead;
+      boolean shouldAbort = forceAbort || remaining > readahead;
       if (!shouldAbort) {
         try {
           // clean close. This will read to the end of the stream,
@@ -470,6 +470,27 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     }
   }
 
+  /**
+   * Forcibly reset the stream, by aborting the connection. The next
+   * {@code read()} operation will trigger the opening of a new HTTPS
+   * connection.
+   *
+   * This is potentially very inefficient, and should only be invoked
+   * in extreme circumstances. It logs at info for this reason.
+   * @return true if the connection was actually reset.
+   * @throws IOException if invoked on a closed stream.
+   */
+  @InterfaceStability.Unstable
+  public synchronized boolean resetConnection() throws IOException {
+    checkNotClosed();
+    boolean connectionOpen = wrappedStream != null;
+    if (connectionOpen) {
+      LOG.info("Forced reset of connection to {}", uri);
+      closeStream("reset()", contentRangeFinish, true);
+    }
+    return connectionOpen;
+  }
+
   @Override
   public synchronized int available() throws IOException {
     checkNotClosed();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6a39232/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index dedbfd4..aeb8403 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -509,6 +509,7 @@ public final class S3AUtils {
     Preconditions.checkArgument(v >= min,
         String.format("Value of %s: %d is below the minimum value %d",
             key, v, min));
+    LOG.debug("Value of {} is {}", key, v);
     return v;
   }
 
@@ -529,6 +530,7 @@ public final class S3AUtils {
     Preconditions.checkArgument(v >= min,
         String.format("Value of %s: %d is below the minimum value %d",
             key, v, min));
+    LOG.debug("Value of {} is {}", key, v);
     return v;
   }
 
@@ -550,6 +552,7 @@ public final class S3AUtils {
     Preconditions.checkArgument(v >= min,
             String.format("Value of %s: %d is below the minimum value %d",
                     key, v, min));
+    LOG.debug("Value of {} is {}", key, v);
     return v;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6a39232/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
new file mode 100644
index 0000000..d79720e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
@@ -0,0 +1,52 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Troubleshooting S3A
+
+Here are some lower level details and hints on troubleshooting and tuning
+the S3A client.
+
+## Logging at lower levels
+
+The AWS SDK and the Apache HTTP components can be configured to log at
+more detail, as can S3A itself.
+
+```properties
+log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
+log4j.logger.com.amazonaws.request=DEBUG
+log4j.logger.org.apache.http=DEBUG
+log4j.logger.org.apache.http.wire=ERROR
+```
+
+Be aware that logging HTTP headers may leak sensitive AWS account information,
+so should not be shared.
+
+## Advanced: network performance
+
+An example of this is covered in [HADOOP-13871](https://issues.apache.org/jira/browse/HADOOP-13871).
+
+1. For public data, use `curl`:
+
+        curl -O https://landsat-pds.s3.amazonaws.com/scene_list.gz
+1. Use `nettop` to monitor a processes connections.
+
+Consider reducing the connection timeout of the s3a connection.
+
+```xml
+<property>
+  <name>fs.s3a.connection.timeout</name>
+  <value>15000</value>
+</property>
+```
+This *may* cause the client to react faster to network pauses.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6a39232/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
index cc8187e..e36d086 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.util.LineReader;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
@@ -216,12 +217,18 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
     long count = 0;
     // implicitly rounding down here
     long blockCount = len / blockSize;
+    long totalToRead = blockCount * blockSize;
+    long minimumBandwidth = 128 * 1024;
+    int maxResetCount = 4;
+    int resetCount = 0;
     for (long i = 0; i < blockCount; i++) {
       int offset = 0;
       int remaining = blockSize;
+      long blockId = i + 1;
       NanoTimer blockTimer = new NanoTimer();
       int reads = 0;
       while (remaining > 0) {
+        NanoTimer readTimer = new NanoTimer();
         int bytesRead = in.read(block, offset, remaining);
         reads++;
         if (bytesRead == 1) {
@@ -230,14 +237,48 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
         remaining -= bytesRead;
         offset += bytesRead;
         count += bytesRead;
+        readTimer.end();
+        if (bytesRead != 0) {
+          LOG.debug("Bytes in read #{}: {} , block bytes: {}," +
+                  " remaining in block: {}" +
+                  " duration={} nS; ns/byte: {}, bandwidth={} MB/s",
+              reads, bytesRead, blockSize - remaining, remaining,
+              readTimer.duration(),
+              readTimer.nanosPerOperation(bytesRead),
+              readTimer.bandwidthDescription(bytesRead));
+        } else {
+          LOG.warn("0 bytes returned by read() operation #{}", reads);
+        }
+      }
+      blockTimer.end("Reading block %d in %d reads", blockId, reads);
+      String bw = blockTimer.bandwidthDescription(blockSize);
+      LOG.info("Bandwidth of block {}: {} MB/s: ", blockId, bw);
+      if (bandwidth(blockTimer, blockSize) < minimumBandwidth) {
+        LOG.warn("Bandwidth {} too low on block {}: resetting connection",
+            bw, blockId);
+        Assert.assertTrue("Bandwidth of " + bw +" too low after  "
+            + resetCount + " attempts", resetCount <= maxResetCount);
+        resetCount++;
+        // reset the connection
+        getS3AInputStream(in).resetConnection();
       }
-      blockTimer.end("Reading block %d in %d reads", i, reads);
     }
-    timer2.end("Time to read %d bytes in %d blocks", len, blockCount);
-    bandwidth(timer2, count);
+    timer2.end("Time to read %d bytes in %d blocks", totalToRead, blockCount);
+    LOG.info("Overall Bandwidth {} MB/s; reset connections {}",
+        timer2.bandwidth(totalToRead), resetCount);
     logStreamStatistics();
   }
 
+  /**
+   * Work out the bandwidth in bytes/second.
+   * @param timer timer measuring the duration
+   * @param bytes bytes
+   * @return the number of bytes/second of the recorded operation
+   */
+  public static double bandwidth(NanoTimer timer, long bytes) {
+    return bytes * 1.0e9 / timer.duration();
+  }
+
   @Test
   public void testLazySeekEnabled() throws Throwable {
     describe("Verify that seeks do not trigger any IO");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6a39232/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
index c4174bf..9da621f 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.fs.s3a.S3ATestConstants;
 import org.apache.hadoop.fs.s3a.Statistic;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 
-import org.junit.Assert;
 import org.junit.Assume;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -163,14 +162,23 @@ public class S3AScaleTestBase extends AbstractS3ATestBase {
    */
   protected S3AInstrumentation.InputStreamStatistics getInputStreamStatistics(
       FSDataInputStream in) {
+    return getS3AInputStream(in).getS3AStreamStatistics();
+  }
+
+  /**
+   * Get the inner stream of an input stream.
+   * Raises an exception if the inner stream is not an S3A input stream
+   * @param in wrapper
+   * @return the inner stream
+   * @throws AssertionError if the inner stream is of the wrong type
+   */
+  protected S3AInputStream getS3AInputStream(
+      FSDataInputStream in) {
     InputStream inner = in.getWrappedStream();
     if (inner instanceof S3AInputStream) {
-      S3AInputStream s3a = (S3AInputStream) inner;
-      return s3a.getS3AStreamStatistics();
+      return (S3AInputStream) inner;
     } else {
-      Assert.fail("Not an S3AInputStream: " + inner);
-      // never reached
-      return null;
+      throw new AssertionError("Not an S3AInputStream: " + inner);
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/29] hadoop git commit: HADOOP-13913. maven issuing pom.version being deprecation warnings. Contributed by Steve Loughran

Posted by as...@apache.org.
HADOOP-13913. maven issuing pom.version being deprecation warnings. Contributed by Steve Loughran


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fcbe1523
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fcbe1523
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fcbe1523

Branch: refs/heads/YARN-5085
Commit: fcbe1523422eeea74773dcd6b1c5ca410bac0d6f
Parents: f121645
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Dec 16 15:34:56 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Fri Dec 16 15:34:56 2016 -0800

----------------------------------------------------------------------
 hadoop-project/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcbe1523/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 36172f2..c9ee793 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -130,7 +130,7 @@
     <aws-java-sdk.version>1.11.45</aws-java-sdk.version>
     <!-- the version of Hadoop declared in the version resources; can be overridden
     so that Hadoop 3.x can declare itself a 2.x artifact. -->
-    <declared.hadoop.version>${pom.version}</declared.hadoop.version>
+    <declared.hadoop.version>${project.version}</declared.hadoop.version>
   </properties>
 
   <dependencyManagement>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/29] hadoop git commit: HDFS-11233. Fix javac warnings related to the deprecated APIs after upgrading Jackson. Contributed by Yiqun Lin.

Posted by as...@apache.org.
HDFS-11233. Fix javac warnings related to the deprecated APIs after upgrading Jackson. Contributed by Yiqun Lin.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2d4731c0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2d4731c0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2d4731c0

Branch: refs/heads/YARN-5085
Commit: 2d4731c067ff64cd88f496eac8faaf302faa2ccc
Parents: c6a3923
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Dec 13 11:11:15 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Dec 13 11:11:15 2016 +0900

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java    | 2 +-
 .../hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java  | 4 ++--
 .../org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java | 2 +-
 .../main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java | 2 +-
 .../java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java   | 2 +-
 .../web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java | 2 +-
 .../hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java  | 2 +-
 .../hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java    | 2 +-
 .../hadoop/hdfs/server/diskbalancer/command/Command.java     | 2 +-
 .../server/diskbalancer/connectors/JsonNodeConnector.java    | 2 +-
 .../server/diskbalancer/datamodel/DiskBalancerCluster.java   | 2 +-
 .../server/diskbalancer/datamodel/DiskBalancerVolume.java    | 2 +-
 .../hadoop/hdfs/server/diskbalancer/planner/NodePlan.java    | 2 +-
 .../test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java   | 8 ++++----
 14 files changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
index 0cdd107..505273e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
@@ -37,7 +37,7 @@ import java.io.IOException;
 public class DiskBalancerWorkItem {
   private static final ObjectMapper MAPPER = new ObjectMapper();
   private static final ObjectReader READER =
-      new ObjectMapper().reader(DiskBalancerWorkItem.class);
+      new ObjectMapper().readerFor(DiskBalancerWorkItem.class);
 
   private  long startTime;
   private long secondsElapsed;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
index 452ab27..af34c49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
@@ -43,9 +43,9 @@ public class DiskBalancerWorkStatus {
   private static final ObjectMapper MAPPER_WITH_INDENT_OUTPUT =
       new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT);
   private static final ObjectReader READER_WORKSTATUS =
-      new ObjectMapper().reader(DiskBalancerWorkStatus.class);
+      new ObjectMapper().readerFor(DiskBalancerWorkStatus.class);
   private static final ObjectReader READER_WORKENTRY = new ObjectMapper()
-      .reader(defaultInstance().constructCollectionType(List.class,
+      .readerFor(defaultInstance().constructCollectionType(List.class,
           DiskBalancerWorkEntry.class));
 
   private final List<DiskBalancerWorkEntry> currentState;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java
index 668d56e..8da5655 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
 @InterfaceStability.Unstable
 public final class CombinedHostsFileReader {
   private static final ObjectReader READER =
-      new ObjectMapper().reader(DatanodeAdminProperties.class);
+      new ObjectMapper().readerFor(DatanodeAdminProperties.class);
   private static final JsonFactory JSON_FACTORY = new JsonFactory();
 
   private CombinedHostsFileReader() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index 3690a86..246f242 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -526,7 +526,7 @@ class JsonUtilClient {
     }
 
     final String namesInJson = (String) json.get("XAttrNames");
-    ObjectReader reader = new ObjectMapper().reader(List.class);
+    ObjectReader reader = new ObjectMapper().readerFor(List.class);
     final List<Object> xattrs = reader.readValue(namesInJson);
     final List<String> names =
         Lists.newArrayListWithCapacity(json.keySet().size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index fbb4bd6..26cfc01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -152,7 +152,7 @@ public class WebHdfsFileSystem extends FileSystem
   private String restCsrfCustomHeader;
   private Set<String> restCsrfMethodsToIgnore;
   private static final ObjectReader READER =
-      new ObjectMapper().reader(Map.class);
+      new ObjectMapper().readerFor(Map.class);
 
   private DFSOpsCountStatistics storageStatistics;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java
index f90ca19..c6ebdd6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java
@@ -56,7 +56,7 @@ import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull;
 public class ConfRefreshTokenBasedAccessTokenProvider
     extends AccessTokenProvider {
   private static final ObjectReader READER =
-      new ObjectMapper().reader(Map.class);
+      new ObjectMapper().readerFor(Map.class);
 
   public static final String OAUTH_REFRESH_TOKEN_KEY
       = "dfs.webhdfs.oauth2.refresh.token";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java
index 8c0ddd4..5c629e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java
@@ -56,7 +56,7 @@ import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull;
 public abstract class CredentialBasedAccessTokenProvider
     extends AccessTokenProvider {
   private static final ObjectReader READER =
-      new ObjectMapper().reader(Map.class);
+      new ObjectMapper().readerFor(Map.class);
 
   public static final String OAUTH_CREDENTIAL_KEY
       = "dfs.webhdfs.oauth2.credential";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 97bb09b..c317715 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -106,7 +106,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   private static final ObjectWriter WRITER =
       new ObjectMapper().writerWithDefaultPrettyPrinter();
   private static final ObjectReader READER =
-      new ObjectMapper().reader(BlockIteratorState.class);
+      new ObjectMapper().readerFor(BlockIteratorState.class);
 
   private final FsDatasetImpl dataset;
   private final String storageID;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index cee37e5..22fd5be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -77,7 +77,7 @@ import java.util.TreeSet;
  */
 public abstract class Command extends Configured implements Closeable {
   private static final ObjectReader READER =
-      new ObjectMapper().reader(HashMap.class);
+      new ObjectMapper().readerFor(HashMap.class);
   static final Logger LOG = LoggerFactory.getLogger(Command.class);
   private Map<String, String> validArgs = new HashMap<>();
   private URI clusterURI;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
index 323938a..838511b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
@@ -38,7 +38,7 @@ public class JsonNodeConnector implements ClusterConnector {
   private static final Logger LOG =
       LoggerFactory.getLogger(JsonNodeConnector.class);
   private static final ObjectReader READER =
-      new ObjectMapper().reader(DiskBalancerCluster.class);
+      new ObjectMapper().readerFor(DiskBalancerCluster.class);
   private final URL clusterURI;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
index 1b17829..2f54141 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
@@ -73,7 +73,7 @@ public class DiskBalancerCluster {
   private static final Logger LOG =
       LoggerFactory.getLogger(DiskBalancerCluster.class);
   private static final ObjectReader READER =
-      new ObjectMapper().reader(DiskBalancerCluster.class);
+      new ObjectMapper().readerFor(DiskBalancerCluster.class);
   private final Set<String> exclusionList;
   private final Set<String> inclusionList;
   private ClusterConnector clusterConnector;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
index ccaa7d9..47a925c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
@@ -33,7 +33,7 @@ import java.io.IOException;
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class DiskBalancerVolume {
   private static final ObjectReader READER =
-      new ObjectMapper().reader(DiskBalancerVolume.class);
+      new ObjectMapper().readerFor(DiskBalancerVolume.class);
 
   private String path;
   private long capacity;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
index 153536a..c72bb1f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
@@ -166,7 +166,7 @@ public class NodePlan {
   public String toJson() throws IOException {
     ObjectMapper mapper = new ObjectMapper();
     JavaType planType = mapper.constructType(NodePlan.class);
-    return mapper.writerWithType(planType)
+    return mapper.writerFor(planType)
         .writeValueAsString(this);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d4731c0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
index 94cec85..5a3d451 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
@@ -71,7 +71,7 @@ public class TestJsonUtil {
     System.out.println("fstatus = " + fstatus);
     final String json = JsonUtil.toJsonString(status, true);
     System.out.println("json    = " + json.replace(",", ",\n  "));
-    ObjectReader reader = new ObjectMapper().reader(Map.class);
+    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
     final HdfsFileStatus s2 =
         JsonUtilClient.toFileStatus((Map<?, ?>) reader.readValue(json), true);
     final FileStatus fs2 = toFileStatus(s2, parent);
@@ -159,7 +159,7 @@ public class TestJsonUtil {
   public void testToAclStatus() throws IOException {
     String jsonString =
         "{\"AclStatus\":{\"entries\":[\"user::rwx\",\"user:user1:rw-\",\"group::rw-\",\"other::r-x\"],\"group\":\"supergroup\",\"owner\":\"testuser\",\"stickyBit\":false}}";
-    ObjectReader reader = new ObjectMapper().reader(Map.class);
+    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
     Map<?, ?> json = reader.readValue(jsonString);
 
     List<AclEntry> aclSpec =
@@ -219,7 +219,7 @@ public class TestJsonUtil {
     String jsonString = 
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
-    ObjectReader reader = new ObjectMapper().reader(Map.class);
+    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
     Map<?, ?> json = reader.readValue(jsonString);
     XAttr xAttr1 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
         setName("a1").setValue(XAttrCodec.decodeValue("0x313233")).build();
@@ -245,7 +245,7 @@ public class TestJsonUtil {
     String jsonString = 
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
-    ObjectReader reader = new ObjectMapper().reader(Map.class);
+    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
     Map<?, ?> json = reader.readValue(jsonString);
 
     // Get xattr: user.a2


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/29] hadoop git commit: HDFS-11249. Redundant toString() in DFSConfigKeys.java. Contributed by Jagadesh Kiran N.

Posted by as...@apache.org.
HDFS-11249. Redundant toString() in DFSConfigKeys.java. Contributed by Jagadesh Kiran N.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/36947f79
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/36947f79
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/36947f79

Branch: refs/heads/YARN-5085
Commit: 36947f79ba66bf8ce150813e5dc1d06c789f2b2a
Parents: 64a2d5b
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Dec 16 00:33:27 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Dec 16 00:33:27 2016 +0900

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java        | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/36947f79/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index cffc4bd..80ceaaf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -249,7 +249,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
      need to update the literal here as well as TestDFSConfigKeys.
    */
   public static final String  DFS_WEBHDFS_AUTHENTICATION_FILTER_DEFAULT =
-      "org.apache.hadoop.hdfs.web.AuthFilter".toString();
+      "org.apache.hadoop.hdfs.web.AuthFilter";
   @Deprecated
   public static final String  DFS_WEBHDFS_USER_PATTERN_KEY =
       HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/29] hadoop git commit: HDFS-11226. cacheadmin, cryptoadmin and storagepolicyadmin should support generic options. Contributed by Brahma Reddy Battula

Posted by as...@apache.org.
HDFS-11226. cacheadmin, cryptoadmin and storagepolicyadmin should support generic options. Contributed by Brahma Reddy Battula


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/754f15ba
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/754f15ba
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/754f15ba

Branch: refs/heads/YARN-5085
Commit: 754f15bae61b81ad3c2e3f722d1feaebf374e2c4
Parents: 2d4731c
Author: Mingliang Liu <li...@apache.org>
Authored: Mon Dec 12 17:36:52 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Mon Dec 12 18:21:15 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java   | 8 ++++++--
 .../main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java  | 8 ++++++--
 .../org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java     | 6 +++++-
 3 files changed, 17 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/754f15ba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
index 270a662..522f701 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 
 import com.google.common.base.Joiner;
+import org.apache.hadoop.util.ToolRunner;
 
 /**
  * This class implements command-line operations on the HDFS Cache.
@@ -64,6 +65,7 @@ public class CacheAdmin extends Configured implements Tool {
   public int run(String[] args) throws IOException {
     if (args.length == 0) {
       AdminHelper.printUsage(false, "cacheadmin", COMMANDS);
+      ToolRunner.printGenericCommandUsage(System.err);
       return 1;
     }
     AdminHelper.Command command = AdminHelper.determineCommand(args[0],
@@ -74,6 +76,7 @@ public class CacheAdmin extends Configured implements Tool {
         System.err.println("Command names must start with dashes.");
       }
       AdminHelper.printUsage(false, "cacheadmin", COMMANDS);
+      ToolRunner.printGenericCommandUsage(System.err);
       return 1;
     }
     List<String> argsList = new LinkedList<String>();
@@ -88,9 +91,10 @@ public class CacheAdmin extends Configured implements Tool {
     }
   }
 
-  public static void main(String[] argsArray) throws IOException {
+  public static void main(String[] argsArray) throws Exception {
     CacheAdmin cacheAdmin = new CacheAdmin(new Configuration());
-    System.exit(cacheAdmin.run(argsArray));
+    int res = ToolRunner.run(cacheAdmin, argsArray);
+    System.exit(res);
   }
 
   private static CacheDirectiveInfo.Expiration parseExpirationString(String ttlString)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/754f15ba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
index b78da31..225f11a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 
 /**
  * This class implements crypto command-line operations.
@@ -54,6 +55,7 @@ public class CryptoAdmin extends Configured implements Tool {
   public int run(String[] args) throws IOException {
     if (args.length == 0) {
       AdminHelper.printUsage(false, "crypto", COMMANDS);
+      ToolRunner.printGenericCommandUsage(System.err);
       return 1;
     }
     final AdminHelper.Command command = AdminHelper.determineCommand(args[0],
@@ -64,6 +66,7 @@ public class CryptoAdmin extends Configured implements Tool {
         System.err.println("Command names must start with dashes.");
       }
       AdminHelper.printUsage(false, "crypto", COMMANDS);
+      ToolRunner.printGenericCommandUsage(System.err);
       return 1;
     }
     final List<String> argsList = new LinkedList<String>();
@@ -78,9 +81,10 @@ public class CryptoAdmin extends Configured implements Tool {
     }
   }
 
-  public static void main(String[] argsArray) throws IOException {
+  public static void main(String[] argsArray) throws Exception {
     final CryptoAdmin cryptoAdmin = new CryptoAdmin(new Configuration());
-    System.exit(cryptoAdmin.run(argsArray));
+    int res = ToolRunner.run(cryptoAdmin, argsArray);
+    System.exit(res);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/754f15ba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
index 24079b3..d99b88a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -42,7 +43,8 @@ public class StoragePolicyAdmin extends Configured implements Tool {
   public static void main(String[] argsArray) throws Exception {
     final StoragePolicyAdmin admin = new StoragePolicyAdmin(new
         Configuration());
-    System.exit(admin.run(argsArray));
+    int res = ToolRunner.run(admin, argsArray);
+    System.exit(res);
   }
 
   public StoragePolicyAdmin(Configuration conf) {
@@ -53,6 +55,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
   public int run(String[] args) throws Exception {
     if (args.length == 0) {
       AdminHelper.printUsage(false, "storagepolicies", COMMANDS);
+      ToolRunner.printGenericCommandUsage(System.err);
       return 1;
     }
     final AdminHelper.Command command = AdminHelper.determineCommand(args[0],
@@ -63,6 +66,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
         System.err.println("Command names must start with dashes.");
       }
       AdminHelper.printUsage(false, "storagepolicies", COMMANDS);
+      ToolRunner.printGenericCommandUsage(System.err);
       return 1;
     }
     final List<String> argsList = new LinkedList<>();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/29] hadoop git commit: HADOOP-13709. Ability to clean up subprocesses spawned by Shell when the process exits. Contributed by Eric Badger

Posted by as...@apache.org.
HADOOP-13709. Ability to clean up subprocesses spawned by Shell when the process exits. Contributed by Eric Badger


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9947aeb6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9947aeb6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9947aeb6

Branch: refs/heads/YARN-5085
Commit: 9947aeb60c3dd075544866fd6e4dab0ad8b4afa2
Parents: ef34bf2
Author: Jason Lowe <jl...@apache.org>
Authored: Tue Dec 13 22:55:09 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue Dec 13 22:55:09 2016 +0000

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/util/Shell.java | 24 +++++++
 .../java/org/apache/hadoop/util/TestShell.java  | 68 ++++++++++++++++++++
 2 files changed, 92 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9947aeb6/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
index 5fc9869..83877b7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
@@ -26,9 +26,11 @@ import java.io.InputStream;
 import java.io.InterruptedIOException;
 import java.nio.charset.Charset;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Map;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.WeakHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -48,6 +50,8 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public abstract class Shell {
+  private static final Map <Process, Object> CHILD_PROCESSES =
+      Collections.synchronizedMap(new WeakHashMap<Process, Object>());
   public static final Logger LOG = LoggerFactory.getLogger(Shell.class);
 
   /**
@@ -916,6 +920,7 @@ public abstract class Shell {
     } else {
       process = builder.start();
     }
+    CHILD_PROCESSES.put(process, null);
 
     if (timeOutInterval > 0) {
       timeOutTimer = new Timer("Shell command timeout");
@@ -1012,6 +1017,7 @@ public abstract class Shell {
         LOG.warn("Error while closing the error stream", ioe);
       }
       process.destroy();
+      CHILD_PROCESSES.remove(process);
       lastTime = Time.monotonicNow();
     }
   }
@@ -1310,4 +1316,22 @@ public abstract class Shell {
       }
     }
   }
+
+  /**
+   * Static method to destroy all running <code>Shell</code> processes
+   * Iterates through a list of all currently running <code>Shell</code>
+   * processes and destroys them one by one. This method is thread safe and
+   * is intended to be used in a shutdown hook.
+   */
+  public static void destroyAllProcesses() {
+    synchronized (CHILD_PROCESSES) {
+      for (Process key : CHILD_PROCESSES.keySet()) {
+        Process process = key;
+        if (key != null) {
+          process.destroy();
+        }
+      }
+      CHILD_PROCESSES.clear();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9947aeb6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
index 67903f7..5cc011b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.util;
 
+import com.google.common.base.Supplier;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.security.alias.AbstractJavaKeyStoreProvider;
 import org.junit.Assert;
@@ -471,4 +472,71 @@ public class TestShell extends Assert {
     assertEquals("'foo'\\''bar'", Shell.bashQuote("foo'bar"));
     assertEquals("''\\''foo'\\''bar'\\'''", Shell.bashQuote("'foo'bar'"));
   }
+
+  @Test(timeout=120000)
+  public void testShellKillAllProcesses() throws Throwable {
+    Assume.assumeFalse(WINDOWS);
+    StringBuffer sleepCommand = new StringBuffer();
+    sleepCommand.append("sleep 200");
+    String[] shellCmd = {"bash", "-c", sleepCommand.toString()};
+    final ShellCommandExecutor shexc1 = new ShellCommandExecutor(shellCmd);
+    final ShellCommandExecutor shexc2 = new ShellCommandExecutor(shellCmd);
+
+    Thread shellThread1 = new Thread() {
+      @Override
+      public void run() {
+        try {
+          shexc1.execute();
+        } catch(IOException ioe) {
+          //ignore IOException from thread interrupt
+        }
+      }
+    };
+    Thread shellThread2 = new Thread() {
+      @Override
+      public void run() {
+        try {
+          shexc2.execute();
+        } catch(IOException ioe) {
+          //ignore IOException from thread interrupt
+        }
+      }
+    };
+
+    shellThread1.start();
+    shellThread2.start();
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return shexc1.getProcess() != null;
+      }
+    }, 10, 10000);
+
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return shexc2.getProcess() != null;
+      }
+    }, 10, 10000);
+
+    Shell.destroyAllProcesses();
+    final Process process1 = shexc1.getProcess();
+    final Process process2 = shexc2.getProcess();
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return !process1.isAlive();
+      }
+    }, 10, 10000);
+
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return !process2.isAlive();
+      }
+    }, 10, 10000);
+
+    assertFalse("Process 1 was not killed within timeout", process1.isAlive());
+    assertFalse("Process 2 was not killed within timeout", process2.isAlive());
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/29] hadoop git commit: This closes #119. (empty commit)

Posted by as...@apache.org.
This closes #119. (empty commit)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f121645c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f121645c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f121645c

Branch: refs/heads/YARN-5085
Commit: f121645c4bdfdabfd6e4c40cd0c3dc8930372e17
Parents: a956390
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Dec 16 11:04:32 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Dec 16 11:04:32 2016 -0800

----------------------------------------------------------------------

----------------------------------------------------------------------



---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/29] hadoop git commit: HDFS-10958. Add instrumentation hooks around Datanode disk IO.

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
index dc63238..d3006c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
@@ -45,6 +45,10 @@ abstract public class ReplicaInfo extends Block
   /** volume where the replica belongs. */
   private FsVolumeSpi volume;
 
+  /** This is used by some tests and FsDatasetUtil#computeChecksum. */
+  private static final FileIoProvider DEFAULT_FILE_IO_PROVIDER =
+      new FileIoProvider(null);
+
   /**
   * Constructor
   * @param vol volume where replica is located
@@ -64,7 +68,18 @@ abstract public class ReplicaInfo extends Block
   public FsVolumeSpi getVolume() {
     return volume;
   }
-  
+
+  /**
+   * Get the {@link FileIoProvider} for disk IO operations.
+   */
+  public FileIoProvider getFileIoProvider() {
+    // In tests and when invoked via FsDatasetUtil#computeChecksum, the
+    // target volume for this replica may be unknown and hence null.
+    // Use the DEFAULT_FILE_IO_PROVIDER with no-op hooks.
+    return (volume != null) ? volume.getFileIoProvider()
+        : DEFAULT_FILE_IO_PROVIDER;
+  }
+
   /**
    * Set the volume where this replica is located on disk.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index a11a207..4947ecf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
@@ -418,4 +419,6 @@ public interface FsVolumeSpi
    */
   class VolumeCheckContext {
   }
+
+  FileIoProvider getFileIoProvider();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
index 54d0e96..f40315a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
@@ -24,8 +24,8 @@ import java.io.InputStream;
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIOException;
 import org.slf4j.Logger;
 
@@ -38,12 +38,15 @@ public class ReplicaInputStreams implements Closeable {
   private InputStream dataIn;
   private InputStream checksumIn;
   private FsVolumeReference volumeRef;
+  private final FileIoProvider fileIoProvider;
   private FileDescriptor dataInFd = null;
 
   /** Create an object with a data input stream and a checksum input stream. */
-  public ReplicaInputStreams(InputStream dataStream,
-      InputStream checksumStream, FsVolumeReference volumeRef) {
+  public ReplicaInputStreams(
+      InputStream dataStream, InputStream checksumStream,
+      FsVolumeReference volumeRef, FileIoProvider fileIoProvider) {
     this.volumeRef = volumeRef;
+    this.fileIoProvider = fileIoProvider;
     this.dataIn = dataStream;
     this.checksumIn = checksumStream;
     if (dataIn instanceof FileInputStream) {
@@ -103,7 +106,7 @@ public class ReplicaInputStreams implements Closeable {
   public void dropCacheBehindReads(String identifier, long offset, long len,
       int flags) throws NativeIOException {
     assert this.dataInFd != null : "null dataInFd!";
-    NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+    fileIoProvider.posixFadvise(getVolumeRef().getVolume(),
         identifier, dataInFd, offset, len, flags);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
index a66847a..1614ba2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
@@ -24,11 +24,10 @@ import java.io.OutputStream;
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIOException;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 
 /**
@@ -43,21 +42,22 @@ public class ReplicaOutputStreams implements Closeable {
   /** Stream to checksum. */
   private final OutputStream checksumOut;
   private final DataChecksum checksum;
-  private final boolean isTransientStorage;
-  private final long slowLogThresholdMs;
+  private final FsVolumeSpi volume;
+  private final FileIoProvider fileIoProvider;
 
   /**
    * Create an object with a data output stream, a checksum output stream
    * and a checksum.
    */
-  public ReplicaOutputStreams(OutputStream dataOut,
-      OutputStream checksumOut, DataChecksum checksum,
-      boolean isTransientStorage, long slowLogThresholdMs) {
+  public ReplicaOutputStreams(
+      OutputStream dataOut, OutputStream checksumOut, DataChecksum checksum,
+      FsVolumeSpi volume, FileIoProvider fileIoProvider) {
+
     this.dataOut = dataOut;
     this.checksum = checksum;
-    this.slowLogThresholdMs = slowLogThresholdMs;
-    this.isTransientStorage = isTransientStorage;
     this.checksumOut = checksumOut;
+    this.volume = volume;
+    this.fileIoProvider = fileIoProvider;
 
     try {
       if (this.dataOut instanceof FileOutputStream) {
@@ -93,7 +93,7 @@ public class ReplicaOutputStreams implements Closeable {
 
   /** @return is writing to a transient storage? */
   public boolean isTransientStorage() {
-    return isTransientStorage;
+    return volume.isTransientStorage();
   }
 
   @Override
@@ -112,7 +112,7 @@ public class ReplicaOutputStreams implements Closeable {
    */
   public void syncDataOut() throws IOException {
     if (dataOut instanceof FileOutputStream) {
-      sync((FileOutputStream)dataOut);
+      fileIoProvider.sync(volume, (FileOutputStream) dataOut);
     }
   }
   
@@ -121,7 +121,7 @@ public class ReplicaOutputStreams implements Closeable {
    */
   public void syncChecksumOut() throws IOException {
     if (checksumOut instanceof FileOutputStream) {
-      sync((FileOutputStream)checksumOut);
+      fileIoProvider.sync(volume, (FileOutputStream) checksumOut);
     }
   }
 
@@ -129,60 +129,34 @@ public class ReplicaOutputStreams implements Closeable {
    * Flush the data stream if it supports it.
    */
   public void flushDataOut() throws IOException {
-    flush(dataOut);
+    if (dataOut != null) {
+      fileIoProvider.flush(volume, dataOut);
+    }
   }
 
   /**
    * Flush the checksum stream if it supports it.
    */
   public void flushChecksumOut() throws IOException {
-    flush(checksumOut);
-  }
-
-  private void flush(OutputStream dos) throws IOException {
-    long begin = Time.monotonicNow();
-    dos.flush();
-    long duration = Time.monotonicNow() - begin;
-    LOG.trace("ReplicaOutputStreams#flush takes {} ms.", duration);
-    if (duration > slowLogThresholdMs) {
-      LOG.warn("Slow flush took {} ms (threshold={} ms)", duration,
-          slowLogThresholdMs);
+    if (checksumOut != null) {
+      fileIoProvider.flush(volume, checksumOut);
     }
   }
 
-  private void sync(FileOutputStream fos) throws IOException {
-    long begin = Time.monotonicNow();
-    fos.getChannel().force(true);
-    long duration = Time.monotonicNow() - begin;
-    LOG.trace("ReplicaOutputStreams#sync takes {} ms.", duration);
-    if (duration > slowLogThresholdMs) {
-      LOG.warn("Slow fsync took {} ms (threshold={} ms)", duration,
-          slowLogThresholdMs);
-    }
-  }
-
-  public long writeToDisk(byte[] b, int off, int len) throws IOException {
-    long begin = Time.monotonicNow();
+  public void writeDataToDisk(byte[] b, int off, int len)
+      throws IOException {
     dataOut.write(b, off, len);
-    long duration = Time.monotonicNow() - begin;
-    LOG.trace("DatanodeIO#writeToDisk takes {} ms.", duration);
-    if (duration > slowLogThresholdMs) {
-      LOG.warn("Slow BlockReceiver write data to disk cost: {} ms " +
-          "(threshold={} ms)", duration, slowLogThresholdMs);
-    }
-    return duration;
   }
 
   public void syncFileRangeIfPossible(long offset, long nbytes,
       int flags) throws NativeIOException {
-    assert this.outFd != null : "null outFd!";
-    NativeIO.POSIX.syncFileRangeIfPossible(outFd, offset, nbytes, flags);
+    fileIoProvider.syncFileRange(
+        volume, outFd, offset, nbytes, flags);
   }
 
   public void dropCacheBehindWrites(String identifier,
       long offset, long len, int flags) throws NativeIOException {
-    assert this.outFd != null : "null outFd!";
-    NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-        identifier, outFd, offset, len, flags);
+    fileIoProvider.posixFadvise(
+        volume, identifier, outFd, offset, len, flags);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
index 63e82f3..8273ebb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -32,13 +32,11 @@ import java.util.Iterator;
 import java.util.Scanner;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CachingGetSpaceUsed;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.GetSpaceUsed;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
@@ -46,10 +44,10 @@ 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.FileIoProvider;
 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.LocalReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
@@ -64,7 +62,6 @@ import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.Timer;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.io.Files;
 
 /**
  * A block pool slice represents a portion of a block pool stored on a volume.
@@ -96,6 +93,7 @@ class BlockPoolSlice {
   private final long cachedDfsUsedCheckTime;
   private final Timer timer;
   private final int maxDataLength;
+  private final FileIoProvider fileIoProvider;
 
   // TODO:FEDERATION scalability issue - a thread per DU is needed
   private final GetSpaceUsed dfsUsage;
@@ -113,6 +111,7 @@ class BlockPoolSlice {
       Configuration conf, Timer timer) throws IOException {
     this.bpid = bpid;
     this.volume = volume;
+    this.fileIoProvider = volume.getFileIoProvider();
     this.currentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
     this.finalizedDir = new File(
         currentDir, DataStorage.STORAGE_DIR_FINALIZED);
@@ -147,19 +146,14 @@ class BlockPoolSlice {
     //
     this.tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP);
     if (tmpDir.exists()) {
-      DataStorage.fullyDelete(tmpDir);
+      fileIoProvider.fullyDelete(volume, tmpDir);
     }
     this.rbwDir = new File(currentDir, DataStorage.STORAGE_DIR_RBW);
-    if (!rbwDir.mkdirs()) {  // create rbw directory if not exist
-      if (!rbwDir.isDirectory()) {
-        throw new IOException("Mkdirs failed to create " + rbwDir.toString());
-      }
-    }
-    if (!tmpDir.mkdirs()) {
-      if (!tmpDir.isDirectory()) {
-        throw new IOException("Mkdirs failed to create " + tmpDir.toString());
-      }
-    }
+
+    // create the rbw and tmp directories if they don't exist.
+    fileIoProvider.mkdirs(volume, rbwDir);
+    fileIoProvider.mkdirs(volume, tmpDir);
+
     // Use cached value initially if available. Or the following call will
     // block until the initial du command completes.
     this.dfsUsage = new CachingGetSpaceUsed.Builder().setPath(bpDir)
@@ -266,7 +260,7 @@ class BlockPoolSlice {
    */
   void saveDfsUsed() {
     File outFile = new File(currentDir, DU_CACHE_FILE);
-    if (outFile.exists() && !outFile.delete()) {
+    if (!fileIoProvider.deleteWithExistsCheck(volume, outFile)) {
       FsDatasetImpl.LOG.warn("Failed to delete old dfsUsed file in " +
         outFile.getParent());
     }
@@ -277,7 +271,7 @@ class BlockPoolSlice {
           new FileOutputStream(outFile), "UTF-8")) {
         // mtime is written last, so that truncated writes won't be valid.
         out.write(Long.toString(used) + " " + Long.toString(timer.now()));
-        out.flush();
+        fileIoProvider.flush(volume, out);
       }
     } catch (IOException ioe) {
       // If write failed, the volume might be bad. Since the cache file is
@@ -292,7 +286,8 @@ class BlockPoolSlice {
    */
   File createTmpFile(Block b) throws IOException {
     File f = new File(tmpDir, b.getBlockName());
-    File tmpFile = DatanodeUtil.createTmpFile(b, f);
+    File tmpFile = DatanodeUtil.createFileWithExistsCheck(
+        volume, b, f, fileIoProvider);
     // If any exception during creation, its expected that counter will not be
     // incremented, So no need to decrement
     incrNumBlocks();
@@ -305,7 +300,8 @@ class BlockPoolSlice {
    */
   File createRbwFile(Block b) throws IOException {
     File f = new File(rbwDir, b.getBlockName());
-    File rbwFile = DatanodeUtil.createTmpFile(b, f);
+    File rbwFile = DatanodeUtil.createFileWithExistsCheck(
+        volume, b, f, fileIoProvider);
     // If any exception during creation, its expected that counter will not be
     // incremented, So no need to decrement
     incrNumBlocks();
@@ -314,11 +310,7 @@ class BlockPoolSlice {
 
   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);
-      }
-    }
+    fileIoProvider.mkdirsWithExistsCheck(volume, blockDir);
     File blockFile = FsDatasetImpl.moveBlockFiles(b, replicaInfo, blockDir);
     File metaFile = FsDatasetUtil.getMetaFile(blockFile, b.getGenerationStamp());
     if (dfsUsage instanceof CachingGetSpaceUsed) {
@@ -340,9 +332,9 @@ class BlockPoolSlice {
     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);
+    fileIoProvider.moveFile(volume, blockFile, targetBlockFile);
     FsDatasetImpl.LOG.info("Moved " + blockFile + " to " + targetBlockFile);
-    FileUtils.moveFile(metaFile, targetMetaFile);
+    fileIoProvider.moveFile(volume, metaFile, targetMetaFile);
     FsDatasetImpl.LOG.info("Moved " + metaFile + " to " + targetMetaFile);
 
     ReplicaInfo newReplicaInfo =
@@ -394,16 +386,13 @@ class BlockPoolSlice {
     File blockFile = FsDatasetUtil.getOrigFile(unlinkedTmp);
     if (blockFile.exists()) {
       // If the original block file still exists, then no recovery is needed.
-      if (!unlinkedTmp.delete()) {
+      if (!fileIoProvider.delete(volume, unlinkedTmp)) {
         throw new IOException("Unable to cleanup unlinked tmp file " +
             unlinkedTmp);
       }
       return null;
     } else {
-      if (!unlinkedTmp.renameTo(blockFile)) {
-        throw new IOException("Unable to rename unlinked tmp file " +
-            unlinkedTmp);
-      }
+      fileIoProvider.rename(volume, unlinkedTmp, blockFile);
       return blockFile;
     }
   }
@@ -416,7 +405,7 @@ class BlockPoolSlice {
    */
   private int moveLazyPersistReplicasToFinalized(File source)
       throws IOException {
-    File files[] = FileUtil.listFiles(source);
+    File[] files = fileIoProvider.listFiles(volume, source);
     int numRecovered = 0;
     for (File file : files) {
       if (file.isDirectory()) {
@@ -431,24 +420,25 @@ class BlockPoolSlice {
 
         if (blockFile.exists()) {
 
-          if (!targetDir.exists() && !targetDir.mkdirs()) {
+          try {
+            fileIoProvider.mkdirsWithExistsCheck(volume, targetDir);
+          } catch(IOException ioe) {
             LOG.warn("Failed to mkdirs " + targetDir);
             continue;
           }
 
           final File targetMetaFile = new File(targetDir, metaFile.getName());
           try {
-            LocalReplica.rename(metaFile, targetMetaFile);
+            fileIoProvider.rename(volume, metaFile, targetMetaFile);
           } catch (IOException e) {
             LOG.warn("Failed to move meta file from "
                 + metaFile + " to " + targetMetaFile, e);
             continue;
-
           }
 
           final File targetBlockFile = new File(targetDir, blockFile.getName());
           try {
-            LocalReplica.rename(blockFile, targetBlockFile);
+            fileIoProvider.rename(volume, blockFile, targetBlockFile);
           } catch (IOException e) {
             LOG.warn("Failed to move block file from "
                 + blockFile + " to " + targetBlockFile, e);
@@ -465,7 +455,7 @@ class BlockPoolSlice {
       }
     }
 
-    FileUtil.fullyDelete(source);
+    fileIoProvider.fullyDelete(volume, source);
     return numRecovered;
   }
 
@@ -508,7 +498,7 @@ class BlockPoolSlice {
           loadRwr = false;
         }
         sc.close();
-        if (!restartMeta.delete()) {
+        if (!fileIoProvider.delete(volume, restartMeta)) {
           FsDatasetImpl.LOG.warn("Failed to delete restart meta file: " +
               restartMeta.getPath());
         }
@@ -568,7 +558,7 @@ class BlockPoolSlice {
                         final RamDiskReplicaTracker lazyWriteReplicaMap,
                         boolean isFinalized)
       throws IOException {
-    File files[] = FileUtil.listFiles(dir);
+    File[] files = fileIoProvider.listFiles(volume, dir);
     for (File file : files) {
       if (file.isDirectory()) {
         addToReplicasMap(volumeMap, file, lazyWriteReplicaMap, isFinalized);
@@ -581,8 +571,9 @@ class BlockPoolSlice {
           continue;
         }
       }
-      if (!Block.isBlockFilename(file))
+      if (!Block.isBlockFilename(file)) {
         continue;
+      }
 
       long genStamp = FsDatasetUtil.getGenerationStampFromFile(
           files, file);
@@ -700,7 +691,8 @@ class BlockPoolSlice {
         return 0;
       }
       try (DataInputStream checksumIn = new DataInputStream(
-          new BufferedInputStream(new FileInputStream(metaFile),
+          new BufferedInputStream(
+              fileIoProvider.getFileInputStream(volume, metaFile),
               ioFileBufferSize))) {
         // read and handle the common header here. For now just a version
         final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(
@@ -713,9 +705,10 @@ class BlockPoolSlice {
         if (numChunks == 0) {
           return 0;
         }
-        try (InputStream blockIn = new FileInputStream(blockFile);
+        try (InputStream blockIn = fileIoProvider.getFileInputStream(
+                 volume, blockFile);
              ReplicaInputStreams ris = new ReplicaInputStreams(blockIn,
-                 checksumIn, volume.obtainReference())) {
+                 checksumIn, volume.obtainReference(), fileIoProvider)) {
           ris.skipChecksumFully((numChunks - 1) * checksumSize);
           long lastChunkStartPos = (numChunks - 1) * bytesPerChecksum;
           ris.skipDataFully(lastChunkStartPos);
@@ -734,7 +727,8 @@ class BlockPoolSlice {
           // truncate if extra bytes are present without CRC
           if (blockFile.length() > validFileLength) {
             try (RandomAccessFile blockRAF =
-                     new RandomAccessFile(blockFile, "rw")) {
+                     fileIoProvider.getRandomAccessFile(
+                         volume, blockFile, "rw")) {
               // truncate blockFile
               blockRAF.setLength(validFileLength);
             }
@@ -786,12 +780,14 @@ class BlockPoolSlice {
     }
     FileInputStream inputStream = null;
     try {
-      inputStream = new FileInputStream(replicaFile);
+      inputStream = fileIoProvider.getFileInputStream(volume, replicaFile);
       BlockListAsLongs blocksList =
           BlockListAsLongs.readFrom(inputStream, maxDataLength);
-      Iterator<BlockReportReplica> iterator = blocksList.iterator();
-      while (iterator.hasNext()) {
-        BlockReportReplica replica = iterator.next();
+      if (blocksList == null) {
+        return false;
+      }
+
+      for (BlockReportReplica replica : blocksList) {
         switch (replica.getState()) {
         case FINALIZED:
           addReplicaToReplicasMap(replica, tmpReplicaMap, lazyWriteReplicaMap, true);
@@ -828,7 +824,7 @@ class BlockPoolSlice {
       return false;
     }
     finally {
-      if (!replicaFile.delete()) {
+      if (!fileIoProvider.delete(volume, replicaFile)) {
         LOG.info("Failed to delete replica cache file: " +
             replicaFile.getPath());
       }
@@ -842,41 +838,29 @@ class BlockPoolSlice {
         blocksListToPersist.getNumberOfBlocks()== 0) {
       return;
     }
-    File tmpFile = new File(currentDir, REPLICA_CACHE_FILE + ".tmp");
-    if (tmpFile.exists() && !tmpFile.delete()) {
-      LOG.warn("Failed to delete tmp replicas file in " +
-        tmpFile.getPath());
-      return;
-    }
-    File replicaCacheFile = new File(currentDir, REPLICA_CACHE_FILE);
-    if (replicaCacheFile.exists() && !replicaCacheFile.delete()) {
-      LOG.warn("Failed to delete replicas file in " +
-          replicaCacheFile.getPath());
+    final File tmpFile = new File(currentDir, REPLICA_CACHE_FILE + ".tmp");
+    final File replicaCacheFile = new File(currentDir, REPLICA_CACHE_FILE);
+    if (!fileIoProvider.deleteWithExistsCheck(volume, tmpFile) ||
+        !fileIoProvider.deleteWithExistsCheck(volume, replicaCacheFile)) {
       return;
     }
 
     FileOutputStream out = null;
     try {
-      out = new FileOutputStream(tmpFile);
+      out = fileIoProvider.getFileOutputStream(volume, tmpFile);
       blocksListToPersist.writeTo(out);
       out.close();
       // Renaming the tmp file to replicas
-      Files.move(tmpFile, replicaCacheFile);
+      fileIoProvider.moveFile(volume, tmpFile, replicaCacheFile);
     } catch (Exception e) {
       // If write failed, the volume might be bad. Since the cache file is
       // not critical, log the error, delete both the files (tmp and cache)
       // and continue.
       LOG.warn("Failed to write replicas to cache ", e);
-      if (replicaCacheFile.exists() && !replicaCacheFile.delete()) {
-        LOG.warn("Failed to delete replicas file: " +
-            replicaCacheFile.getPath());
-      }
+      fileIoProvider.deleteWithExistsCheck(volume, replicaCacheFile);
     } finally {
       IOUtils.closeStream(out);
-      if (tmpFile.exists() && !tmpFile.delete()) {
-        LOG.warn("Failed to delete tmp file in " +
-            tmpFile.getPath());
-      }
+      fileIoProvider.deleteWithExistsCheck(volume, tmpFile);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
index 97dcf8d..416609d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
@@ -272,8 +272,10 @@ class FsDatasetAsyncDiskService {
       }
 
       File trashDirFile = new File(trashDirectory);
-      if (!trashDirFile.exists() && !trashDirFile.mkdirs()) {
-        LOG.error("Failed to create trash directory " + trashDirectory);
+      try {
+        volume.getFileIoProvider().mkdirsWithExistsCheck(
+            volume, trashDirFile);
+      } catch (IOException e) {
         return false;
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 6065df2..35561cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -21,6 +21,7 @@ import java.io.BufferedOutputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -57,6 +58,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@@ -418,6 +420,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
                               .setDataset(this)
                               .setStorageID(sd.getStorageUuid())
                               .setStorageDirectory(sd)
+                              .setFileIoProvider(datanode.getFileIoProvider())
                               .setConf(this.conf)
                               .build();
     FsVolumeReference ref = fsVolume.obtainReference();
@@ -437,6 +440,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         .setDataset(this)
         .setStorageID(storageUuid)
         .setStorageDirectory(sd)
+        .setFileIoProvider(datanode.getFileIoProvider())
         .setConf(conf)
         .build();
   }
@@ -818,7 +822,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         InputStream blockInStream = info.getDataInputStream(blkOffset);
         try {
           InputStream metaInStream = info.getMetadataInputStream(metaOffset);
-          return new ReplicaInputStreams(blockInStream, metaInStream, ref);
+          return new ReplicaInputStreams(
+              blockInStream, metaInStream, ref, datanode.getFileIoProvider());
         } catch (IOException e) {
           IOUtils.cleanup(null, blockInStream);
           throw e;
@@ -1027,9 +1032,16 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   static void computeChecksum(ReplicaInfo srcReplica, File dstMeta,
       int smallBufferSize, final Configuration conf)
       throws IOException {
-    File srcMeta = new File(srcReplica.getMetadataURI());
-    final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(srcMeta,
-        DFSUtilClient.getIoFileBufferSize(conf));
+    final File srcMeta = new File(srcReplica.getMetadataURI());
+
+    DataChecksum checksum;
+    try (FileInputStream fis =
+             srcReplica.getFileIoProvider().getFileInputStream(
+                 srcReplica.getVolume(), srcMeta)) {
+      checksum = BlockMetadataHeader.readDataChecksum(
+          fis, DFSUtilClient.getIoFileBufferSize(conf), srcMeta);
+    }
+
     final byte[] data = new byte[1 << 16];
     final byte[] crcs = new byte[checksum.getChecksumSize(data.length)];
 
@@ -2161,16 +2173,21 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         return;
       }
 
-      final long diskGS = diskMetaFile != null && diskMetaFile.exists() ?
+      final FileIoProvider fileIoProvider = datanode.getFileIoProvider();
+      final boolean diskMetaFileExists = diskMetaFile != null &&
+          fileIoProvider.exists(vol, diskMetaFile);
+      final boolean diskFileExists = diskFile != null &&
+          fileIoProvider.exists(vol, diskFile);
+
+      final long diskGS = diskMetaFileExists ?
           Block.getGenerationStamp(diskMetaFile.getName()) :
-            HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+          HdfsConstants.GRANDFATHER_GENERATION_STAMP;
 
-      if (diskFile == null || !diskFile.exists()) {
+      if (!diskFileExists) {
         if (memBlockInfo == null) {
           // Block file does not exist and block does not exist in memory
           // If metadata file exists then delete it
-          if (diskMetaFile != null && diskMetaFile.exists()
-              && diskMetaFile.delete()) {
+          if (diskMetaFileExists && fileIoProvider.delete(vol, diskMetaFile)) {
             LOG.warn("Deleted a metadata file without a block "
                 + diskMetaFile.getAbsolutePath());
           }
@@ -2186,8 +2203,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           LOG.warn("Removed block " + blockId
               + " from memory with missing block file on the disk");
           // Finally remove the metadata file
-          if (diskMetaFile != null && diskMetaFile.exists()
-              && diskMetaFile.delete()) {
+          if (diskMetaFileExists && fileIoProvider.delete(vol, diskMetaFile)) {
             LOG.warn("Deleted a metadata file for the deleted block "
                 + diskMetaFile.getAbsolutePath());
           }
@@ -2223,7 +2239,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       // Compare block files
       if (memBlockInfo.blockDataExists()) {
         if (memBlockInfo.getBlockURI().compareTo(diskFile.toURI()) != 0) {
-          if (diskMetaFile.exists()) {
+          if (diskMetaFileExists) {
             if (memBlockInfo.metadataExists()) {
               // We have two sets of block+meta files. Decide which one to
               // keep.
@@ -2239,7 +2255,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
                   memBlockInfo, diskBlockInfo, volumeMap);
             }
           } else {
-            if (!diskFile.delete()) {
+            if (!fileIoProvider.delete(vol, diskFile)) {
               LOG.warn("Failed to delete " + diskFile);
             }
           }
@@ -2278,8 +2294,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           // as the block file, then use the generation stamp from it
           try {
             File memFile = new File(memBlockInfo.getBlockURI());
-            long gs = diskMetaFile != null && diskMetaFile.exists()
-                && diskMetaFile.getParent().equals(memFile.getParent()) ? diskGS
+            long gs = diskMetaFileExists &&
+                diskMetaFile.getParent().equals(memFile.getParent()) ? diskGS
                 : HdfsConstants.GRANDFATHER_GENERATION_STAMP;
 
             LOG.warn("Updating generation stamp for block " + blockId

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
index 563f66a..32759c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
+import java.io.FileDescriptor;
 import java.io.FileInputStream;
 import java.io.FilenameFilter;
 import java.io.IOException;
@@ -80,7 +81,7 @@ public class FsDatasetUtil {
     return matches[0];
   }
 
-  public static FileInputStream openAndSeek(File file, long offset)
+  public static FileDescriptor openAndSeek(File file, long offset)
       throws IOException {
     RandomAccessFile raf = null;
     try {
@@ -88,7 +89,7 @@ public class FsDatasetUtil {
       if (offset > 0) {
         raf.seek(offset);
       }
-      return new FileInputStream(raf.getFD());
+      return raf.getFD();
     } catch(IOException ioe) {
       IOUtils.cleanup(null, raf);
       throw ioe;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index c317715..74ee063 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -19,14 +19,12 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.BufferedWriter;
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.RandomAccessFile;
 import java.net.URI;
 import java.nio.channels.ClosedChannelException;
-import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.nio.file.StandardCopyOption;
 import java.util.Collections;
@@ -46,8 +44,8 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -75,7 +73,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.CloseableReferenceCount;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.StringUtils;
@@ -132,6 +129,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   // limit the visible capacity for tests. If negative, then we just
   // query from the filesystem.
   protected volatile long configuredCapacity;
+  private final FileIoProvider fileIoProvider;
 
   /**
    * Per-volume worker pool that processes new blocks to cache.
@@ -141,8 +139,9 @@ public class FsVolumeImpl implements FsVolumeSpi {
    */
   protected ThreadPoolExecutor cacheExecutor;
   
-  FsVolumeImpl(FsDatasetImpl dataset, String storageID, StorageDirectory sd,
-      Configuration conf) throws IOException {
+  FsVolumeImpl(
+      FsDatasetImpl dataset, String storageID, StorageDirectory sd,
+      FileIoProvider fileIoProvider, Configuration conf) throws IOException {
 
     if (sd.getStorageLocation() == null) {
       throw new IOException("StorageLocation specified for storage directory " +
@@ -162,6 +161,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT));
     this.configuredCapacity = -1;
     this.conf = conf;
+    this.fileIoProvider = fileIoProvider;
     cacheExecutor = initializeCacheExecutor(parent);
   }
 
@@ -664,8 +664,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
      */
     private String getNextSubDir(String prev, File dir)
           throws IOException {
-      List<String> children =
-          IOUtils.listDirectory(dir, SubdirFilter.INSTANCE);
+      List<String> children = fileIoProvider.listDirectory(
+          FsVolumeImpl.this, dir, SubdirFilter.INSTANCE);
       cache = null;
       cacheMs = 0;
       if (children.size() == 0) {
@@ -718,8 +718,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
       }
       File dir = Paths.get(bpidDir.getAbsolutePath(), "current", "finalized",
                     state.curFinalizedDir, state.curFinalizedSubDir).toFile();
-      List<String> entries =
-          IOUtils.listDirectory(dir, BlockFileFilter.INSTANCE);
+      List<String> entries = fileIoProvider.listDirectory(
+          FsVolumeImpl.this, dir, BlockFileFilter.INSTANCE);
       if (entries.size() == 0) {
         entries = null;
       } else {
@@ -839,19 +839,18 @@ public class FsVolumeImpl implements FsVolumeSpi {
     public void save() throws IOException {
       state.lastSavedMs = Time.now();
       boolean success = false;
-      try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(
-                new FileOutputStream(getTempSaveFile(), false), "UTF-8"))) {
+      try (BufferedWriter writer = new BufferedWriter(
+          new OutputStreamWriter(fileIoProvider.getFileOutputStream(
+              FsVolumeImpl.this, getTempSaveFile()), "UTF-8"))) {
         WRITER.writeValue(writer, state);
         success = true;
       } finally {
         if (!success) {
-          if (getTempSaveFile().delete()) {
-            LOG.debug("save({}, {}): error deleting temporary file.",
-                storageID, bpid);
-          }
+          fileIoProvider.delete(FsVolumeImpl.this, getTempSaveFile());
         }
       }
-      Files.move(getTempSaveFile().toPath(), getSaveFile().toPath(),
+      fileIoProvider.move(FsVolumeImpl.this,
+          getTempSaveFile().toPath(), getSaveFile().toPath(),
           StandardCopyOption.ATOMIC_MOVE);
       if (LOG.isTraceEnabled()) {
         LOG.trace("save({}, {}): saved {}", storageID, bpid,
@@ -1042,11 +1041,12 @@ public class FsVolumeImpl implements FsVolumeSpi {
     File finalizedDir = new File(bpCurrentDir,
         DataStorage.STORAGE_DIR_FINALIZED);
     File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
-    if (finalizedDir.exists() && !DatanodeUtil.dirNoFilesRecursive(
-        finalizedDir)) {
+    if (fileIoProvider.exists(this, finalizedDir) &&
+        !DatanodeUtil.dirNoFilesRecursive(this, finalizedDir, fileIoProvider)) {
       return false;
     }
-    if (rbwDir.exists() && FileUtil.list(rbwDir).length != 0) {
+    if (fileIoProvider.exists(this, rbwDir) &&
+        fileIoProvider.list(this, rbwDir).length != 0) {
       return false;
     }
     return true;
@@ -1067,35 +1067,38 @@ public class FsVolumeImpl implements FsVolumeSpi {
         DataStorage.STORAGE_DIR_LAZY_PERSIST);
     File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
     if (force) {
-      DataStorage.fullyDelete(bpDir);
+      fileIoProvider.fullyDelete(this, bpDir);
     } else {
-      if (!rbwDir.delete()) {
+      if (!fileIoProvider.delete(this, rbwDir)) {
         throw new IOException("Failed to delete " + rbwDir);
       }
-      if (!DatanodeUtil.dirNoFilesRecursive(finalizedDir) ||
-          !FileUtil.fullyDelete(finalizedDir)) {
+      if (!DatanodeUtil.dirNoFilesRecursive(
+              this, finalizedDir, fileIoProvider) ||
+          !fileIoProvider.fullyDelete(
+              this, finalizedDir)) {
         throw new IOException("Failed to delete " + finalizedDir);
       }
       if (lazypersistDir.exists() &&
-        ((!DatanodeUtil.dirNoFilesRecursive(lazypersistDir) ||
-          !FileUtil.fullyDelete(lazypersistDir)))) {
+          ((!DatanodeUtil.dirNoFilesRecursive(
+              this, lazypersistDir, fileIoProvider) ||
+              !fileIoProvider.fullyDelete(this, lazypersistDir)))) {
         throw new IOException("Failed to delete " + lazypersistDir);
       }
-      DataStorage.fullyDelete(tmpDir);
-      for (File f : FileUtil.listFiles(bpCurrentDir)) {
-        if (!f.delete()) {
+      fileIoProvider.fullyDelete(this, tmpDir);
+      for (File f : fileIoProvider.listFiles(this, bpCurrentDir)) {
+        if (!fileIoProvider.delete(this, f)) {
           throw new IOException("Failed to delete " + f);
         }
       }
-      if (!bpCurrentDir.delete()) {
+      if (!fileIoProvider.delete(this, bpCurrentDir)) {
         throw new IOException("Failed to delete " + bpCurrentDir);
       }
-      for (File f : FileUtil.listFiles(bpDir)) {
-        if (!f.delete()) {
+      for (File f : fileIoProvider.listFiles(this, bpDir)) {
+        if (!fileIoProvider.delete(this, f)) {
           throw new IOException("Failed to delete " + f);
         }
       }
-      if (!bpDir.delete()) {
+      if (!fileIoProvider.delete(this, bpDir)) {
         throw new IOException("Failed to delete " + bpDir);
       }
     }
@@ -1118,7 +1121,10 @@ public class FsVolumeImpl implements FsVolumeSpi {
 
   private byte[] loadLastPartialChunkChecksum(
       File blockFile, File metaFile) throws IOException {
-    DataChecksum dcs = BlockMetadataHeader.readHeader(metaFile).getChecksum();
+    // readHeader closes the temporary FileInputStream.
+    DataChecksum dcs = BlockMetadataHeader
+        .readHeader(fileIoProvider.getFileInputStream(this, metaFile))
+        .getChecksum();
     final int checksumSize = dcs.getChecksumSize();
     final long onDiskLen = blockFile.length();
     final int bytesPerChecksum = dcs.getBytesPerChecksum();
@@ -1132,7 +1138,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
     int offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
         (int)(onDiskLen / bytesPerChecksum * checksumSize);
     byte[] lastChecksum = new byte[checksumSize];
-    try (RandomAccessFile raf = new RandomAccessFile(metaFile, "r")) {
+    try (RandomAccessFile raf = fileIoProvider.getRandomAccessFile(
+        this, metaFile, "r")) {
       raf.seek(offsetInChecksum);
       raf.read(lastChecksum, 0, checksumSize);
     }
@@ -1246,8 +1253,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
         copyReplicaWithNewBlockIdAndGS(rur, bpid, newBlockId, recoveryId);
     File blockFile = copiedReplicaFiles[1];
     File metaFile = copiedReplicaFiles[0];
-    LocalReplica.truncateBlock(blockFile, metaFile,
-        rur.getNumBytes(), newlength);
+    LocalReplica.truncateBlock(rur.getVolume(), blockFile, metaFile,
+        rur.getNumBytes(), newlength, fileIoProvider);
 
     LocalReplicaInPipeline newReplicaInfo = new ReplicaBuilder(ReplicaState.RBW)
         .setBlockId(newBlockId)
@@ -1283,6 +1290,11 @@ public class FsVolumeImpl implements FsVolumeSpi {
         getFinalizedDir(bpid), report, reportCompiler);
   }
 
+  @Override
+  public FileIoProvider getFileIoProvider() {
+    return fileIoProvider;
+  }
+
   private LinkedList<ScanInfo> compileReport(File bpFinalizedDir,
       File dir, LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
         throws InterruptedException {
@@ -1291,7 +1303,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
 
     List <String> fileNames;
     try {
-      fileNames = IOUtils.listDirectory(dir, BlockDirFilter.INSTANCE);
+      fileNames = fileIoProvider.listDirectory(
+          this, dir, BlockDirFilter.INSTANCE);
     } catch (IOException ioe) {
       LOG.warn("Exception occured while compiling report: ", ioe);
       // Initiate a check on disk failure.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
index a1f7e91..5371eda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
 
 /**
  * This class is to be used as a builder for {@link FsVolumeImpl} objects.
@@ -31,6 +32,7 @@ public class FsVolumeImplBuilder {
   private String storageID;
   private StorageDirectory sd;
   private Configuration conf;
+  private FileIoProvider fileIoProvider;
 
   public FsVolumeImplBuilder() {
     dataset = null;
@@ -59,7 +61,15 @@ public class FsVolumeImplBuilder {
     return this;
   }
 
+  FsVolumeImplBuilder setFileIoProvider(FileIoProvider fileIoProvider) {
+    this.fileIoProvider = fileIoProvider;
+    return this;
+  }
+
   FsVolumeImpl build() throws IOException {
-    return new FsVolumeImpl(dataset, storageID, sd, conf);
+    return new FsVolumeImpl(
+        dataset, storageID, sd,
+        fileIoProvider != null ? fileIoProvider : new FileIoProvider(null),
+        conf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
index e963d41..20cec6a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
@@ -701,7 +701,7 @@ public class TestFileAppend{
       ReplicaBeingWritten rbw =
           (ReplicaBeingWritten)replicaHandler.getReplica();
       ReplicaOutputStreams
-          outputStreams = rbw.createStreams(false, DEFAULT_CHECKSUM, 300);
+          outputStreams = rbw.createStreams(false, DEFAULT_CHECKSUM);
       OutputStream dataOutput = outputStreams.getDataOut();
 
       byte[] appendBytes = new byte[1];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index ae52905..a0041dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -122,6 +122,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   static final byte[] nullCrcFileData;
 
   private final AutoCloseableLock datasetLock;
+  private final FileIoProvider fileIoProvider;
 
   static {
     DataChecksum checksum = DataChecksum.newDataChecksum(
@@ -260,7 +261,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
     @Override
     synchronized public ReplicaOutputStreams createStreams(boolean isCreate, 
-        DataChecksum requestedChecksum, long slowLogThresholdMs)
+        DataChecksum requestedChecksum)
         throws IOException {
       if (finalized) {
         throw new IOException("Trying to write to a finalized replica "
@@ -268,7 +269,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       } else {
         SimulatedOutputStream crcStream = new SimulatedOutputStream();
         return new ReplicaOutputStreams(oStream, crcStream, requestedChecksum,
-            volume.isTransientStorage(), slowLogThresholdMs);
+            volume, fileIoProvider);
       }
     }
 
@@ -474,9 +475,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   
   static class SimulatedVolume implements FsVolumeSpi {
     private final SimulatedStorage storage;
+    private final FileIoProvider fileIoProvider;
 
-    SimulatedVolume(final SimulatedStorage storage) {
+    SimulatedVolume(final SimulatedStorage storage,
+                    final FileIoProvider fileIoProvider) {
       this.storage = storage;
+      this.fileIoProvider = fileIoProvider;
     }
 
     @Override
@@ -560,6 +564,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     }
 
     @Override
+    public FileIoProvider getFileIoProvider() {
+      return fileIoProvider;
+    }
+
+    @Override
     public VolumeCheckResult check(VolumeCheckContext context)
         throws Exception {
       return VolumeCheckResult.HEALTHY;
@@ -590,10 +599,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     }
 
     registerMBean(datanodeUuid);
+    this.fileIoProvider = new FileIoProvider(conf);
     this.storage = new SimulatedStorage(
         conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY),
         conf.getEnum(CONFIG_PROPERTY_STATE, DEFAULT_STATE));
-    this.volume = new SimulatedVolume(this.storage);
+    this.volume = new SimulatedVolume(this.storage, this.fileIoProvider);
     this.datasetLock = new AutoCloseableLock();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 8439991..619eda0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -673,7 +673,7 @@ public class TestBlockRecovery {
     ReplicaOutputStreams streams = null;
     try {
       streams = replicaInfo.createStreams(true,
-          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512), 300);
+          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512));
       streams.getChecksumOut().write('a');
       dn.data.initReplicaRecovery(new RecoveringBlock(block, null, RECOVERY_ID+1));
       BlockRecoveryWorker.RecoveryTaskContiguous RecoveryTaskContiguous =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index d7c8383..cc0915d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -905,6 +905,11 @@ public class TestDirectoryScanner {
       return null;
     }
 
+    @Override
+    public FileIoProvider getFileIoProvider() {
+      return null;
+    }
+
 
     @Override
     public VolumeCheckResult check(VolumeCheckContext context)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
index fa980c2..4e724bc7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
@@ -83,7 +83,7 @@ public class TestSimulatedFSDataset {
       ReplicaInPipeline bInfo = fsdataset.createRbw(
           StorageType.DEFAULT, b, false).getReplica();
       ReplicaOutputStreams out = bInfo.createStreams(true,
-          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512), 300);
+          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512));
       try {
         OutputStream dataOut  = out.getDataOut();
         assertEquals(0, fsdataset.getLength(b));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index 2417c9d..5cd86e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -134,7 +134,7 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   @Override
   public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
       long ckoff) throws IOException {
-    return new ReplicaInputStreams(null, null, null);
+    return new ReplicaInputStreams(null, null, null, null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
index 6fa2830..5c172e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
@@ -58,10 +58,10 @@ public class ExternalReplicaInPipeline implements ReplicaInPipeline {
 
   @Override
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum, long slowLogThresholdMs)
+      DataChecksum requestedChecksum)
       throws IOException {
-    return new ReplicaOutputStreams(null, null, requestedChecksum, false,
-        slowLogThresholdMs);
+    return new ReplicaOutputStreams(null, null, requestedChecksum,
+        null, null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
index 2753a61..e607de5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
@@ -26,6 +26,7 @@ import java.util.LinkedList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
 import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
@@ -115,6 +116,11 @@ public class ExternalVolumeImpl implements FsVolumeSpi {
   }
 
   @Override
+  public FileIoProvider getFileIoProvider() {
+    return null;
+  }
+
+  @Override
   public VolumeCheckResult check(VolumeCheckContext context)
       throws Exception {
     return VolumeCheckResult.HEALTHY;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
index a089d39..3bac7b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
@@ -99,6 +99,8 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
         .add(DFSConfigKeys.DFS_DATANODE_STARTUP_KEY);
     configurationPropsToSkipCompare
         .add(DFSConfigKeys.DFS_NAMENODE_STARTUP_KEY);
+    configurationPropsToSkipCompare
+        .add(DFSConfigKeys.DFS_DATANODE_FILE_IO_EVENTS_CLASS_KEY);
 
     // Allocate
     xmlPropsToSkipCompare = new HashSet<String>();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/29] hadoop git commit: HDFS-10684. WebHDFS DataNode calls fail without parameter createparent. Contributed by John Zhuge.

Posted by as...@apache.org.
HDFS-10684. WebHDFS DataNode calls fail without parameter createparent. Contributed by John Zhuge.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fbdbbd57
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fbdbbd57
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fbdbbd57

Branch: refs/heads/YARN-5085
Commit: fbdbbd57cdc3d8c778fca9266a7cadf298c8ff6c
Parents: e24a923
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Dec 13 18:01:31 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Dec 13 18:01:31 2016 -0800

----------------------------------------------------------------------
 .../hdfs/web/resources/CreateParentParam.java   |  2 +-
 .../hdfs/web/resources/OverwriteParam.java      |  2 +-
 .../hdfs/web/TestWebHdfsFileSystemContract.java | 40 ++++++++++++++++++++
 3 files changed, 42 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbdbbd57/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateParentParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateParentParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateParentParam.java
index eaa5e8d..029efa0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateParentParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateParentParam.java
@@ -39,7 +39,7 @@ public class CreateParentParam extends BooleanParam {
    * @param str a string representation of the parameter value.
    */
   public CreateParentParam(final String str) {
-    this(DOMAIN.parse(str));
+    this(DOMAIN.parse(str == null ? DEFAULT : str));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbdbbd57/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java
index 9610b93..d7f5fb0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java
@@ -39,7 +39,7 @@ public class OverwriteParam extends BooleanParam {
    * @param str a string representation of the parameter value.
    */
   public OverwriteParam(final String str) {
-    this(DOMAIN.parse(str));
+    super(DOMAIN, DOMAIN.parse(str == null ? DEFAULT : str));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbdbbd57/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
index a68b1ac..4854471 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
@@ -25,6 +25,7 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.URL;
+import java.text.MessageFormat;
 import java.util.Arrays;
 import java.util.Map;
 import java.util.Random;
@@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.web.resources.NamenodeAddressParam;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -531,6 +533,44 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
     }
   }
 
+  public void testDatanodeCreateMissingParameter() throws IOException {
+    final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem) fs;
+    final Path testDir = new Path(MessageFormat.format("/test/{0}/{1}",
+        TestWebHdfsFileSystemContract.class,
+        GenericTestUtils.getMethodName()));
+    assertTrue(webhdfs.mkdirs(testDir));
+
+    for (String dnCreateParam : new String[]{
+        CreateFlagParam.NAME,
+        CreateParentParam.NAME,
+        OverwriteParam.NAME
+    }) {
+      final HttpOpParam.Op op = PutOpParam.Op.CREATE;
+      final Path newfile = new Path(testDir, "newfile_" + dnCreateParam);
+      final URL url = webhdfs.toUrl(op, newfile);
+      HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      conn.setRequestMethod(op.getType().toString());
+      conn.setDoOutput(false);
+      conn.setInstanceFollowRedirects(false);
+      conn.connect();
+      final String redirect = conn.getHeaderField("Location");
+      conn.disconnect();
+
+      //remove createparent
+      WebHdfsFileSystem.LOG.info("redirect = " + redirect);
+      String re = "&" + dnCreateParam + "=[^&]*";
+      String modified = redirect.replaceAll(re, "");
+      WebHdfsFileSystem.LOG.info("modified = " + modified);
+
+      //connect to datanode
+      conn = (HttpURLConnection)new URL(modified).openConnection();
+      conn.setRequestMethod(op.getType().toString());
+      conn.setDoOutput(op.getDoOutput());
+      conn.connect();
+      assertEquals(HttpServletResponse.SC_CREATED, conn.getResponseCode());
+    }
+  }
+
   @Test
   public void testAccess() throws IOException, InterruptedException {
     Path p1 = new Path("/pathX");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/29] hadoop git commit: HADOOP-13890. Maintain HTTP/host as SPNEGO SPN support and fix KerberosName parsing. Contributed by Xiaoyu Yao.

Posted by as...@apache.org.
HADOOP-13890. Maintain HTTP/host as SPNEGO SPN support and fix KerberosName parsing. Contributed by Xiaoyu Yao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f5e0bd30
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f5e0bd30
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f5e0bd30

Branch: refs/heads/YARN-5085
Commit: f5e0bd30fde654ed48fe73e5c0523030365385a4
Parents: 6ba9587
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Wed Dec 14 13:41:40 2016 -0800
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Wed Dec 14 13:45:21 2016 -0800

----------------------------------------------------------------------
 .../server/KerberosAuthenticationHandler.java   | 19 +++++++++--------
 .../authentication/util/KerberosName.java       |  4 ++--
 .../authentication/util/TestKerberosName.java   | 22 ++++++++++++++++++++
 .../delegation/web/TestWebDelegationToken.java  |  4 ++++
 4 files changed, 38 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5e0bd30/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
index f51bbd6..e0ee227 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
@@ -73,7 +73,7 @@ import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
  * </ul>
  */
 public class KerberosAuthenticationHandler implements AuthenticationHandler {
-  private static final Logger LOG = LoggerFactory.getLogger(
+  public static final Logger LOG = LoggerFactory.getLogger(
       KerberosAuthenticationHandler.class);
 
   /**
@@ -274,14 +274,14 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
         loginContexts.add(loginContext);
         KerberosName kerbName = new KerberosName(spnegoPrincipal);
         if (kerbName.getHostName() != null
-            && kerbName.getRealm() != null
             && kerbName.getServiceName() != null
             && kerbName.getServiceName().equals("HTTP")) {
-          LOG.trace("Map server: {} to principal: {}", kerbName.getHostName(),
+          boolean added = serverPrincipalMap.put(kerbName.getHostName(),
               spnegoPrincipal);
-          serverPrincipalMap.put(kerbName.getHostName(), spnegoPrincipal);
+          LOG.info("Map server: {} to principal: [{}], added = {}",
+              kerbName.getHostName(), spnegoPrincipal, added);
         } else {
-          LOG.warn("HTTP principal: {} is invalid for SPNEGO!",
+          LOG.warn("HTTP principal: [{}] is invalid for SPNEGO!",
               spnegoPrincipal);
         }
       }
@@ -419,8 +419,8 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
               @Override
               public AuthenticationToken run() throws Exception {
                 if (LOG.isTraceEnabled()) {
-                  LOG.trace("SPNEGO with principals: {}",
-                      serverPrincipals.toString());
+                  LOG.trace("SPNEGO with server principals: {} for {}",
+                      serverPrincipals.toString(), serverName);
                 }
                 AuthenticationToken token = null;
                 Exception lastException = null;
@@ -464,7 +464,7 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
     GSSCredential gssCreds = null;
     AuthenticationToken token = null;
     try {
-      LOG.trace("SPNEGO initiated with principal {}", serverPrincipal);
+      LOG.trace("SPNEGO initiated with server principal [{}]", serverPrincipal);
       gssCreds = this.gssManager.createCredential(
           this.gssManager.createName(serverPrincipal,
               KerberosUtil.getOidInstance("NT_GSS_KRB5_PRINCIPAL")),
@@ -491,7 +491,8 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
         String userName = kerberosName.getShortName();
         token = new AuthenticationToken(userName, clientPrincipal, getType());
         response.setStatus(HttpServletResponse.SC_OK);
-        LOG.trace("SPNEGO completed for principal [{}]", clientPrincipal);
+        LOG.trace("SPNEGO completed for client principal [{}]",
+            clientPrincipal);
       }
     } finally {
       if (gssContext != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5e0bd30/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
index 0b668f1..6d15b6b 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
@@ -54,7 +54,7 @@ public class KerberosName {
    * A pattern that matches a Kerberos name with at most 2 components.
    */
   private static final Pattern nameParser =
-    Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)");
+      Pattern.compile("([^/@]+)(/([^/@]+))?(@([^/@]+))?");
 
   /**
    * A pattern that matches a string with out '$' and then a single
@@ -109,7 +109,7 @@ public class KerberosName {
     } else {
       serviceName = match.group(1);
       hostName = match.group(3);
-      realm = match.group(4);
+      realm = match.group(5);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5e0bd30/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosName.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosName.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosName.java
index f85b3e1..a375bc9 100644
--- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosName.java
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosName.java
@@ -83,6 +83,28 @@ public class TestKerberosName {
   }
 
   @Test
+  public void testParsing() throws Exception {
+    final String principalNameFull = "HTTP/abc.com@EXAMPLE.COM";
+    final String principalNameWoRealm = "HTTP/abc.com";
+    final String principalNameWoHost = "HTTP@EXAMPLE.COM";
+
+    final KerberosName kerbNameFull = new KerberosName(principalNameFull);
+    Assert.assertEquals("HTTP", kerbNameFull.getServiceName());
+    Assert.assertEquals("abc.com", kerbNameFull.getHostName());
+    Assert.assertEquals("EXAMPLE.COM", kerbNameFull.getRealm());
+
+    final KerberosName kerbNamewoRealm = new KerberosName(principalNameWoRealm);
+    Assert.assertEquals("HTTP", kerbNamewoRealm.getServiceName());
+    Assert.assertEquals("abc.com", kerbNamewoRealm.getHostName());
+    Assert.assertEquals(null, kerbNamewoRealm.getRealm());
+
+    final KerberosName kerbNameWoHost = new KerberosName(principalNameWoHost);
+    Assert.assertEquals("HTTP", kerbNameWoHost.getServiceName());
+    Assert.assertEquals(null, kerbNameWoHost.getHostName());
+    Assert.assertEquals("EXAMPLE.COM", kerbNameWoHost.getRealm());
+  }
+
+  @Test
   public void testToLowerCase() throws Exception {
     String rules =
         "RULE:[1:$1]/L\n" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5e0bd30/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
index 89f15da..7319e4c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHa
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
 import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
 import org.eclipse.jetty.servlet.ServletContextHandler;
@@ -197,6 +199,8 @@ public class TestWebDelegationToken {
     UserGroupInformation.setConfiguration(conf);
 
     jetty = createJettyServer();
+    GenericTestUtils.setLogLevel(KerberosAuthenticationHandler.LOG,
+        Level.TRACE);
   }
 
   @After


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/29] hadoop git commit: HDFS-11160. VolumeScanner reports write-in-progress replicas as corrupt incorrectly. Contributed by Wei-Chiu Chuang and Yongjun Zhang.

Posted by as...@apache.org.
HDFS-11160. VolumeScanner reports write-in-progress replicas as corrupt incorrectly. Contributed by Wei-Chiu Chuang and Yongjun Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/aebb9127
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/aebb9127
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/aebb9127

Branch: refs/heads/YARN-5085
Commit: aebb9127bae872835d057e1c6a6e6b3c6a8be6cd
Parents: 6fce191
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Thu Dec 15 16:32:50 2016 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Thu Dec 15 16:32:50 2016 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/BlockScanner.java      |  10 +-
 .../hdfs/server/datanode/BlockSender.java       |  18 +++-
 .../hdfs/server/datanode/FinalizedReplica.java  |  29 ++++++
 .../hdfs/server/datanode/VolumeScanner.java     |   9 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |  11 ++
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |  18 +++-
 .../server/datanode/SimulatedFSDataset.java     |   6 ++
 .../hdfs/server/datanode/TestBlockScanner.java  | 100 +++++++++++++++++++
 .../server/datanode/TestDirectoryScanner.java   |   6 ++
 .../datanode/extdataset/ExternalVolumeImpl.java |   7 ++
 10 files changed, 203 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
index 21484fb..8081895 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
@@ -64,7 +64,15 @@ public class BlockScanner {
   /**
    * The scanner configuration.
    */
-  private final Conf conf;
+  private Conf conf;
+
+  @VisibleForTesting
+  void setConf(Conf conf) {
+    this.conf = conf;
+    for (Entry<String, VolumeScanner> entry : scanners.entrySet()) {
+      entry.getValue().setConf(conf);
+    }
+  }
 
   /**
    * The cached scanner configuration.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index 99597dc..203ee35 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -240,14 +240,23 @@ class BlockSender implements java.io.Closeable {
         Preconditions.checkArgument(sendChecksum,
             "If verifying checksum, currently must also send it.");
       }
-      
+
+      // if there is a append write happening right after the BlockSender
+      // is constructed, the last partial checksum maybe overwritten by the
+      // append, the BlockSender need to use the partial checksum before
+      // the append write.
+      ChunkChecksum chunkChecksum = null;
       final long replicaVisibleLength;
       try(AutoCloseableLock lock = datanode.data.acquireDatasetLock()) {
         replica = getReplica(block, datanode);
         replicaVisibleLength = replica.getVisibleLength();
+        if (replica instanceof FinalizedReplica) {
+          // Load last checksum in case the replica is being written
+          // concurrently
+          final FinalizedReplica frep = (FinalizedReplica) replica;
+          chunkChecksum = frep.getLastChecksumAndDataLen();
+        }
       }
-      // if there is a write in progress
-      ChunkChecksum chunkChecksum = null;
       if (replica.getState() == ReplicaState.RBW) {
         final ReplicaInPipeline rbw = (ReplicaInPipeline) replica;
         waitForMinLength(rbw, startOffset + length);
@@ -473,7 +482,7 @@ class BlockSender implements java.io.Closeable {
               bytesOnDisk));
     }
   }
-  
+
   /**
    * Converts an IOExcpetion (not subclasses) to SocketException.
    * This is typically done to indicate to upper layers that the error 
@@ -547,7 +556,6 @@ class BlockSender implements java.io.Closeable {
       if (lastDataPacket && lastChunkChecksum != null) {
         int start = checksumOff + checksumDataLen - checksumSize;
         byte[] updatedChecksum = lastChunkChecksum.getChecksum();
-        
         if (updatedChecksum != null) {
           System.arraycopy(updatedChecksum, 0, buf, start, checksumSize);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
index 81a4ab4..e3e0450 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -113,4 +115,31 @@ public class FinalizedReplica extends LocalReplica {
     throw new UnsupportedOperationException("Replica of type " + getState() +
         " does not support createInfo");
   }
+
+  /**
+   * gets the last chunk checksum and the length of the block corresponding
+   * to that checksum.
+   * Note, need to be called with the FsDataset lock acquired. May improve to
+   * lock only the FsVolume in the future.
+   * @throws IOException
+   */
+  public ChunkChecksum getLastChecksumAndDataLen() throws IOException {
+    ChunkChecksum chunkChecksum = null;
+    try {
+      byte[] lastChecksum = getVolume().loadLastPartialChunkChecksum(
+          getBlockFile(), getMetaFile());
+      if (lastChecksum != null) {
+        chunkChecksum =
+            new ChunkChecksum(getVisibleLength(), lastChecksum);
+      }
+    } catch (FileNotFoundException e) {
+      // meta file is lost. Try to continue anyway.
+      DataNode.LOG.warn("meta file " + getMetaFile() +
+          " is missing!");
+    } catch (IOException ioe) {
+      DataNode.LOG.warn("Unable to read checksum from meta file " +
+          getMetaFile(), ioe);
+    }
+    return chunkChecksum;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
index 1e44fb6..8b29fce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
@@ -69,7 +69,12 @@ public class VolumeScanner extends Thread {
   /**
    * The configuration.
    */
-  private final Conf conf;
+  private Conf conf;
+
+  @VisibleForTesting
+  void setConf(Conf conf) {
+    this.conf = conf;
+  }
 
   /**
    * The DataNode this VolumEscanner is associated with.
@@ -429,6 +434,7 @@ public class VolumeScanner extends Thread {
     if (block == null) {
       return -1; // block not found.
     }
+    LOG.debug("start scanning block {}", block);
     BlockSender blockSender = null;
     try {
       blockSender = new BlockSender(block, 0, -1,
@@ -610,6 +616,7 @@ public class VolumeScanner extends Thread {
               break;
             }
             if (timeout > 0) {
+              LOG.debug("{}: wait for {} milliseconds", this, timeout);
               wait(timeout);
               if (stopping) {
                 break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index 4947ecf..8aa2fd9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@ -402,6 +402,17 @@ public interface FsVolumeSpi
   }
 
   /**
+   * Load last partial chunk checksum from checksum file.
+   * Need to be called with FsDataset lock acquired.
+   * @param blockFile
+   * @param metaFile
+   * @return the last partial checksum
+   * @throws IOException
+   */
+  byte[] loadLastPartialChunkChecksum(File blockFile, File metaFile)
+      throws IOException;
+
+  /**
    * Compile a list of {@link ScanInfo} for the blocks in
    * the block pool with id {@code bpid}.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 74ee063..e1bc886 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -1119,7 +1119,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
   }
 
 
-  private byte[] loadLastPartialChunkChecksum(
+  @Override
+  public byte[] loadLastPartialChunkChecksum(
       File blockFile, File metaFile) throws IOException {
     // readHeader closes the temporary FileInputStream.
     DataChecksum dcs = BlockMetadataHeader
@@ -1135,13 +1136,22 @@ public class FsVolumeImpl implements FsVolumeSpi {
       return null;
     }
 
-    int offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
-        (int)(onDiskLen / bytesPerChecksum * checksumSize);
+    long offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
+        (onDiskLen / bytesPerChecksum) * checksumSize;
     byte[] lastChecksum = new byte[checksumSize];
     try (RandomAccessFile raf = fileIoProvider.getRandomAccessFile(
         this, metaFile, "r")) {
       raf.seek(offsetInChecksum);
-      raf.read(lastChecksum, 0, checksumSize);
+      int readBytes = raf.read(lastChecksum, 0, checksumSize);
+      if (readBytes == -1) {
+        throw new IOException("Expected to read " + checksumSize +
+            " bytes from offset " + offsetInChecksum +
+            " but reached end of file.");
+      } else if (readBytes != checksumSize) {
+        throw new IOException("Expected to read " + checksumSize +
+            " bytes from offset " + offsetInChecksum + " but read " +
+            readBytes + " bytes.");
+      }
     }
     return lastChecksum;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index a0041dd..d3efe48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -557,6 +557,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     }
 
     @Override
+    public byte[] loadLastPartialChunkChecksum(
+        File blockFile, File metaFile) throws IOException {
+      return null;
+    }
+
+    @Override
     public LinkedList<ScanInfo> compileReport(String bpid,
         LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
         throws InterruptedException, IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
index 6d35cc5..b627800 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
@@ -36,8 +36,12 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeoutException;
 
 import com.google.common.base.Supplier;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils.MaterializedReplica;
@@ -870,4 +874,100 @@ public class TestBlockScanner {
     }
     info.sem.release(1);
   }
+
+  /**
+   * Test concurrent append and scan.
+   * @throws Exception
+   */
+  @Test(timeout=120000)
+  public void testAppendWhileScanning() throws Exception {
+    GenericTestUtils.setLogLevel(DataNode.LOG, Level.ALL);
+    Configuration conf = new Configuration();
+    // throttle the block scanner: 1MB per second
+    conf.setLong(DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND, 1048576);
+    // Set a really long scan period.
+    conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L);
+    conf.set(INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER,
+        TestScanResultHandler.class.getName());
+    conf.setLong(INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS, 0L);
+    final int numExpectedFiles = 1;
+    final int numExpectedBlocks = 1;
+    final int numNameServices = 1;
+    // the initial file length can not be too small.
+    // Otherwise checksum file stream buffer will be pre-filled and
+    // BlockSender will not see the updated checksum.
+    final int initialFileLength = 2*1024*1024+100;
+    final TestContext ctx = new TestContext(conf, numNameServices);
+    // create one file, with one block.
+    ctx.createFiles(0, numExpectedFiles, initialFileLength);
+    final TestScanResultHandler.Info info =
+        TestScanResultHandler.getInfo(ctx.volumes.get(0));
+    String storageID = ctx.volumes.get(0).getStorageID();
+    synchronized (info) {
+      info.sem = new Semaphore(numExpectedBlocks*2);
+      info.shouldRun = true;
+      info.notify();
+    }
+    // VolumeScanner scans the first block when DN starts.
+    // Due to throttler, this should take approximately 2 seconds.
+    waitForRescan(info, numExpectedBlocks);
+
+    // update throttler to schedule rescan immediately.
+    // this number must be larger than initial file length, otherwise
+    // throttler prevents immediate rescan.
+    conf.setLong(DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND,
+        initialFileLength+32*1024);
+    BlockScanner.Conf newConf = new BlockScanner.Conf(conf);
+    ctx.datanode.getBlockScanner().setConf(newConf);
+    // schedule the first block for scanning
+    ExtendedBlock first = ctx.getFileBlock(0, 0);
+    ctx.datanode.getBlockScanner().markSuspectBlock(storageID, first);
+
+    // append the file before VolumeScanner completes scanning the block,
+    // which takes approximately 2 seconds to complete.
+    FileSystem fs = ctx.cluster.getFileSystem();
+    FSDataOutputStream os = fs.append(ctx.getPath(0));
+    long seed = -1;
+    int size = 200;
+    final byte[] bytes = AppendTestUtil.randomBytes(seed, size);
+    os.write(bytes);
+    os.hflush();
+    os.close();
+    fs.close();
+
+    // verify that volume scanner does not find bad blocks after append.
+    waitForRescan(info, numExpectedBlocks);
+
+    GenericTestUtils.setLogLevel(DataNode.LOG, Level.INFO);
+  }
+
+  private void waitForRescan(final TestScanResultHandler.Info info,
+      final int numExpectedBlocks)
+      throws TimeoutException, InterruptedException {
+    LOG.info("Waiting for the first 1 blocks to be scanned.");
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        synchronized (info) {
+          if (info.blocksScanned >= numExpectedBlocks) {
+            LOG.info("info = {}.  blockScanned has now reached 1.", info);
+            return true;
+          } else {
+            LOG.info("info = {}.  Waiting for blockScanned to reach 1.", info);
+            return false;
+          }
+        }
+      }
+    }, 1000, 30000);
+
+    synchronized (info) {
+      assertEquals("Expected 1 good block.",
+          numExpectedBlocks, info.goodBlocks.size());
+      info.goodBlocks.clear();
+      assertEquals("Expected 1 blocksScanned",
+          numExpectedBlocks, info.blocksScanned);
+      assertEquals("Did not expect bad blocks.", 0, info.badBlocks.size());
+      info.blocksScanned = 0;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index cc0915d..9b0aa82 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -899,6 +899,12 @@ public class TestDirectoryScanner {
     }
 
     @Override
+    public byte[] loadLastPartialChunkChecksum(
+        File blockFile, File metaFile) throws IOException {
+      return null;
+    }
+
+    @Override
     public LinkedList<ScanInfo> compileReport(String bpid,
         LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
         throws InterruptedException, IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aebb9127/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
index e607de5..2d33e20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.server.datanode.extdataset;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.URI;
 import java.nio.channels.ClosedChannelException;
@@ -109,6 +110,12 @@ public class ExternalVolumeImpl implements FsVolumeSpi {
   }
 
   @Override
+  public byte[] loadLastPartialChunkChecksum(
+      File blockFile, File metaFile) throws IOException {
+    return null;
+  }
+
+  @Override
   public LinkedList<ScanInfo> compileReport(String bpid,
       LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
       throws InterruptedException, IOException {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/29] hadoop git commit: HDFS-10958. Add instrumentation hooks around Datanode disk IO.

Posted by as...@apache.org.
HDFS-10958. Add instrumentation hooks around Datanode disk IO.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6ba9587d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6ba9587d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6ba9587d

Branch: refs/heads/YARN-5085
Commit: 6ba9587d370fbf39c129c08c00ebbb894ccc1389
Parents: 72bff19
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Dec 14 11:18:58 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Wed Dec 14 11:18:58 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/io/nativeio/NativeIO.java |   40 +-
 .../server/datanode/BlockMetadataHeader.java    |   29 +-
 .../dev-support/findbugsExcludeFile.xml         |   27 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    2 +
 .../hdfs/server/datanode/BlockReceiver.java     |   14 +-
 .../hdfs/server/datanode/BlockSender.java       |   10 +-
 .../server/datanode/CountingFileIoEvents.java   |  107 ++
 .../hadoop/hdfs/server/datanode/DataNode.java   |   12 +
 .../hdfs/server/datanode/DataNodeMXBean.java    |    5 +
 .../hdfs/server/datanode/DataStorage.java       |    6 +
 .../hdfs/server/datanode/DatanodeUtil.java      |   19 +-
 .../server/datanode/DefaultFileIoEvents.java    |   67 ++
 .../hdfs/server/datanode/FileIoEvents.java      |   97 ++
 .../hdfs/server/datanode/FileIoProvider.java    | 1006 ++++++++++++++++++
 .../hdfs/server/datanode/LocalReplica.java      |  133 +--
 .../server/datanode/LocalReplicaInPipeline.java |   43 +-
 .../hdfs/server/datanode/ReplicaInPipeline.java |    4 +-
 .../hdfs/server/datanode/ReplicaInfo.java       |   17 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |    3 +
 .../datanode/fsdataset/ReplicaInputStreams.java |   11 +-
 .../fsdataset/ReplicaOutputStreams.java         |   72 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |  122 +--
 .../impl/FsDatasetAsyncDiskService.java         |    6 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   46 +-
 .../datanode/fsdataset/impl/FsDatasetUtil.java  |    5 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   89 +-
 .../fsdataset/impl/FsVolumeImplBuilder.java     |   12 +-
 .../org/apache/hadoop/hdfs/TestFileAppend.java  |    2 +-
 .../server/datanode/SimulatedFSDataset.java     |   18 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |    2 +-
 .../server/datanode/TestDirectoryScanner.java   |    5 +
 .../server/datanode/TestSimulatedFSDataset.java |    2 +-
 .../extdataset/ExternalDatasetImpl.java         |    2 +-
 .../extdataset/ExternalReplicaInPipeline.java   |    6 +-
 .../datanode/extdataset/ExternalVolumeImpl.java |    6 +
 .../hadoop/tools/TestHdfsConfigFields.java      |    2 +
 36 files changed, 1684 insertions(+), 365 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
index a123f18..f3ff1c7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
@@ -742,47 +742,19 @@ public class NativeIO {
   }
 
   /**
-   * Create a FileInputStream that shares delete permission on the
-   * file opened, i.e. other process can delete the file the
-   * FileInputStream is reading. Only Windows implementation uses
-   * the native interface.
-   */
-  public static FileInputStream getShareDeleteFileInputStream(File f)
-      throws IOException {
-    if (!Shell.WINDOWS) {
-      // On Linux the default FileInputStream shares delete permission
-      // on the file opened.
-      //
-      return new FileInputStream(f);
-    } else {
-      // Use Windows native interface to create a FileInputStream that
-      // shares delete permission on the file opened.
-      //
-      FileDescriptor fd = Windows.createFile(
-          f.getAbsolutePath(),
-          Windows.GENERIC_READ,
-          Windows.FILE_SHARE_READ |
-              Windows.FILE_SHARE_WRITE |
-              Windows.FILE_SHARE_DELETE,
-          Windows.OPEN_EXISTING);
-      return new FileInputStream(fd);
-    }
-  }
-
-  /**
-   * Create a FileInputStream that shares delete permission on the
+   * Create a FileDescriptor that shares delete permission on the
    * file opened at a given offset, i.e. other process can delete
-   * the file the FileInputStream is reading. Only Windows implementation
+   * the file the FileDescriptor is reading. Only Windows implementation
    * uses the native interface.
    */
-  public static FileInputStream getShareDeleteFileInputStream(File f, long seekOffset)
-      throws IOException {
+  public static FileDescriptor getShareDeleteFileDescriptor(
+      File f, long seekOffset) throws IOException {
     if (!Shell.WINDOWS) {
       RandomAccessFile rf = new RandomAccessFile(f, "r");
       if (seekOffset > 0) {
         rf.seek(seekOffset);
       }
-      return new FileInputStream(rf.getFD());
+      return rf.getFD();
     } else {
       // Use Windows native interface to create a FileInputStream that
       // shares delete permission on the file opened, and set it to the
@@ -797,7 +769,7 @@ public class NativeIO {
           NativeIO.Windows.OPEN_EXISTING);
       if (seekOffset > 0)
         NativeIO.Windows.setFilePointer(fd, seekOffset, NativeIO.Windows.FILE_BEGIN);
-      return new FileInputStream(fd);
+      return fd;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
index eb19492..738f496 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
@@ -31,7 +31,6 @@ import java.nio.channels.FileChannel;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -79,18 +78,15 @@ public class BlockMetadataHeader {
 
   /**
    * Read the checksum header from the meta file.
+   * inputStream must be closed by the caller.
    * @return the data checksum obtained from the header.
    */
-  public static DataChecksum readDataChecksum(File metaFile, int bufSize)
+  public static DataChecksum readDataChecksum(
+      FileInputStream inputStream, int bufSize, File metaFile)
       throws IOException {
-    DataInputStream in = null;
-    try {
-      in = new DataInputStream(new BufferedInputStream(
-        new FileInputStream(metaFile), bufSize));
-      return readDataChecksum(in, metaFile);
-    } finally {
-      IOUtils.closeStream(in);
-    }
+    DataInputStream in = new DataInputStream(new BufferedInputStream(
+        inputStream, bufSize));
+    return readDataChecksum(in, metaFile);
   }
 
   /**
@@ -111,6 +107,7 @@ public class BlockMetadataHeader {
 
   /**
    * Read the header without changing the position of the FileChannel.
+   * This is used by the client for short-circuit reads.
    *
    * @param fc The FileChannel to read.
    * @return the Metadata Header.
@@ -144,18 +141,16 @@ public class BlockMetadataHeader {
 
   /**
    * Reads header at the top of metadata file and returns the header.
+   * Closes the input stream after reading the header.
    *
    * @return metadata header for the block
    * @throws IOException
    */
-  public static BlockMetadataHeader readHeader(File file) throws IOException {
-    DataInputStream in = null;
-    try {
-      in = new DataInputStream(new BufferedInputStream(
-                               new FileInputStream(file)));
+  public static BlockMetadataHeader readHeader(
+      FileInputStream fis) throws IOException {
+    try (DataInputStream in = new DataInputStream(
+        new BufferedInputStream(fis))) {
       return readHeader(in);
-    } finally {
-      IOUtils.closeStream(in);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index e6e4057..3fa4e8d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -74,6 +74,33 @@
      </Match>
 
      <!--
+       This class exposes stream constructors. The newly created streams are not
+       supposed to be closed in the constructor. Ignore the OBL warning.
+     -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.datanode.FileIoProvider$WrappedFileOutputStream" />
+       <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
+     </Match>
+
+     <!--
+       This class exposes stream constructors. The newly created streams are not
+       supposed to be closed in the constructor. Ignore the OBL warning.
+     -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.datanode.FileIoProvider$WrappedFileInputStream" />
+       <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
+     </Match>
+
+     <!--
+       This class exposes stream constructors. The newly created streams are not
+       supposed to be closed in the constructor. Ignore the OBL warning.
+     -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.datanode.FileIoProvider$WrappedRandomAccessFile" />
+       <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
+     </Match>
+
+     <!--
       lastAppliedTxid is carefully unsynchronized in the BackupNode in a couple spots.
       See the comments in BackupImage for justification.
      -->

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index df21857..cffc4bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -687,6 +687,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_PLUGINS_KEY = "dfs.datanode.plugins";
   public static final String  DFS_DATANODE_FSDATASET_FACTORY_KEY = "dfs.datanode.fsdataset.factory";
   public static final String  DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_POLICY_KEY = "dfs.datanode.fsdataset.volume.choosing.policy";
+  public static final String DFS_DATANODE_FILE_IO_EVENTS_CLASS_KEY =
+      "dfs.datanode.fileio.events.class";
   public static final String  DFS_DATANODE_AVAILABLE_SPACE_VOLUME_CHOOSING_POLICY_BALANCED_SPACE_THRESHOLD_KEY = "dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold";
   public static final long    DFS_DATANODE_AVAILABLE_SPACE_VOLUME_CHOOSING_POLICY_BALANCED_SPACE_THRESHOLD_DEFAULT = 1024L * 1024L * 1024L * 10L; // 10 GB
   public static final String  DFS_DATANODE_AVAILABLE_SPACE_VOLUME_CHOOSING_POLICY_BALANCED_SPACE_PREFERENCE_FRACTION_KEY = "dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-fraction";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index f372072..441bd91 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -244,8 +244,7 @@ class BlockReceiver implements Closeable {
       
       final boolean isCreate = isDatanode || isTransfer 
           || stage == BlockConstructionStage.PIPELINE_SETUP_CREATE;
-      streams = replicaInfo.createStreams(isCreate, requestedChecksum,
-          datanodeSlowLogThresholdMs);
+      streams = replicaInfo.createStreams(isCreate, requestedChecksum);
       assert streams != null : "null streams!";
 
       // read checksum meta information
@@ -400,9 +399,8 @@ class BlockReceiver implements Closeable {
       checksumOut.flush();
       long flushEndNanos = System.nanoTime();
       if (isSync) {
-        long fsyncStartNanos = flushEndNanos;
         streams.syncChecksumOut();
-        datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos);
+        datanode.metrics.addFsyncNanos(System.nanoTime() - flushEndNanos);
       }
       flushTotalNanos += flushEndNanos - flushStartNanos;
     }
@@ -703,8 +701,10 @@ class BlockReceiver implements Closeable {
           int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
           
           // Write data to disk.
-          long duration = streams.writeToDisk(dataBuf.array(),
+          long begin = Time.monotonicNow();
+          streams.writeDataToDisk(dataBuf.array(),
               startByteToDisk, numBytesToDisk);
+          long duration = Time.monotonicNow() - begin;
 
           if (duration > maxWriteToDiskMs) {
             maxWriteToDiskMs = duration;
@@ -1029,9 +1029,7 @@ class BlockReceiver implements Closeable {
    * will be overwritten.
    */
   private void adjustCrcFilePosition() throws IOException {
-    if (streams.getDataOut() != null) {
-      streams.flushDataOut();
-    }
+    streams.flushDataOut();
     if (checksumOut != null) {
       checksumOut.flush();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index 9182c88..d7aebd8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -166,6 +166,7 @@ class BlockSender implements java.io.Closeable {
   private final boolean dropCacheBehindAllReads;
   
   private long lastCacheDropOffset;
+  private final FileIoProvider fileIoProvider;
   
   @VisibleForTesting
   static long CACHE_DROP_INTERVAL_BYTES = 1024 * 1024; // 1MB
@@ -197,6 +198,7 @@ class BlockSender implements java.io.Closeable {
     InputStream blockIn = null;
     DataInputStream checksumIn = null;
     FsVolumeReference volumeRef = null;
+    this.fileIoProvider = datanode.getFileIoProvider();
     try {
       this.block = block;
       this.corruptChecksumOk = corruptChecksumOk;
@@ -401,7 +403,8 @@ class BlockSender implements java.io.Closeable {
         DataNode.LOG.debug("replica=" + replica);
       }
       blockIn = datanode.data.getBlockInputStream(block, offset); // seek to offset
-      ris = new ReplicaInputStreams(blockIn, checksumIn, volumeRef);
+      ris = new ReplicaInputStreams(
+          blockIn, checksumIn, volumeRef, fileIoProvider);
     } catch (IOException ioe) {
       IOUtils.closeStream(this);
       throw ioe;
@@ -568,8 +571,9 @@ class BlockSender implements java.io.Closeable {
         FileChannel fileCh = ((FileInputStream)ris.getDataIn()).getChannel();
         LongWritable waitTime = new LongWritable();
         LongWritable transferTime = new LongWritable();
-        sockOut.transferToFully(fileCh, blockInPosition, dataLen,
-            waitTime, transferTime);
+        fileIoProvider.transferToSocketFully(
+            ris.getVolumeRef().getVolume(), sockOut, fileCh, blockInPosition,
+            dataLen, waitTime, transferTime);
         datanode.metrics.addSendDataPacketBlockedOnNetworkNanos(waitTime.get());
         datanode.metrics.addSendDataPacketTransferNanos(transferTime.get());
         blockInPosition += dataLen;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/CountingFileIoEvents.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/CountingFileIoEvents.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/CountingFileIoEvents.java
new file mode 100644
index 0000000..a70c151
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/CountingFileIoEvents.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider.OPERATION;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+
+import javax.annotation.Nullable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * {@link FileIoEvents} that simply counts the number of operations.
+ * Not meant to be used outside of testing.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class CountingFileIoEvents implements FileIoEvents {
+  private final Map<OPERATION, Counts> counts;
+
+  private static class Counts {
+    private final AtomicLong successes = new AtomicLong(0);
+    private final AtomicLong failures = new AtomicLong(0);
+
+    @JsonProperty("Successes")
+    public long getSuccesses() {
+      return successes.get();
+    }
+
+    @JsonProperty("Failures")
+    public long getFailures() {
+      return failures.get();
+    }
+  }
+
+  public CountingFileIoEvents() {
+    counts = new HashMap<>();
+    for (OPERATION op : OPERATION.values()) {
+      counts.put(op, new Counts());
+    }
+  }
+
+  @Override
+  public long beforeMetadataOp(
+      @Nullable FsVolumeSpi volume, OPERATION op) {
+    return 0;
+  }
+
+  @Override
+  public void afterMetadataOp(
+      @Nullable FsVolumeSpi volume, OPERATION op, long begin) {
+    counts.get(op).successes.incrementAndGet();
+  }
+
+  @Override
+  public long beforeFileIo(
+      @Nullable FsVolumeSpi volume, OPERATION op, long len) {
+    return 0;
+  }
+
+  @Override
+  public void afterFileIo(
+      @Nullable FsVolumeSpi volume, OPERATION op, long begin, long len) {
+    counts.get(op).successes.incrementAndGet();
+  }
+
+  @Override
+  public void onFailure(
+      @Nullable FsVolumeSpi volume, OPERATION op, Exception e, long begin) {
+    counts.get(op).failures.incrementAndGet();
+
+  }
+
+  @Override
+  public String getStatistics() {
+    ObjectMapper objectMapper = new ObjectMapper();
+    try {
+      return objectMapper.writeValueAsString(counts);
+    } catch (JsonProcessingException e) {
+      // Failed to serialize. Don't log the exception call stack.
+      FileIoProvider.LOG.error("Failed to serialize statistics" + e);
+      return null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index b845da0..794b1ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -299,6 +299,7 @@ public class DataNode extends ReconfigurableBase
   public static final Log METRICS_LOG = LogFactory.getLog("DataNodeMetricsLog");
 
   private static final String DATANODE_HTRACE_PREFIX = "datanode.htrace.";
+  private final FileIoProvider fileIoProvider;
 
   /**
    * Use {@link NetUtils#createSocketAddr(String)} instead.
@@ -411,6 +412,7 @@ public class DataNode extends ReconfigurableBase
     this.tracer = createTracer(conf);
     this.tracerConfigurationManager =
         new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
+    this.fileIoProvider = new FileIoProvider(conf);
     this.fileDescriptorPassingDisabledReason = null;
     this.maxNumberOfBlocksToLog = 0;
     this.confVersion = null;
@@ -437,6 +439,7 @@ public class DataNode extends ReconfigurableBase
     this.tracer = createTracer(conf);
     this.tracerConfigurationManager =
         new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
+    this.fileIoProvider = new FileIoProvider(conf);
     this.blockScanner = new BlockScanner(this);
     this.lastDiskErrorCheck = 0;
     this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
@@ -617,6 +620,10 @@ public class DataNode extends ReconfigurableBase
         PipelineAck.ECN.SUPPORTED;
   }
 
+  public FileIoProvider getFileIoProvider() {
+    return fileIoProvider;
+  }
+
   /**
    * Contains the StorageLocations for changed data volumes.
    */
@@ -3008,6 +3015,11 @@ public class DataNode extends ReconfigurableBase
     }
   }
   
+  @Override // DataNodeMXBean
+  public String getFileIoProviderStatistics() {
+    return fileIoProvider.getStatistics();
+  }
+
   public void refreshNamenodes(Configuration conf) throws IOException {
     blockPoolManager.refreshNamenodes(conf);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
index 90c38d7..37f9635 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
@@ -120,4 +120,9 @@ public interface DataNodeMXBean {
    * @return  DiskBalancer Status
    */
   String getDiskBalancerStatus();
+
+  /**
+   * Gets the {@link FileIoProvider} statistics.
+   */
+  String getFileIoProviderStatistics();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index f4deb6d..5163e6b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -1356,6 +1356,12 @@ public class DataStorage extends Storage {
     bpStorageMap.remove(bpId);
   }
 
+  /**
+   * Prefer FileIoProvider#fullydelete.
+   * @param dir
+   * @return
+   */
+  @Deprecated
   public static boolean fullyDelete(final File dir) {
     boolean result = FileUtil.fullyDelete(dir);
     return result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
index ad054a8..c98ff54 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 
 /** Provide utility methods for Datanode. */
@@ -55,15 +56,17 @@ public class DatanodeUtil {
    * @throws IOException 
    * if the file already exists or if the file cannot be created.
    */
-  public static File createTmpFile(Block b, File f) throws IOException {
-    if (f.exists()) {
+  public static File createFileWithExistsCheck(
+      FsVolumeSpi volume, Block b, File f,
+      FileIoProvider fileIoProvider) throws IOException {
+    if (fileIoProvider.exists(volume, f)) {
       throw new IOException("Failed to create temporary file for " + b
           + ".  File " + f + " should not be present, but is.");
     }
     // Create the zero-length temp file
     final boolean fileCreated;
     try {
-      fileCreated = f.createNewFile();
+      fileCreated = fileIoProvider.createFile(volume, f);
     } catch (IOException ioe) {
       throw new IOException(DISK_ERROR + "Failed to create " + f, ioe);
     }
@@ -92,13 +95,17 @@ public class DatanodeUtil {
    * @return true if there are no files
    * @throws IOException if unable to list subdirectories
    */
-  public static boolean dirNoFilesRecursive(File dir) throws IOException {
-    File[] contents = dir.listFiles();
+  public static boolean dirNoFilesRecursive(
+      FsVolumeSpi volume, File dir,
+      FileIoProvider fileIoProvider) throws IOException {
+    File[] contents = fileIoProvider.listFiles(volume, dir);
     if (contents == null) {
       throw new IOException("Cannot list contents of " + dir);
     }
     for (File f : contents) {
-      if (!f.isDirectory() || (f.isDirectory() && !dirNoFilesRecursive(f))) {
+      if (!f.isDirectory() ||
+          (f.isDirectory() && !dirNoFilesRecursive(
+              volume, f, fileIoProvider))) {
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DefaultFileIoEvents.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DefaultFileIoEvents.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DefaultFileIoEvents.java
new file mode 100644
index 0000000..bd4932b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DefaultFileIoEvents.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider.OPERATION;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+
+import javax.annotation.Nullable;
+
+/**
+ * The default implementation of {@link FileIoEvents} that do nothing.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class DefaultFileIoEvents implements FileIoEvents {
+  @Override
+  public long beforeMetadataOp(
+      @Nullable FsVolumeSpi volume, OPERATION op) {
+    return 0;
+  }
+
+  @Override
+  public void afterMetadataOp(
+      @Nullable FsVolumeSpi volume, OPERATION op, long begin) {
+  }
+
+  @Override
+  public long beforeFileIo(
+      @Nullable FsVolumeSpi volume, OPERATION op, long len) {
+    return 0;
+  }
+
+  @Override
+  public void afterFileIo(
+      @Nullable FsVolumeSpi volume, OPERATION op, long begin, long len) {
+  }
+
+  @Override
+  public void onFailure(
+      @Nullable FsVolumeSpi volume, OPERATION op, Exception e, long begin) {
+  }
+
+  @Override
+  public @Nullable String getStatistics() {
+    // null is valid JSON.
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoEvents.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoEvents.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoEvents.java
new file mode 100644
index 0000000..48e703f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoEvents.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider.OPERATION;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+
+import javax.annotation.Nullable;
+
+/**
+ * The following hooks can be implemented for instrumentation/fault
+ * injection.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface FileIoEvents {
+
+  /**
+   * Invoked before a filesystem metadata operation.
+   *
+   * @param volume  target volume for the operation. Null if unavailable.
+   * @param op  type of operation.
+   * @return  timestamp at which the operation was started. 0 if
+   *          unavailable.
+   */
+  long beforeMetadataOp(@Nullable FsVolumeSpi volume, OPERATION op);
+
+  /**
+   * Invoked after a filesystem metadata operation has completed.
+   *
+   * @param volume  target volume for the operation.  Null if unavailable.
+   * @param op  type of operation.
+   * @param begin  timestamp at which the operation was started. 0
+   *               if unavailable.
+   */
+  void afterMetadataOp(@Nullable FsVolumeSpi volume, OPERATION op, long begin);
+
+  /**
+   * Invoked before a read/write/flush/channel transfer operation.
+   *
+   * @param volume  target volume for the operation. Null if unavailable.
+   * @param op  type of operation.
+   * @param len  length of the file IO. 0 for flush.
+   * @return  timestamp at which the operation was started. 0 if
+   *          unavailable.
+   */
+  long beforeFileIo(@Nullable FsVolumeSpi volume, OPERATION op, long len);
+
+
+  /**
+   * Invoked after a read/write/flush/channel transfer operation
+   * has completed.
+   *
+   * @param volume  target volume for the operation. Null if unavailable.
+   * @param op  type of operation.
+   * @param len   of the file IO. 0 for flush.
+   * @return  timestamp at which the operation was started. 0 if
+   *          unavailable.
+   */
+  void afterFileIo(@Nullable FsVolumeSpi volume, OPERATION op,
+                   long begin, long len);
+
+  /**
+   * Invoked if an operation fails with an exception.
+   *  @param volume  target volume for the operation. Null if unavailable.
+   * @param op  type of operation.
+   * @param e  Exception encountered during the operation.
+   * @param begin  time at which the operation was started.
+   */
+  void onFailure(
+      @Nullable FsVolumeSpi volume, OPERATION op, Exception e, long begin);
+
+  /**
+   * Return statistics as a JSON string.
+   * @return
+   */
+  @Nullable String getStatistics();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoProvider.java
new file mode 100644
index 0000000..2344114
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoProvider.java
@@ -0,0 +1,1006 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIOException;
+import org.apache.hadoop.net.SocketOutputStream;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.Flushable;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.nio.file.CopyOption;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+
+import static org.apache.hadoop.hdfs.server.datanode.FileIoProvider.OPERATION.*;
+
+/**
+ * This class abstracts out various file IO operations performed by the
+ * DataNode and invokes event hooks before and after each file IO.
+ *
+ * Behavior can be injected into these events by implementing
+ * {@link FileIoEvents} and replacing the default implementation
+ * with {@link DFSConfigKeys#DFS_DATANODE_FILE_IO_EVENTS_CLASS_KEY}.
+ *
+ * Most functions accept an optional {@link FsVolumeSpi} parameter for
+ * instrumentation/logging.
+ *
+ * Some methods may look redundant, especially the multiple variations of
+ * move/rename/list. They exist to retain behavior compatibility for existing
+ * code.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class FileIoProvider {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      FileIoProvider.class);
+
+  private final FileIoEvents eventHooks;
+
+  /**
+   * @param conf  Configuration object. May be null. When null,
+   *              the event handlers are no-ops.
+   */
+  public FileIoProvider(@Nullable Configuration conf) {
+    if (conf != null) {
+      final Class<? extends FileIoEvents> clazz = conf.getClass(
+          DFSConfigKeys.DFS_DATANODE_FILE_IO_EVENTS_CLASS_KEY,
+          DefaultFileIoEvents.class,
+          FileIoEvents.class);
+      eventHooks = ReflectionUtils.newInstance(clazz, conf);
+    } else {
+      eventHooks = new DefaultFileIoEvents();
+    }
+  }
+
+  /**
+   * Lists the types of file system operations. Passed to the
+   * IO hooks so implementations can choose behavior based on
+   * specific operations.
+   */
+  public enum OPERATION {
+    OPEN,
+    EXISTS,
+    LIST,
+    DELETE,
+    MOVE,
+    MKDIRS,
+    TRANSFER,
+    SYNC,
+    FADVISE,
+    READ,
+    WRITE,
+    FLUSH,
+    NATIVE_COPY
+  }
+
+  /**
+   * Retrieve statistics from the underlying {@link FileIoEvents}
+   * implementation as a JSON string, if it maintains them.
+   * @return statistics as a JSON string. May be null.
+   */
+  public @Nullable String getStatistics() {
+    return eventHooks.getStatistics();
+  }
+
+  /**
+   * See {@link Flushable#flush()}.
+   *
+   * @param  volume target volume. null if unavailable.
+   * @throws IOException
+   */
+  public void flush(
+      @Nullable FsVolumeSpi volume, Flushable f) throws IOException {
+    final long begin = eventHooks.beforeFileIo(volume, FLUSH, 0);
+    try {
+      f.flush();
+      eventHooks.afterFileIo(volume, FLUSH, begin, 0);
+    } catch (Exception e) {
+      eventHooks.onFailure(volume, FLUSH, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Sync the given {@link FileOutputStream}.
+   *
+   * @param  volume target volume. null if unavailable.
+   * @throws IOException
+   */
+  public void sync(
+      @Nullable FsVolumeSpi volume, FileOutputStream fos) throws IOException {
+    final long begin = eventHooks.beforeFileIo(volume, SYNC, 0);
+    try {
+      fos.getChannel().force(true);
+      eventHooks.afterFileIo(volume, SYNC, begin, 0);
+    } catch (Exception e) {
+      eventHooks.onFailure(volume, SYNC, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Call sync_file_range on the given file descriptor.
+   *
+   * @param  volume target volume. null if unavailable.
+   * @throws IOException
+   */
+  public void syncFileRange(
+      @Nullable FsVolumeSpi volume, FileDescriptor outFd,
+      long offset, long numBytes, int flags) throws NativeIOException {
+    final long begin = eventHooks.beforeFileIo(volume, SYNC, 0);
+    try {
+      NativeIO.POSIX.syncFileRangeIfPossible(outFd, offset, numBytes, flags);
+      eventHooks.afterFileIo(volume, SYNC, begin, 0);
+    } catch (Exception e) {
+      eventHooks.onFailure(volume, SYNC, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Call posix_fadvise on the given file descriptor.
+   *
+   * @param  volume target volume. null if unavailable.
+   * @throws IOException
+   */
+  public void posixFadvise(
+      @Nullable FsVolumeSpi volume, String identifier, FileDescriptor outFd,
+      long offset, long length, int flags) throws NativeIOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, FADVISE);
+    try {
+      NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+          identifier, outFd, offset, length, flags);
+      eventHooks.afterMetadataOp(volume, FADVISE, begin);
+    } catch (Exception e) {
+      eventHooks.onFailure(volume, FADVISE, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Delete a file.
+   * @param volume  target volume. null if unavailable.
+   * @param f  File to delete.
+   * @return  true if the file was successfully deleted.
+   */
+  public boolean delete(@Nullable FsVolumeSpi volume, File f) {
+    final long begin = eventHooks.beforeMetadataOp(volume, DELETE);
+    try {
+      boolean deleted = f.delete();
+      eventHooks.afterMetadataOp(volume, DELETE, begin);
+      return deleted;
+    } catch (Exception e) {
+      eventHooks.onFailure(volume, DELETE, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Delete a file, first checking to see if it exists.
+   * @param volume  target volume. null if unavailable.
+   * @param f  File to delete
+   * @return  true if the file was successfully deleted or if it never
+   *          existed.
+   */
+  public boolean deleteWithExistsCheck(@Nullable FsVolumeSpi volume, File f) {
+    final long begin = eventHooks.beforeMetadataOp(volume, DELETE);
+    try {
+      boolean deleted = !f.exists() || f.delete();
+      eventHooks.afterMetadataOp(volume, DELETE, begin);
+      if (!deleted) {
+        LOG.warn("Failed to delete file {}", f);
+      }
+      return deleted;
+    } catch (Exception e) {
+      eventHooks.onFailure(volume, DELETE, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Transfer data from a FileChannel to a SocketOutputStream.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param sockOut  SocketOutputStream to write the data.
+   * @param fileCh  FileChannel from which to read data.
+   * @param position  position within the channel where the transfer begins.
+   * @param count  number of bytes to transfer.
+   * @param waitTime  returns the nanoseconds spent waiting for the socket
+   *                  to become writable.
+   * @param transferTime  returns the nanoseconds spent transferring data.
+   * @throws IOException
+   */
+  public void transferToSocketFully(
+      @Nullable FsVolumeSpi volume, SocketOutputStream sockOut,
+      FileChannel fileCh, long position, int count,
+      LongWritable waitTime, LongWritable transferTime) throws IOException {
+    final long begin = eventHooks.beforeFileIo(volume, TRANSFER, count);
+    try {
+      sockOut.transferToFully(fileCh, position, count,
+          waitTime, transferTime);
+      eventHooks.afterFileIo(volume, TRANSFER, begin, count);
+    } catch (Exception e) {
+      eventHooks.onFailure(volume, TRANSFER, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Create a file.
+   * @param volume  target volume. null if unavailable.
+   * @param f  File to be created.
+   * @return  true if the file does not exist and was successfully created.
+   *          false if the file already exists.
+   * @throws IOException
+   */
+  public boolean createFile(
+      @Nullable FsVolumeSpi volume, File f) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
+    try {
+      boolean created = f.createNewFile();
+      eventHooks.afterMetadataOp(volume, OPEN, begin);
+      return created;
+    } catch (Exception e) {
+      eventHooks.onFailure(volume, OPEN, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Create a FileInputStream using
+   * {@link FileInputStream#FileInputStream(File)}.
+   *
+   * Wraps the created input stream to intercept read calls
+   * before delegating to the wrapped stream.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param f  File object.
+   * @return  FileInputStream to the given file.
+   * @throws  FileNotFoundException
+   */
+  public FileInputStream getFileInputStream(
+      @Nullable FsVolumeSpi volume, File f) throws FileNotFoundException {
+    final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
+    FileInputStream fis = null;
+    try {
+      fis = new WrappedFileInputStream(volume, f);
+      eventHooks.afterMetadataOp(volume, OPEN, begin);
+      return fis;
+    } catch(Exception e) {
+      org.apache.commons.io.IOUtils.closeQuietly(fis);
+      eventHooks.onFailure(volume, OPEN, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Create a FileOutputStream using
+   * {@link FileOutputStream#FileOutputStream(File, boolean)}.
+   *
+   * Wraps the created output stream to intercept write calls
+   * before delegating to the wrapped stream.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param f  File object.
+   * @param append  if true, then bytes will be written to the end of the
+   *                file rather than the beginning.
+   * @param  FileOutputStream to the given file object.
+   * @throws FileNotFoundException
+   */
+  public FileOutputStream getFileOutputStream(
+      @Nullable FsVolumeSpi volume, File f,
+      boolean append) throws FileNotFoundException {
+    final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
+    FileOutputStream fos = null;
+    try {
+      fos = new WrappedFileOutputStream(volume, f, append);
+      eventHooks.afterMetadataOp(volume, OPEN, begin);
+      return fos;
+    } catch(Exception e) {
+      org.apache.commons.io.IOUtils.closeQuietly(fos);
+      eventHooks.onFailure(volume, OPEN, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Create a FileOutputStream using
+   * {@link FileOutputStream#FileOutputStream(File, boolean)}.
+   *
+   * Wraps the created output stream to intercept write calls
+   * before delegating to the wrapped stream.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param f  File object.
+   * @return  FileOutputStream to the given file object.
+   * @throws  FileNotFoundException
+   */
+  public FileOutputStream getFileOutputStream(
+      @Nullable FsVolumeSpi volume, File f) throws FileNotFoundException {
+    return getFileOutputStream(volume, f, false);
+  }
+
+  /**
+   * Create a FileOutputStream using
+   * {@link FileOutputStream#FileOutputStream(FileDescriptor)}.
+   *
+   * Wraps the created output stream to intercept write calls
+   * before delegating to the wrapped stream.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param f  File object.
+   * @return  FileOutputStream to the given file object.
+   * @throws  FileNotFoundException
+   */
+  public FileOutputStream getFileOutputStream(
+      @Nullable FsVolumeSpi volume, FileDescriptor fd) {
+    return new WrappedFileOutputStream(volume, fd);
+  }
+
+  /**
+   * Create a FileInputStream using
+   * {@link NativeIO#getShareDeleteFileDescriptor}.
+   * Wraps the created input stream to intercept input calls
+   * before delegating to the wrapped stream.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param f  File object.
+   * @param offset  the offset position, measured in bytes from the
+   *                beginning of the file, at which to set the file
+   *                pointer.
+   * @return FileOutputStream to the given file object.
+   * @throws FileNotFoundException
+   */
+  public FileInputStream getShareDeleteFileInputStream(
+      @Nullable FsVolumeSpi volume, File f,
+      long offset) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
+    FileInputStream fis = null;
+    try {
+      fis = new WrappedFileInputStream(volume,
+          NativeIO.getShareDeleteFileDescriptor(f, offset));
+      eventHooks.afterMetadataOp(volume, OPEN, begin);
+      return fis;
+    } catch(Exception e) {
+      org.apache.commons.io.IOUtils.closeQuietly(fis);
+      eventHooks.onFailure(volume, OPEN, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Create a FileInputStream using
+   * {@link FileInputStream#FileInputStream(File)} and position
+   * it at the given offset.
+   *
+   * Wraps the created input stream to intercept read calls
+   * before delegating to the wrapped stream.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param f  File object.
+   * @param offset  the offset position, measured in bytes from the
+   *                beginning of the file, at which to set the file
+   *                pointer.
+   * @throws FileNotFoundException
+   */
+  public FileInputStream openAndSeek(
+      @Nullable FsVolumeSpi volume, File f, long offset) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
+    FileInputStream fis = null;
+    try {
+      fis = new WrappedFileInputStream(volume,
+          FsDatasetUtil.openAndSeek(f, offset));
+      eventHooks.afterMetadataOp(volume, OPEN, begin);
+      return fis;
+    } catch(Exception e) {
+      org.apache.commons.io.IOUtils.closeQuietly(fis);
+      eventHooks.onFailure(volume, OPEN, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Create a RandomAccessFile using
+   * {@link RandomAccessFile#RandomAccessFile(File, String)}.
+   *
+   * Wraps the created input stream to intercept IO calls
+   * before delegating to the wrapped RandomAccessFile.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param f  File object.
+   * @param mode  See {@link RandomAccessFile} for a description
+   *              of the mode string.
+   * @return RandomAccessFile representing the given file.
+   * @throws FileNotFoundException
+   */
+  public RandomAccessFile getRandomAccessFile(
+      @Nullable FsVolumeSpi volume, File f,
+      String mode) throws FileNotFoundException {
+    final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
+    RandomAccessFile raf = null;
+    try {
+      raf = new WrappedRandomAccessFile(volume, f, mode);
+      eventHooks.afterMetadataOp(volume, OPEN, begin);
+      return raf;
+    } catch(Exception e) {
+      org.apache.commons.io.IOUtils.closeQuietly(raf);
+      eventHooks.onFailure(volume, OPEN, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Delete the given directory using {@link FileUtil#fullyDelete(File)}.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param dir  directory to be deleted.
+   * @return true on success false on failure.
+   */
+  public boolean fullyDelete(@Nullable FsVolumeSpi volume, File dir) {
+    final long begin = eventHooks.beforeMetadataOp(volume, DELETE);
+    try {
+      boolean deleted = FileUtil.fullyDelete(dir);
+      eventHooks.afterMetadataOp(volume, DELETE, begin);
+      return deleted;
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, DELETE, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Move the src file to the target using
+   * {@link FileUtil#replaceFile(File, File)}.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param src  source path.
+   * @param target  target path.
+   * @throws IOException
+   */
+  public void replaceFile(
+      @Nullable FsVolumeSpi volume, File src, File target) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, MOVE);
+    try {
+      FileUtil.replaceFile(src, target);
+      eventHooks.afterMetadataOp(volume, MOVE, begin);
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, MOVE, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Move the src file to the target using
+   * {@link Storage#rename(File, File)}.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param src  source path.
+   * @param target  target path.
+   * @throws IOException
+   */
+  public void rename(
+      @Nullable FsVolumeSpi volume, File src, File target)
+      throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, MOVE);
+    try {
+      Storage.rename(src, target);
+      eventHooks.afterMetadataOp(volume, MOVE, begin);
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, MOVE, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Move the src file to the target using
+   * {@link FileUtils#moveFile(File, File)}.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param src  source path.
+   * @param target  target path.
+   * @throws IOException
+   */
+  public void moveFile(
+      @Nullable FsVolumeSpi volume, File src, File target)
+      throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, MOVE);
+    try {
+      FileUtils.moveFile(src, target);
+      eventHooks.afterMetadataOp(volume, MOVE, begin);
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, MOVE, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Move the src file to the target using
+   * {@link Files#move(Path, Path, CopyOption...)}.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param src  source path.
+   * @param target  target path.
+   * @param options  See {@link Files#move} for a description
+   *                of the options.
+   * @throws IOException
+   */
+  public void move(
+      @Nullable FsVolumeSpi volume, Path src, Path target,
+      CopyOption... options) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, MOVE);
+    try {
+      Files.move(src, target, options);
+      eventHooks.afterMetadataOp(volume, MOVE, begin);
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, MOVE, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * See {@link Storage#nativeCopyFileUnbuffered(File, File, boolean)}.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param src  an existing file to copy, must not be {@code null}
+   * @param target  the new file, must not be {@code null}
+   * @param preserveFileDate  true if the file date of the copy
+   *                         should be the same as the original
+   * @throws IOException
+   */
+  public void nativeCopyFileUnbuffered(
+      @Nullable FsVolumeSpi volume, File src, File target,
+      boolean preserveFileDate) throws IOException {
+    final long length = src.length();
+    final long begin = eventHooks.beforeFileIo(volume, NATIVE_COPY, length);
+    try {
+      Storage.nativeCopyFileUnbuffered(src, target, preserveFileDate);
+      eventHooks.afterFileIo(volume, NATIVE_COPY, begin, length);
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, NATIVE_COPY, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * See {@link File#mkdirs()}.
+   *
+   * @param volume target volume. null if unavailable.
+   * @param dir  directory to be created.
+   * @return  true only if the directory was created. false if
+   *          the directory already exists.
+   * @throws IOException if a directory with the given name does
+   *                     not exist and could not be created.
+   */
+  public boolean mkdirs(
+      @Nullable FsVolumeSpi volume, File dir) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, MKDIRS);
+    boolean created = false;
+    boolean isDirectory;
+    try {
+      created = dir.mkdirs();
+      isDirectory = !created && dir.isDirectory();
+      eventHooks.afterMetadataOp(volume, MKDIRS, begin);
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, MKDIRS, e, begin);
+      throw e;
+    }
+
+    if (!created && !isDirectory) {
+      throw new IOException("Mkdirs failed to create " + dir);
+    }
+    return created;
+  }
+
+  /**
+   * Create the target directory using {@link File#mkdirs()} only if
+   * it doesn't exist already.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param dir  directory to be created.
+   * @throws IOException  if the directory could not created
+   */
+  public void mkdirsWithExistsCheck(
+      @Nullable FsVolumeSpi volume, File dir) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, MKDIRS);
+    boolean succeeded = false;
+    try {
+      succeeded = dir.isDirectory() || dir.mkdirs();
+      eventHooks.afterMetadataOp(volume, MKDIRS, begin);
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, MKDIRS, e, begin);
+      throw e;
+    }
+
+    if (!succeeded) {
+      throw new IOException("Mkdirs failed to create " + dir);
+    }
+  }
+
+  /**
+   * Get a listing of the given directory using
+   * {@link FileUtil#listFiles(File)}.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param dir  Directory to be listed.
+   * @return  array of file objects representing the directory entries.
+   * @throws IOException
+   */
+  public File[] listFiles(
+      @Nullable FsVolumeSpi volume, File dir) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, LIST);
+    try {
+      File[] children = FileUtil.listFiles(dir);
+      eventHooks.afterMetadataOp(volume, LIST, begin);
+      return children;
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, LIST, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Get a listing of the given directory using
+   * {@link FileUtil#listFiles(File)}.
+   *
+   * @param volume  target volume. null if unavailable.
+   * @param   Driectory to be listed.
+   * @return  array of strings representing the directory entries.
+   * @throws IOException
+   */
+  public String[] list(
+      @Nullable FsVolumeSpi volume, File dir) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, LIST);
+    try {
+      String[] children = FileUtil.list(dir);
+      eventHooks.afterMetadataOp(volume, LIST, begin);
+      return children;
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, LIST, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Get a listing of the given directory using
+   * {@link IOUtils#listDirectory(File, FilenameFilter)}.
+   *
+   * @param volume target volume. null if unavailable.
+   * @param dir Directory to list.
+   * @param filter {@link FilenameFilter} to filter the directory entries.
+   * @throws IOException
+   */
+  public List<String> listDirectory(
+      @Nullable FsVolumeSpi volume, File dir,
+      FilenameFilter filter) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, LIST);
+    try {
+      List<String> children = IOUtils.listDirectory(dir, filter);
+      eventHooks.afterMetadataOp(volume, LIST, begin);
+      return children;
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, LIST, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Retrieves the number of links to the specified file.
+   *
+   * @param volume target volume. null if unavailable.
+   * @param f file whose link count is being queried.
+   * @return number of hard-links to the given file, including the
+   *         given path itself.
+   * @throws IOException
+   */
+  public int getHardLinkCount(
+      @Nullable FsVolumeSpi volume, File f) throws IOException {
+    final long begin = eventHooks.beforeMetadataOp(volume, LIST);
+    try {
+      int count = HardLink.getLinkCount(f);
+      eventHooks.afterMetadataOp(volume, LIST, begin);
+      return count;
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, LIST, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * Check for file existence using {@link File#exists()}.
+   *
+   * @param volume target volume. null if unavailable.
+   * @param f file object.
+   * @return true if the file exists.
+   */
+  public boolean exists(@Nullable FsVolumeSpi volume, File f) {
+    final long begin = eventHooks.beforeMetadataOp(volume, EXISTS);
+    try {
+      boolean exists = f.exists();
+      eventHooks.afterMetadataOp(volume, EXISTS, begin);
+      return exists;
+    } catch(Exception e) {
+      eventHooks.onFailure(volume, EXISTS, e, begin);
+      throw e;
+    }
+  }
+
+  /**
+   * A thin wrapper over {@link FileInputStream} that allows
+   * instrumenting disk IO.
+   */
+  private final class WrappedFileInputStream extends FileInputStream {
+    private @Nullable final FsVolumeSpi volume;
+
+    /**
+     * {@inheritDoc}.
+     */
+    private WrappedFileInputStream(@Nullable FsVolumeSpi volume, File f)
+        throws FileNotFoundException {
+      super(f);
+      this.volume = volume;
+    }
+
+    /**
+     * {@inheritDoc}.
+     */
+    private WrappedFileInputStream(
+        @Nullable FsVolumeSpi volume, FileDescriptor fd) {
+      super(fd);
+      this.volume = volume;
+    }
+
+    /**
+     * {@inheritDoc}.
+     */
+    @Override
+    public int read() throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, READ, 1);
+      try {
+        int b = super.read();
+        eventHooks.afterFileIo(volume, READ, begin, 1);
+        return b;
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, READ, e, begin);
+        throw e;
+      }
+    }
+
+    /**
+     * {@inheritDoc}.
+     */
+    @Override
+    public int read(@Nonnull byte[] b) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, READ, b.length);
+      try {
+        int numBytesRead = super.read(b);
+        eventHooks.afterFileIo(volume, READ, begin, numBytesRead);
+        return numBytesRead;
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, READ, e, begin);
+        throw e;
+      }
+    }
+
+    /**
+     * {@inheritDoc}.
+     */
+    @Override
+    public int read(@Nonnull byte[] b, int off, int len) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, READ, len);
+      try {
+        int numBytesRead = super.read(b, off, len);
+        eventHooks.afterFileIo(volume, READ, begin, numBytesRead);
+        return numBytesRead;
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, READ, e, begin);
+        throw e;
+      }
+    }
+  }
+
+  /**
+   * A thin wrapper over {@link FileOutputStream} that allows
+   * instrumenting disk IO.
+   */
+  private final class WrappedFileOutputStream extends FileOutputStream {
+    private @Nullable final FsVolumeSpi volume;
+
+    /**
+     * {@inheritDoc}.
+     */
+    private WrappedFileOutputStream(
+        @Nullable FsVolumeSpi volume, File f,
+        boolean append) throws FileNotFoundException {
+      super(f, append);
+      this.volume = volume;
+    }
+
+    /**
+     * {@inheritDoc}.
+     */
+    private WrappedFileOutputStream(
+        @Nullable FsVolumeSpi volume, FileDescriptor fd) {
+      super(fd);
+      this.volume = volume;
+    }
+
+    /**
+     * {@inheritDoc}.
+     */
+    @Override
+    public void write(int b) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, WRITE, 1);
+      try {
+        super.write(b);
+        eventHooks.afterFileIo(volume, WRITE, begin, 1);
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, WRITE, e, begin);
+        throw e;
+      }
+    }
+
+    /**
+     * {@inheritDoc}.
+     */
+    @Override
+    public void write(@Nonnull byte[] b) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, WRITE, b.length);
+      try {
+        super.write(b);
+        eventHooks.afterFileIo(volume, WRITE, begin, b.length);
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, WRITE, e, begin);
+        throw e;
+      }
+    }
+
+    /**
+     * {@inheritDoc}.
+     */
+    @Override
+    public void write(@Nonnull byte[] b, int off, int len) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, WRITE, len);
+      try {
+        super.write(b, off, len);
+        eventHooks.afterFileIo(volume, WRITE, begin, len);
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, WRITE, e, begin);
+        throw e;
+      }
+    }
+  }
+
+  /**
+   * A thin wrapper over {@link FileInputStream} that allows
+   * instrumenting IO.
+   */
+  private final class WrappedRandomAccessFile extends RandomAccessFile {
+    private @Nullable final FsVolumeSpi volume;
+
+    public WrappedRandomAccessFile(
+        @Nullable FsVolumeSpi volume, File f, String mode)
+        throws FileNotFoundException {
+      super(f, mode);
+      this.volume = volume;
+    }
+
+    @Override
+    public int read() throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, READ, 1);
+      try {
+        int b = super.read();
+        eventHooks.afterFileIo(volume, READ, begin, 1);
+        return b;
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, READ, e, begin);
+        throw e;
+      }
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, READ, len);
+      try {
+        int numBytesRead = super.read(b, off, len);
+        eventHooks.afterFileIo(volume, READ, begin, numBytesRead);
+        return numBytesRead;
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, READ, e, begin);
+        throw e;
+      }
+    }
+
+    @Override
+    public int read(byte[] b) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, READ, b.length);
+      try {
+        int numBytesRead = super.read(b);
+        eventHooks.afterFileIo(volume, READ, begin, numBytesRead);
+        return numBytesRead;
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, READ, e, begin);
+        throw e;
+      }
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, WRITE, 1);
+      try {
+        super.write(b);
+        eventHooks.afterFileIo(volume, WRITE, begin, 1);
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, WRITE, e, begin);
+        throw e;
+      }
+    }
+
+    @Override
+    public void write(@Nonnull byte[] b) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, WRITE, b.length);
+      try {
+        super.write(b);
+        eventHooks.afterFileIo(volume, WRITE, begin, b.length);
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, WRITE, e, begin);
+        throw e;
+      }
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      final long begin = eventHooks.beforeFileIo(volume, WRITE, len);
+      try {
+        super.write(b, off, len);
+        eventHooks.afterFileIo(volume, WRITE, begin, len);
+      } catch(Exception e) {
+        eventHooks.onFailure(volume, WRITE, e, begin);
+        throw e;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
index e6f7e12..1d46ddd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
@@ -29,17 +29,13 @@ import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 
-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.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.DataChecksum;
@@ -187,20 +183,23 @@ abstract public class LocalReplica extends ReplicaInfo {
    * 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)){
-        copyBytes(in, out, 16 * 1024);
+    final FileIoProvider fileIoProvider = getFileIoProvider();
+    final File tmpFile = DatanodeUtil.createFileWithExistsCheck(
+        getVolume(), b, DatanodeUtil.getUnlinkTmpFile(file), fileIoProvider);
+    try (FileInputStream in = fileIoProvider.getFileInputStream(
+        getVolume(), file)) {
+      try (FileOutputStream out = fileIoProvider.getFileOutputStream(
+          getVolume(), 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());
       }
-      replaceFile(tmpFile, file);
+      fileIoProvider.replaceFile(getVolume(), tmpFile, file);
     } catch (IOException e) {
-      boolean done = tmpFile.delete();
-      if (!done) {
+      if (!fileIoProvider.delete(getVolume(), tmpFile)) {
         DataNode.LOG.info("detachFile failed to delete temporary file " +
                           tmpFile);
       }
@@ -226,19 +225,20 @@ abstract public class LocalReplica extends ReplicaInfo {
    * @throws IOException
    */
   public boolean breakHardLinksIfNeeded() throws IOException {
-    File file = getBlockFile();
+    final File file = getBlockFile();
+    final FileIoProvider fileIoProvider = getFileIoProvider();
     if (file == null || getVolume() == null) {
       throw new IOException("detachBlock:Block not found. " + this);
     }
     File meta = getMetaFile();
 
-    int linkCount = getHardLinkCount(file);
+    int linkCount = fileIoProvider.getHardLinkCount(getVolume(), file);
     if (linkCount > 1) {
       DataNode.LOG.info("Breaking hardlink for " + linkCount + "x-linked " +
           "block " + this);
       breakHardlinks(file, this);
     }
-    if (getHardLinkCount(meta) > 1) {
+    if (fileIoProvider.getHardLinkCount(getVolume(), meta) > 1) {
       breakHardlinks(meta, this);
     }
     return true;
@@ -256,17 +256,18 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public OutputStream getDataOutputStream(boolean append) throws IOException {
-    return new FileOutputStream(getBlockFile(), append);
+    return getFileIoProvider().getFileOutputStream(
+        getVolume(), getBlockFile(), append);
   }
 
   @Override
   public boolean blockDataExists() {
-    return getBlockFile().exists();
+    return getFileIoProvider().exists(getVolume(), getBlockFile());
   }
 
   @Override
   public boolean deleteBlockData() {
-    return fullyDelete(getBlockFile());
+    return getFileIoProvider().fullyDelete(getVolume(), getBlockFile());
   }
 
   @Override
@@ -282,9 +283,10 @@ abstract public class LocalReplica extends ReplicaInfo {
   @Override
   public LengthInputStream getMetadataInputStream(long offset)
       throws IOException {
-    File meta = getMetaFile();
+    final File meta = getMetaFile();
     return new LengthInputStream(
-        FsDatasetUtil.openAndSeek(meta, offset), meta.length());
+        getFileIoProvider().openAndSeek(getVolume(), meta, offset),
+        meta.length());
   }
 
   @Override
@@ -295,12 +297,12 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean metadataExists() {
-    return getMetaFile().exists();
+    return getFileIoProvider().exists(getVolume(), getMetaFile());
   }
 
   @Override
   public boolean deleteMetadata() {
-    return fullyDelete(getMetaFile());
+    return getFileIoProvider().fullyDelete(getVolume(), getMetaFile());
   }
 
   @Override
@@ -320,7 +322,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   private boolean renameFile(File srcfile, File destfile) throws IOException {
     try {
-      rename(srcfile, destfile);
+      getFileIoProvider().rename(getVolume(), srcfile, destfile);
       return true;
     } catch (IOException e) {
       throw new IOException("Failed to move block file for " + this
@@ -360,9 +362,9 @@ abstract public class LocalReplica extends ReplicaInfo {
   @Override
   public void bumpReplicaGS(long newGS) throws IOException {
     long oldGS = getGenerationStamp();
-    File oldmeta = getMetaFile();
+    final File oldmeta = getMetaFile();
     setGenerationStamp(newGS);
-    File newmeta = getMetaFile();
+    final File newmeta = getMetaFile();
 
     // rename meta file to new GS
     if (LOG.isDebugEnabled()) {
@@ -370,7 +372,7 @@ abstract public class LocalReplica extends ReplicaInfo {
     }
     try {
       // calling renameMeta on the ReplicaInfo doesn't work here
-      rename(oldmeta, newmeta);
+      getFileIoProvider().rename(getVolume(), oldmeta, newmeta);
     } catch (IOException e) {
       setGenerationStamp(oldGS); // restore old GS
       throw new IOException("Block " + this + " reopen failed. " +
@@ -381,7 +383,8 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public void truncateBlock(long newLength) throws IOException {
-    truncateBlock(getBlockFile(), getMetaFile(), getNumBytes(), newLength);
+    truncateBlock(getVolume(), getBlockFile(), getMetaFile(),
+        getNumBytes(), newLength, getFileIoProvider());
   }
 
   @Override
@@ -392,32 +395,15 @@ abstract public class LocalReplica extends ReplicaInfo {
   @Override
   public void copyMetadata(URI destination) throws IOException {
     //for local replicas, we assume the destination URI is file
-    nativeCopyFileUnbuffered(getMetaFile(), new File(destination), true);
+    getFileIoProvider().nativeCopyFileUnbuffered(
+        getVolume(), getMetaFile(), new File(destination), true);
   }
 
   @Override
   public void copyBlockdata(URI destination) throws IOException {
     //for local replicas, we assume the destination URI is file
-    nativeCopyFileUnbuffered(getBlockFile(), new File(destination), true);
-  }
-
-  public void renameMeta(File newMetaFile) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming " + getMetaFile() + " to " + newMetaFile);
-    }
-    renameFile(getMetaFile(), newMetaFile);
-  }
-
-  public void renameBlock(File newBlockFile) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming " + getBlockFile() + " to " + newBlockFile
-          + ", file length=" + getBlockFile().length());
-    }
-    renameFile(getBlockFile(), newBlockFile);
-  }
-
-  public static void rename(File from, File to) throws IOException {
-    Storage.rename(from, to);
+    getFileIoProvider().nativeCopyFileUnbuffered(
+        getVolume(), getBlockFile(), new File(destination), true);
   }
 
   /**
@@ -430,11 +416,13 @@ abstract public class LocalReplica extends ReplicaInfo {
   private FileInputStream getDataInputStream(File f, long seekOffset)
       throws IOException {
     FileInputStream fis;
+    final FileIoProvider fileIoProvider = getFileIoProvider();
     if (NativeIO.isAvailable()) {
-      fis = NativeIO.getShareDeleteFileInputStream(f, seekOffset);
+      fis = fileIoProvider.getShareDeleteFileInputStream(
+          getVolume(), f, seekOffset);
     } else {
       try {
-        fis = FsDatasetUtil.openAndSeek(f, seekOffset);
+        fis = fileIoProvider.openAndSeek(getVolume(), f, seekOffset);
       } catch (FileNotFoundException fnfe) {
         throw new IOException("Expected block file at " + f +
             " does not exist.");
@@ -443,30 +431,6 @@ abstract public class LocalReplica extends ReplicaInfo {
     return fis;
   }
 
-  private void nativeCopyFileUnbuffered(File srcFile, File destFile,
-      boolean preserveFileDate) throws IOException {
-    Storage.nativeCopyFileUnbuffered(srcFile, destFile, preserveFileDate);
-  }
-
-  private void copyBytes(InputStream in, OutputStream out, int
-      buffSize) throws IOException{
-    IOUtils.copyBytes(in, out, buffSize);
-  }
-
-  private void replaceFile(File src, File target) throws IOException {
-    FileUtil.replaceFile(src, target);
-  }
-
-  public static boolean fullyDelete(final File dir) {
-    boolean result = DataStorage.fullyDelete(dir);
-    return result;
-  }
-
-  public static int getHardLinkCount(File fileName) throws IOException {
-    int linkCount = HardLink.getLinkCount(fileName);
-    return linkCount;
-  }
-
   /**
    *  Get pin status of a file by checking the sticky bit.
    * @param localFS local file system
@@ -495,8 +459,10 @@ abstract public class LocalReplica extends ReplicaInfo {
     localFS.setPermission(path, permission);
   }
 
-  public static void truncateBlock(File blockFile, File metaFile,
-      long oldlen, long newlen) throws IOException {
+  public static void truncateBlock(
+      FsVolumeSpi volume, File blockFile, File metaFile,
+      long oldlen, long newlen, FileIoProvider fileIoProvider)
+      throws IOException {
     LOG.info("truncateBlock: blockFile=" + blockFile
         + ", metaFile=" + metaFile
         + ", oldlen=" + oldlen
@@ -510,7 +476,10 @@ abstract public class LocalReplica extends ReplicaInfo {
           + ") to newlen (=" + newlen + ")");
     }
 
-    DataChecksum dcs = BlockMetadataHeader.readHeader(metaFile).getChecksum();
+    // fis is closed by BlockMetadataHeader.readHeader.
+    final FileInputStream fis = fileIoProvider.getFileInputStream(
+        volume, metaFile);
+    DataChecksum dcs = BlockMetadataHeader.readHeader(fis).getChecksum();
     int checksumsize = dcs.getChecksumSize();
     int bpc = dcs.getBytesPerChecksum();
     long n = (newlen - 1)/bpc + 1;
@@ -519,16 +488,14 @@ abstract public class LocalReplica extends ReplicaInfo {
     int lastchunksize = (int)(newlen - lastchunkoffset);
     byte[] b = new byte[Math.max(lastchunksize, checksumsize)];
 
-    RandomAccessFile blockRAF = new RandomAccessFile(blockFile, "rw");
-    try {
+    try (RandomAccessFile blockRAF = fileIoProvider.getRandomAccessFile(
+        volume, blockFile, "rw")) {
       //truncate blockFile
       blockRAF.setLength(newlen);
 
       //read last chunk
       blockRAF.seek(lastchunkoffset);
       blockRAF.readFully(b, 0, lastchunksize);
-    } finally {
-      blockRAF.close();
     }
 
     //compute checksum
@@ -536,13 +503,11 @@ abstract public class LocalReplica extends ReplicaInfo {
     dcs.writeValue(b, 0, false);
 
     //update metaFile
-    RandomAccessFile metaRAF = new RandomAccessFile(metaFile, "rw");
-    try {
+    try (RandomAccessFile metaRAF = fileIoProvider.getRandomAccessFile(
+        volume, metaFile, "rw")) {
       metaRAF.setLength(newmetalen);
       metaRAF.seek(newmetalen - checksumsize);
       metaRAF.write(b, 0, checksumsize);
-    } finally {
-      metaRAF.close();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
index 1387155..003f96f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
@@ -245,10 +245,9 @@ public class LocalReplicaInPipeline extends LocalReplica
 
   @Override // ReplicaInPipeline
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum, long slowLogThresholdMs)
-      throws IOException {
-    File blockFile = getBlockFile();
-    File metaFile = getMetaFile();
+      DataChecksum requestedChecksum) throws IOException {
+    final File blockFile = getBlockFile();
+    final File metaFile = getMetaFile();
     if (DataNode.LOG.isDebugEnabled()) {
       DataNode.LOG.debug("writeTo blockfile is " + blockFile +
                          " of size " + blockFile.length());
@@ -262,14 +261,16 @@ public class LocalReplicaInPipeline extends LocalReplica
     // may differ from requestedChecksum for appends.
     final DataChecksum checksum;
 
-    RandomAccessFile metaRAF = new RandomAccessFile(metaFile, "rw");
+    final RandomAccessFile metaRAF =
+        getFileIoProvider().getRandomAccessFile(getVolume(), 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);
+        BlockMetadataHeader header =
+            BlockMetadataHeader.readHeader(metaRAF);
         checksum = header.getChecksum();
 
         if (checksum.getBytesPerChecksum() !=
@@ -302,20 +303,24 @@ public class LocalReplicaInPipeline extends LocalReplica
       checksum = requestedChecksum;
     }
 
+    final FileIoProvider fileIoProvider = getFileIoProvider();
     FileOutputStream blockOut = null;
     FileOutputStream crcOut = null;
     try {
-      blockOut = new FileOutputStream(
-          new RandomAccessFile(blockFile, "rw").getFD());
-      crcOut = new FileOutputStream(metaRAF.getFD());
+      blockOut = fileIoProvider.getFileOutputStream(
+          getVolume(),
+          fileIoProvider.getRandomAccessFile(getVolume(), blockFile, "rw")
+              .getFD());
+      crcOut = fileIoProvider.getFileOutputStream(getVolume(), metaRAF.getFD());
       if (!isCreate) {
         blockOut.getChannel().position(blockDiskSize);
         crcOut.getChannel().position(crcDiskSize);
       }
       return new ReplicaOutputStreams(blockOut, crcOut, checksum,
-          getVolume().isTransientStorage(), slowLogThresholdMs);
+          getVolume(), fileIoProvider);
     } catch (IOException e) {
       IOUtils.closeStream(blockOut);
+      IOUtils.closeStream(crcOut);
       IOUtils.closeStream(metaRAF);
       throw e;
     }
@@ -326,11 +331,11 @@ public class LocalReplicaInPipeline extends LocalReplica
     File blockFile = getBlockFile();
     File restartMeta = new File(blockFile.getParent()  +
         File.pathSeparator + "." + blockFile.getName() + ".restart");
-    if (restartMeta.exists() && !restartMeta.delete()) {
+    if (!getFileIoProvider().deleteWithExistsCheck(getVolume(), restartMeta)) {
       DataNode.LOG.warn("Failed to delete restart meta file: " +
           restartMeta.getPath());
     }
-    return new FileOutputStream(restartMeta);
+    return getFileIoProvider().getFileOutputStream(getVolume(), restartMeta);
   }
 
   @Override
@@ -373,12 +378,14 @@ public class LocalReplicaInPipeline extends LocalReplica
           + " should be derived from LocalReplica");
     }
 
-    LocalReplica oldReplica = (LocalReplica) oldReplicaInfo;
-    File oldmeta = oldReplica.getMetaFile();
-    File newmeta = getMetaFile();
+    final LocalReplica oldReplica = (LocalReplica) oldReplicaInfo;
+    final File oldBlockFile = oldReplica.getBlockFile();
+    final File oldmeta = oldReplica.getMetaFile();
+    final File newmeta = getMetaFile();
+    final FileIoProvider fileIoProvider = getFileIoProvider();
 
     try {
-      oldReplica.renameMeta(newmeta);
+      fileIoProvider.rename(getVolume(), oldmeta, newmeta);
     } catch (IOException e) {
       throw new IOException("Block " + oldReplicaInfo + " reopen failed. " +
                             " Unable to move meta file  " + oldmeta +
@@ -386,10 +393,10 @@ public class LocalReplicaInPipeline extends LocalReplica
     }
 
     try {
-      oldReplica.renameBlock(newBlkFile);
+      fileIoProvider.rename(getVolume(), oldBlockFile, newBlkFile);
     } catch (IOException e) {
       try {
-        renameMeta(oldmeta);
+        fileIoProvider.rename(getVolume(), newmeta, oldmeta);
       } catch (IOException ex) {
         LOG.warn("Cannot move meta file " + newmeta +
             "back to the finalized directory " + oldmeta, ex);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba9587d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
index 5fdbec0..efa6ea6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
@@ -69,13 +69,11 @@ public interface ReplicaInPipeline extends Replica {
    *
    * @param isCreate if it is for creation
    * @param requestedChecksum the checksum the writer would prefer to use
-   * @param slowLogThresholdMs slow io threshold for logging
    * @return output streams for writing
    * @throws IOException if any error occurs
    */
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum, long slowLogThresholdMs)
-      throws IOException;
+      DataChecksum requestedChecksum) throws IOException;
 
   /**
    * Create an output stream to write restart metadata in case of datanode


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/29] hadoop git commit: HADOOP-11804. Shaded Hadoop client artifacts and minicluster. Contributed by Sean Busbey.

Posted by as...@apache.org.
HADOOP-11804. Shaded Hadoop client artifacts and minicluster. Contributed by Sean Busbey.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/70ca1f1e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/70ca1f1e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/70ca1f1e

Branch: refs/heads/YARN-5085
Commit: 70ca1f1e3a328b18eb4e27f7d0f328ae403342d5
Parents: 36947f7
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Dec 15 11:44:59 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Dec 15 11:44:59 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 dev-support/bin/dist-layout-stitching           |   8 +-
 hadoop-client-modules/hadoop-client-api/pom.xml | 216 ++++++
 .../hadoop-client-check-invariants/pom.xml      | 124 +++
 .../hadoop-client-check-test-invariants/pom.xml | 132 ++++
 .../hadoop-client-integration-tests/pom.xml     | 130 ++++
 .../apache/hadoop/example/ITUseMiniCluster.java | 113 +++
 .../src/test/resources/hdfs-site.xml            |  34 +
 .../src/test/resources/log4j.properties         |  24 +
 .../hadoop-client-minicluster/pom.xml           | 745 +++++++++++++++++++
 .../hadoop-client-runtime/pom.xml               | 318 ++++++++
 hadoop-client-modules/hadoop-client/pom.xml     | 338 +++++++++
 hadoop-client-modules/pom.xml                   |  45 ++
 hadoop-client/pom.xml                           | 319 --------
 hadoop-dist/pom.xml                             |  17 +
 hadoop-maven-plugins/pom.xml                    |  22 +
 .../resource/ServicesResourceTransformer.java   | 164 ++++
 .../plugin/shade/resource/package-info.java     |  23 +
 hadoop-project/pom.xml                          |  96 ++-
 pom.xml                                         |   4 +-
 20 files changed, 2544 insertions(+), 329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index d7e321e..eb98a3f 100644
--- a/.gitignore
+++ b/.gitignore
@@ -15,6 +15,7 @@
 .settings
 target
 build
+dependency-reduced-pom.xml
 
 # External tool builders
 */.externalToolBuilders

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/dev-support/bin/dist-layout-stitching
----------------------------------------------------------------------
diff --git a/dev-support/bin/dist-layout-stitching b/dev-support/bin/dist-layout-stitching
index 40ccdc7..fdbc83c 100755
--- a/dev-support/bin/dist-layout-stitching
+++ b/dev-support/bin/dist-layout-stitching
@@ -137,6 +137,12 @@ run copy "${ROOT}/hadoop-tools/hadoop-tools-dist/target/hadoop-tools-dist-${VERS
 run cp -pr "${ROOT}/hadoop-hdfs-project/hadoop-hdfs-httpfs/target/hadoop-hdfs-httpfs-${VERSION}"/* .
 run cp -pr "${ROOT}/hadoop-common-project/hadoop-kms/target/hadoop-kms-${VERSION}"/* .
 
+# copy client jars as-is
+run mkdir -p "share/hadoop/client"
+run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-api/target/hadoop-client-api-${VERSION}.jar" share/hadoop/client/
+run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-runtime/target/hadoop-client-runtime-${VERSION}.jar" share/hadoop/client/
+run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-minicluster/target/hadoop-client-minicluster-${VERSION}.jar" share/hadoop/client/
+
 echo
 echo "Hadoop dist layout available at: ${BASEDIR}/hadoop-${VERSION}"
-echo
\ No newline at end of file
+echo

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client-api/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-api/pom.xml b/hadoop-client-modules/hadoop-client-api/pom.xml
new file mode 100644
index 0000000..9342d53
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client-api/pom.xml
@@ -0,0 +1,216 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+<parent>
+   <groupId>org.apache.hadoop</groupId>
+   <artifactId>hadoop-project</artifactId>
+   <version>3.0.0-alpha2-SNAPSHOT</version>
+   <relativePath>../../hadoop-project</relativePath>
+</parent>
+  <artifactId>hadoop-client-api</artifactId>
+  <version>3.0.0-alpha2-SNAPSHOT</version>
+  <packaging>jar</packaging>
+
+  <description>Apache Hadoop Client</description>
+  <name>Apache Hadoop Client API</name>
+
+  <properties>
+    <shaded.dependency.prefix>org.apache.hadoop.shaded</shaded.dependency.prefix>
+    <!-- We contain no source -->
+    <maven.javadoc.skip>true</maven.javadoc.skip>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <!-- We list this as optional because as a type-pom it won't get included in the shading.
+           Marking it optional means it doesn't count as a transitive dependency of this artifact.
+        -->
+      <optional>true</optional>
+      <exclusions>
+        <!-- these APIs are a part of the SE JDK -->
+        <exclusion>
+          <groupId>javax.xml.bind</groupId>
+          <artifactId>jaxb-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>xml-apis</groupId>
+          <artifactId>xml-apis</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- This comes from our parent pom. If we don't expressly change it here to get included,
+         downstream will get warnings at compile time. -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-annotations</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>jdk.tools</groupId>
+          <artifactId>jdk.tools</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <!-- We contain no source -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+          <configuration>
+            <skipSource>true</skipSource>
+          </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
+            <version>${project.version}</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <artifactSet>
+                <includes>
+                  <include>org.apache.hadoop:*</include>
+                </includes>
+              </artifactSet>
+              <filters>
+                <!-- We get these package level classes from various yarn api jars -->
+                <filter>
+                  <artifact>org.apache.hadoop:hadoop-yarn-common</artifact>
+                  <excludes>
+                    <exclude>org/apache/hadoop/yarn/factories/package-info.class</exclude>
+                    <exclude>org/apache/hadoop/yarn/util/package-info.class</exclude>
+                    <exclude>org/apache/hadoop/yarn/factory/providers/package-info.class</exclude>
+                    <exclude>org/apache/hadoop/yarn/client/api/impl/package-info.class</exclude>
+                    <exclude>org/apache/hadoop/yarn/client/api/package-info.class</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+              <relocations>
+                <relocation>
+                  <pattern>org/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.org.</shadedPattern>
+                  <excludes>
+                    <exclude>org/apache/hadoop/*</exclude>
+                    <exclude>org/apache/hadoop/**/*</exclude>
+                    <!-- Our non-shaded htrace and logging libraries -->
+                    <exclude>org/apache/htrace/*</exclude>
+                    <exclude>org/apache/htrace/**/*</exclude>
+                    <exclude>org/slf4j/*</exclude>
+                    <exclude>org/slf4j/**/*</exclude>
+                    <exclude>org/apache/commons/logging/*</exclude>
+                    <exclude>org/apache/commons/logging/**/*</exclude>
+                    <exclude>org/apache/log4j/*</exclude>
+                    <exclude>org/apache/log4j/**/*</exclude>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Not the org/ packages that are a part of the jdk -->
+                    <exclude>org/ietf/jgss/*</exclude>
+                    <exclude>org/omg/**/*</exclude>
+                    <exclude>org/w3c/dom/*</exclude>
+                    <exclude>org/w3c/dom/**/*</exclude>
+                    <exclude>org/xml/sax/*</exclude>
+                    <exclude>org/xml/sax/**/*</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>com/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.com.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Not the com/ packages that are a part of particular jdk implementations -->
+                    <exclude>com/sun/tools/*</exclude>
+                    <exclude>com/sun/javadoc/*</exclude>
+                    <exclude>com/sun/security/*</exclude>
+                    <exclude>com/sun/jndi/*</exclude>
+                    <exclude>com/sun/management/*</exclude>
+                    <exclude>com/sun/tools/**/*</exclude>
+                    <exclude>com/sun/javadoc/**/*</exclude>
+                    <exclude>com/sun/security/**/*</exclude>
+                    <exclude>com/sun/jndi/**/*</exclude>
+                    <exclude>com/sun/management/**/*</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>io/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.io.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Exclude config keys for Hadoop that look like package names -->
+                    <exclude>io/compression/*</exclude>
+                    <exclude>io/compression/**/*</exclude>
+                    <exclude>io/mapfile/*</exclude>
+                    <exclude>io/mapfile/**/*</exclude>
+                    <exclude>io/map/index/*</exclude>
+                    <exclude>io/seqfile/*</exclude>
+                    <exclude>io/seqfile/**/*</exclude>
+                    <exclude>io/file/buffer/size</exclude>
+                    <exclude>io/skip/checksum/errors</exclude>
+                    <exclude>io/sort/*</exclude>
+                    <exclude>io/serializations</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>javax/servlet/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.javax.servlet.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>net/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.net.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Exclude config keys for Hadoop that look like package names -->
+                    <exclude>net/topology/*</exclude>
+                    <exclude>net/topology/**/*</exclude>
+                  </excludes>
+                </relocation>
+              </relocations>
+              <transformers>
+                <!-- Needed until MSHADE-182 -->
+                <transformer implementation="org.apache.hadoop.maven.plugin.shade.resource.ServicesResourceTransformer"/>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"/>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
+                    <addHeader>false</addHeader>
+                </transformer>
+              </transformers>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>license-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-check-invariants/pom.xml b/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
new file mode 100644
index 0000000..f28816f
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
@@ -0,0 +1,124 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-alpha2-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-client-check-invariants</artifactId>
+  <version>3.0.0-alpha2-SNAPSHOT</version>
+  <packaging>pom</packaging>
+
+  <description>Enforces our invariants for the api and runtime client modules.</description>
+  <name>Apache Hadoop Client Packaging Invariants</name>
+
+  <properties>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-runtime</artifactId>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <version>1.4</version>
+        <dependencies>
+          <dependency>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>extra-enforcer-rules</artifactId>
+            <version>1.0-beta-3</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <id>enforce-banned-dependencies</id>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <banTransitiveDependencies>
+<!--
+                  <message>
+    Our client-facing artifacts are not supposed to have additional dependencies
+    and one or more of them do. The output from the enforcer plugin should give
+    specifics.
+                  </message>
+-->
+                  <excludes>
+                    <!-- annotations is provided, and both artifacts exclude the tools transitive,
+                         but enforcer still sees it.
+                    -->
+                    <exclude>org.apache.hadoop:hadoop-annotations</exclude>
+                    <!-- We leave HTrace as an unshaded dependnecy on purpose so that tracing within a JVM will work -->
+                    <exclude>org.apache.htrace:htrace-core4</exclude>
+                    <!-- Leave slf4j unshaded so downstream users can configure logging. -->
+                    <exclude>org.slf4j:slf4j-api</exclude>
+                    <!-- Leave commons-logging unshaded so downstream users can configure logging. -->
+                    <exclude>commons-logging:commons-logging</exclude>
+                    <!-- Leave log4j unshaded so downstream users can configure logging. -->
+                    <exclude>log4j:log4j</exclude>
+                  </excludes>
+                </banTransitiveDependencies>
+                <banDuplicateClasses>
+                  <findAllDuplicates>true</findAllDuplicates>
+                  <dependencies>
+                    <dependency>
+                      <groupId>org.apache.hadoop</groupId>
+                      <artifactId>hadoop-annotations</artifactId>
+                      <ignoreClasses>
+                        <ignoreClass>*</ignoreClass>
+                      </ignoreClasses>
+                    </dependency>
+                  </dependencies>
+                </banDuplicateClasses>
+              </rules>
+<!-- TODO we need a rule for "we don't have classes that are outside of the org.apache.hadoop package" -->
+<!-- TODO we need a rule for "the constants in this set of classes haven't been shaded / don't have this prefix"
+     Manually checking the set of Keys that look like packages we relocate:
+
+      cat `find . \( -name '*Keys.java' -o -name '*KeysPublic.java' \) -a -path '*/src/main/*'`  | grep -E "\"(io\.|org\.|com\.|net\.)" | grep -v "^package" | grep -v "^import" | grep -v "\"org.apache.hadoop"
+
+     Manually check the set of shaded artifacts to see if the Keys constants have been relocated:
+
+     for clazz in `find . \( -name '*Keys.java' -o -name '*KeysPublic.java' \) -a -path '*/src/main/*'`; do
+       clazz=${clazz#*src/main/java/}
+       clazz="${clazz%.java}"
+       javap -cp hadoop-client-modules/hadoop-client-api/target/hadoop-client-api-3.0.0-alpha2-SNAPSHOT.jar:hadoop-client-modules/hadoop-client-runtime/target/hadoop-client-runtime-3.0.0-alpha2-SNAPSHOT.jar:hadoop-client-modules/hadoop-client-minicluster/target/hadoop-client-minicluster-3.0.0-alpha2-SNAPSHOT.jar \
+           -constants "${clazz//\//.}" | grep "org.apache.hadoop.shaded"
+     done
+-->
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml b/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
new file mode 100644
index 0000000..23ab2ca
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
@@ -0,0 +1,132 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-alpha2-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-client-check-test-invariants</artifactId>
+  <version>3.0.0-alpha2-SNAPSHOT</version>
+  <packaging>pom</packaging>
+
+  <description>Enforces our invariants for the testing client modules.</description>
+  <name>Apache Hadoop Client Packaging Invariants for Test</name>
+
+  <properties>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-runtime</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-minicluster</artifactId>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <version>1.4</version>
+        <dependencies>
+          <dependency>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>extra-enforcer-rules</artifactId>
+            <version>1.0-beta-3</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <id>enforce-banned-dependencies</id>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <banTransitiveDependencies>
+<!--
+                  <message>
+    Our client-facing artifacts are not supposed to have additional dependencies
+    and one or more of them do. The output from the enforcer plugin should give
+    specifics.
+                  </message>
+-->
+                  <excludes>
+                    <!-- annotations is provided, and both artifacts exclude the tools transitive,
+                         but enforcer still sees it.
+                    -->
+                    <exclude>org.apache.hadoop:hadoop-annotations</exclude>
+                    <!-- We leave HTrace as an unshaded dependnecy on purpose so that tracing within a JVM will work -->
+                    <exclude>org.apache.htrace:htrace-core4</exclude>
+                    <!-- Leave slf4j unshaded so downstream users can configure logging. -->
+                    <exclude>org.slf4j:slf4j-api</exclude>
+                    <!-- Leave commons-logging unshaded so downstream users can configure logging. -->
+                    <exclude>commons-logging:commons-logging</exclude>
+                    <!-- Leave log4j unshaded so downstream users can configure logging. -->
+                    <exclude>log4j:log4j</exclude>
+                    <!-- Leave JUnit unshaded so downstream can use our test helper classes -->
+                    <exclude>junit:junit</exclude>
+                    <!-- JUnit brings in hamcrest -->
+                    <exclude> org.hamcrest:hamcrest-core</exclude>
+                  </excludes>
+                </banTransitiveDependencies>
+                <banDuplicateClasses>
+                  <findAllDuplicates>true</findAllDuplicates>
+                  <dependencies>
+                    <dependency>
+                      <groupId>org.apache.hadoop</groupId>
+                      <artifactId>hadoop-annotations</artifactId>
+                      <ignoreClasses>
+                        <ignoreClass>*</ignoreClass>
+                      </ignoreClasses>
+                    </dependency>
+                  </dependencies>
+                </banDuplicateClasses>
+              </rules>
+<!-- TODO we need a rule for "we don't have classes that are outside of the org.apache.hadoop package" -->
+<!-- TODO we need a rule for "the constants in this set of classes haven't been shaded / don't have this prefix"
+     Manually checking the set of Keys that look like packages we relocate:
+
+      cat `find . \( -name '*Keys.java' -o -name '*KeysPublic.java' \) -a -path '*/src/main/*'`  | grep -E "\"(io\.|org\.|com\.|net\.)" | grep -v "^package" | grep -v "^import" | grep -v "\"org.apache.hadoop"
+
+     Manually check the set of shaded artifacts to see if the Keys constants have been relocated:
+
+     for clazz in `find . \( -name '*Keys.java' -o -name '*KeysPublic.java' \) -a -path '*/src/main/*'`; do
+       clazz=${clazz#*src/main/java/}
+       clazz="${clazz%.java}"
+       javap -cp hadoop-client-modules/hadoop-client-api/target/hadoop-client-api-3.0.0-alpha2-SNAPSHOT.jar:hadoop-client-modules/hadoop-client-runtime/target/hadoop-client-runtime-3.0.0-alpha2-SNAPSHOT.jar:hadoop-client-modules/hadoop-client-minicluster/target/hadoop-client-minicluster-3.0.0-alpha2-SNAPSHOT.jar \
+           -constants "${clazz//\//.}" | grep "org.apache.hadoop.shaded"
+     done
+-->
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client-integration-tests/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-integration-tests/pom.xml b/hadoop-client-modules/hadoop-client-integration-tests/pom.xml
new file mode 100644
index 0000000..0df6776
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client-integration-tests/pom.xml
@@ -0,0 +1,130 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-alpha2-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-client-integration-tests</artifactId>
+  <version>3.0.0-alpha2-SNAPSHOT</version>
+
+  <description>Checks that we can use the generated artifacts</description>
+  <name>Apache Hadoop Client Packaging Integration Tests</name>
+
+  <properties>
+    <failsafe.timeout>400</failsafe.timeout>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-runtime</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-minicluster</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <!-- Because our tests rely on our shaded artifacts, we can't compile
+           them until after the package phase has run.
+        -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <executions>
+        <!--
+           First, let's make sure the normal test-compile doesn't try to
+           compile our integration tests.
+        -->
+          <execution>
+            <id>default-testCompile</id>
+            <phase>test-compile</phase>
+            <configuration>
+              <testExcludes>
+                <testExclude>**/IT*</testExclude>
+                <testExclude>**/*IT</testExclude>
+              </testExcludes>
+            </configuration>
+          </execution>
+        <!--
+           Finally, let's make a 'just for integration tests'-compile that
+           fires off prior to our integration tests but after the package
+           phase has created our shaded artifacts.
+        -->
+          <execution>
+            <id>compile-integration-tests</id>
+            <phase>pre-integration-test</phase>
+            <goals>
+              <goal>testCompile</goal>
+            </goals>
+            <configuration>
+              <testIncludes>
+                <testInclude>**/IT*</testInclude>
+                <testInclude>**/*IT</testInclude>
+              </testIncludes>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-failsafe-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>integration-test</goal>
+              <goal>verify</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client-integration-tests/src/test/java/org/apache/hadoop/example/ITUseMiniCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-integration-tests/src/test/java/org/apache/hadoop/example/ITUseMiniCluster.java b/hadoop-client-modules/hadoop-client-integration-tests/src/test/java/org/apache/hadoop/example/ITUseMiniCluster.java
new file mode 100644
index 0000000..3f560d0
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client-integration-tests/src/test/java/org/apache/hadoop/example/ITUseMiniCluster.java
@@ -0,0 +1,113 @@
+/*
+ * 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.example;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+
+/**
+ * Ensure that we can perform operations against the shaded minicluster
+ * given the API and runtime jars by performing some simple smoke tests.
+ */
+public class ITUseMiniCluster {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITUseMiniCluster.class);
+
+  private MiniDFSCluster cluster;
+
+  private static final String TEST_PATH = "/foo/bar/cats/dee";
+  private static final String FILENAME = "test.file";
+
+  private static final String TEXT = "Lorem ipsum dolor sit amet, consectetur "
+      + "adipiscing elit, sed do eiusmod tempor incididunt ut labore et dolore "
+      + "magna aliqua. Ut enim ad minim veniam, quis nostrud exercitation "
+      + "ullamco laboris nisi ut aliquip ex ea commodo consequat. Duis aute "
+      + "irure dolor in reprehenderit in voluptate velit esse cillum dolore eu "
+      + "fugiat nulla pariatur. Excepteur sint occaecat cupidatat non proident,"
+      + " sunt in culpa qui officia deserunt mollit anim id est laborum.";
+
+  @Before
+  public void clusterUp() throws IOException {
+    final Configuration conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3)
+        .build();
+    cluster.waitActive();
+  }
+
+  @After
+  public void clusterDown() {
+    cluster.close();
+  }
+
+  @Test
+  public void useHdfsFileSystem() throws IOException {
+    try (final FileSystem fs = cluster.getFileSystem()) {
+      simpleReadAfterWrite(fs);
+    }
+  }
+
+  public void simpleReadAfterWrite(final FileSystem fs) throws IOException {
+    LOG.info("Testing read-after-write with FS implementation: {}", fs);
+    final Path path = new Path(TEST_PATH, FILENAME);
+    if (!fs.mkdirs(path.getParent())) {
+      throw new IOException("Mkdirs failed to create " +
+          TEST_PATH);
+    }
+    try (final FSDataOutputStream out = fs.create(path)) {
+      out.writeUTF(TEXT);
+    }
+    try (final FSDataInputStream in = fs.open(path)) {
+      final String result = in.readUTF();
+      Assert.assertEquals("Didn't read back text we wrote.", TEXT, result);
+    }
+  }
+
+  @Test
+  public void useWebHDFS() throws IOException, URISyntaxException {
+    try (final FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(
+        cluster.getConfiguration(0), WebHdfsConstants.WEBHDFS_SCHEME)) {
+      simpleReadAfterWrite(fs);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client-integration-tests/src/test/resources/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-integration-tests/src/test/resources/hdfs-site.xml b/hadoop-client-modules/hadoop-client-integration-tests/src/test/resources/hdfs-site.xml
new file mode 100644
index 0000000..cd13532
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client-integration-tests/src/test/resources/hdfs-site.xml
@@ -0,0 +1,34 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+   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.
+-->
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+  <!-- Turn security off for tests by default -->
+  <property>
+    <name>hadoop.security.authentication</name>
+    <value>simple</value>
+  </property>
+  <!-- Disable min block size since most tests use tiny blocks -->
+  <property>
+    <name>dfs.namenode.fs-limits.min-block-size</name>
+    <value>0</value>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client-integration-tests/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-integration-tests/src/test/resources/log4j.properties b/hadoop-client-modules/hadoop-client-integration-tests/src/test/resources/log4j.properties
new file mode 100644
index 0000000..db66160
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client-integration-tests/src/test/resources/log4j.properties
@@ -0,0 +1,24 @@
+#
+#   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.
+#
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client-minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-minicluster/pom.xml b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
new file mode 100644
index 0000000..8ab70d8
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
@@ -0,0 +1,745 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-alpha2-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-client-minicluster</artifactId>
+  <version>3.0.0-alpha2-SNAPSHOT</version>
+  <packaging>jar</packaging>
+
+  <description>Apache Hadoop Minicluster for Clients</description>
+  <name>Apache Hadoop Client Test Minicluster</name>
+
+  <properties>
+    <shaded.dependency.prefix>org.apache.hadoop.shaded</shaded.dependency.prefix>
+    <!-- We contain no source -->
+    <maven.javadoc.skip>true</maven.javadoc.skip>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-api</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-runtime</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <!-- Leave JUnit as a direct dependency -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <!-- Adding hadoop-annotations so we can make it optional to remove from our transitive tree -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-annotations</artifactId>
+      <scope>compile</scope>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>jdk.tools</groupId>
+          <artifactId>jdk.tools</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- uncomment this dependency if you need to use
+         `mvn dependency:tree -Dverbose` to determine if a dependency shows up
+         in both the hadoop-client-* artifacts and something under minicluster.
+    -->
+<!--
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <scope>provided</scope>
+    </dependency>
+ -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <optional>true</optional>
+      <exclusions>
+        <!-- Exclude the in-development timeline service and
+             add it as an optional runtime dependency
+          -->
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+        </exclusion>
+        <!-- exclude everything that comes in via the shaded runtime and api  TODO remove once we have a filter for "is in these artifacts" -->
+        <!-- Skip jersey, since we need it again here. -->
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs-client</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-mapreduce-client-app</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+        </exclusion>
+        <!-- exclude things that came in via transitive in shaded runtime and api -->
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>javax.servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.xml.bind</groupId>
+          <artifactId>jaxb-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.curator</groupId>
+          <artifactId>curator-client</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-mapreduce-client-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-server-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.zookeeper</groupId>
+          <artifactId>zookeeper</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.fusesource.leveldbjni</groupId>
+          <artifactId>leveldbjni-all</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-util</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.protobuf</groupId>
+          <artifactId>protobuf-java</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-collections</groupId>
+          <artifactId>commons-collections</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-io</groupId>
+          <artifactId>commons-io</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-lang</groupId>
+          <artifactId>commons-lang</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>log4j</groupId>
+          <artifactId>log4j</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-databind</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-cli</groupId>
+          <artifactId>commons-cli</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-codec</groupId>
+          <artifactId>commons-codec</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>javax.servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>xmlenc</groupId>
+          <artifactId>xmlenc</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- Add optional runtime dependency on the in-development timeline server module
+         to indicate that downstream folks interested in turning it on need that dep.
+      -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+      <scope>runtime</scope>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- Add back in transitive dependencies of hadoop-minicluster that are test-jar artifacts excluded as a side effect of excluding the jar
+         Note that all of these must be marked "optional" because they won't be removed from the reduced-dependencies pom after they're included.
+      -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>compile</scope>
+      <type>test-jar</type>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>compile</scope>
+      <type>test-jar</type>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+      <scope>compile</scope>
+      <type>test-jar</type>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- Add back in the transitive dependencies excluded from hadoop-common in client TODO remove once we have a filter for "is in these artifacts" -->
+    <!-- skip javax.servlet:servlet-api because it's in client -->
+    <!-- Skip commons-logging:commons-logging-api because it looks like nothing actually included it -->
+    <!-- Skip jetty-util because it's in client -->
+    <dependency>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-core</artifactId>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.ws.rs</groupId>
+          <artifactId>jsr311-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-client</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-json</artifactId>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.xml.bind</groupId>
+          <artifactId>jaxb-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.codehaus.jackson</groupId>
+          <artifactId>jackson-core-asl</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.codehaus.jackson</groupId>
+          <artifactId>jackson-mapper-asl</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.codehaus.jackson</groupId>
+          <artifactId>jackson-jaxrs</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.codehaus.jackson</groupId>
+          <artifactId>jackson-xc</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-server</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-servlet</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jdt</groupId>
+      <artifactId>core</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <!-- skip org.apache.avro:avro-ipc because it doesn't look like hadoop-common actually uses it -->
+    <dependency>
+      <groupId>net.sf.kosmosfs</groupId>
+      <artifactId>kfs</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>net.java.dev.jets3t</groupId>
+      <artifactId>jets3t</artifactId>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-codec</groupId>
+          <artifactId>commons-codec</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.httpcomponents</groupId>
+          <artifactId>httpclient</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.httpcomponents</groupId>
+          <artifactId>httpcore</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>com.jcraft</groupId>
+      <artifactId>jsch</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <!-- add back in transitive dependencies of hadoop-mapreduce-client-app removed in client -->
+    <!-- Skipping javax.servlet:servlet-api because it's in client -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-nodemanager</artifactId>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>javax.servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-server-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.fusesource.leveldbjni</groupId>
+          <artifactId>leveldbjni-all</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-util</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.protobuf</groupId>
+          <artifactId>protobuf-java</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-codec</groupId>
+          <artifactId>commons-codec</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-lang</groupId>
+          <artifactId>commons-lang</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.xml.bind</groupId>
+          <artifactId>jaxb-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-web-proxy</artifactId>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>javax.servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-server-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-httpclient</groupId>
+          <artifactId>commons-httpclient</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-util</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- skipping hadoop-annotations -->
+    <dependency>
+      <groupId>com.google.inject.extensions</groupId>
+      <artifactId>guice-servlet</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <!-- skipping junit:junit because it is test scope -->
+    <!-- skipping avro because it is in client via hadoop-common -->
+    <!-- skipping jline:jline because it is only present at test scope in the original -->
+    <!-- skipping io.netty:netty because it's in client -->
+    <!-- add back in transitive dependencies of hadoop-yarn-api removed in client -->
+    <!-- skipping hadoop-annotations -->
+    <dependency>
+      <groupId>com.google.inject</groupId>
+      <artifactId>guice</artifactId>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey.jersey-test-framework</groupId>
+      <artifactId>jersey-test-framework-grizzly2</artifactId>
+      <optional>true</optional>
+      <exclusions>
+        <!-- excluding because client already has the tomcat version -->
+        <exclusion>
+          <groupId>org.glassfish</groupId>
+          <artifactId>javax.servlet</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- skipping jersey-server because it's above -->
+    <dependency>
+      <groupId>com.sun.jersey.contribs</groupId>
+      <artifactId>jersey-guice</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <!-- skipping guice-servlet because it's above -->
+    <!-- skipping avro because it is in client via hadoop-common -->
+    <!-- skipping jersey-core because it's above -->
+    <!-- skipping jersey-json because it's above. -->
+    <!-- skipping io.netty:netty because it's in client -->
+    <!-- Add back in transitive dependencies from hadoop-mapreduce-client-core that were excluded by client -->
+    <!-- skipping junit:junit because it is test scope -->
+    <!-- skipping guice because it's above -->
+    <!-- skipping jersey-test-framework-grizzly2 because it's above -->
+    <!-- skipping jersey-server because it's above -->
+    <!-- skipping jersey-guice because it's above -->
+    <!-- skipping avro because it is in client via hadoop-common -->
+    <!-- skipping hadoop-annotations -->
+    <!-- skipping guice-servlet because it's above -->
+    <!-- skipping jersey-json because it's above. -->
+    <!-- skipping io.netty:netty because it's in client -->
+    <!-- add back in transitive dependencies of hadoop-mapreduce-client-jobclient that were excluded from client -->
+    <!-- skipping junit:junit because it is test scope -->
+    <!-- skipping avro because it is in client via hadoop-common -->
+    <!-- skipping hadoop-annotations -->
+    <!-- skipping guice-servlet because it's above -->
+    <!-- skipping io.netty:netty because it's in client -->
+  </dependencies>
+  <build>
+    <plugins>
+      <!-- We contain no source -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+          <configuration>
+            <skipSource>true</skipSource>
+          </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
+            <version>${project.version}</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <createSourceJar>true</createSourceJar>
+              <artifactSet>
+                <excludes>
+                  <!-- Fine to expose our already-shaded deps as dependencies -->
+                  <exclude>org.apache.hadoop:hadoop-annotations</exclude>
+                  <exclude>org.apache.hadoop:hadoop-client-api</exclude>
+                  <exclude>org.apache.hadoop:hadoop-client-runtime</exclude>
+                  <!-- Fine to expose our purposefully not-shaded deps as dependencies -->
+                  <exclude>org.apache.htrace:htrace-core4</exclude>
+                  <exclude>org.slf4j:slf4j-api</exclude>
+                  <exclude>commons-logging:commons-logging</exclude>
+                  <exclude>junit:junit</exclude>
+                  <!-- Keep optional runtime deps out of the shading -->
+                  <exclude>org.apache.hadoop:hadoop-yarn-server-timelineservice</exclude>
+                  <exclude>log4j:log4j</exclude>
+                  <!-- We need a filter that matches just those things that are included in the above artiacts -->
+                </excludes>
+              </artifactSet>
+              <filters>
+                <!-- Some of our dependencies include source, so remove it. -->
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>**/*.java</exclude>
+                  </excludes>
+                </filter>
+                <!-- We pull in several test jars; keep out the actual test classes -->
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>**/Test*.class</exclude>
+                  </excludes>
+                </filter>
+                <!-- Since runtime has classes for these jars, we exclude them.
+                     We still want the java services api files, since those were excluded in runtime
+                  -->
+                <filter>
+                  <artifact>com.sun.jersey:jersey-client</artifact>
+                  <excludes>
+                    <exclude>**/*.class</exclude>
+                  </excludes>
+                </filter>
+                <filter>
+                  <artifact>com.sun.jersey:jersey-core</artifact>
+                  <excludes>
+                    <exclude>**/*.class</exclude>
+                  </excludes>
+                </filter>
+                <filter>
+                  <artifact>com.sun.jersey:jersey-servlet</artifact>
+                  <excludes>
+                    <exclude>**/*.class</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+              <relocations>
+                <relocation>
+                  <pattern>org/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.org.</shadedPattern>
+                  <excludes>
+                    <exclude>org/apache/hadoop/*</exclude>
+                    <exclude>org/apache/hadoop/**/*</exclude>
+                    <!-- Our non-shaded htrace and logging libraries -->
+                    <exclude>org/apache/htrace/*</exclude>
+                    <exclude>org/apache/htrace/**/*</exclude>
+                    <exclude>org/slf4j/*</exclude>
+                    <exclude>org/slf4j/**/*</exclude>
+                    <exclude>org/apache/commons/logging/*</exclude>
+                    <exclude>org/apache/commons/logging/**/*</exclude>
+                    <exclude>org/apache/log4j/*</exclude>
+                    <exclude>org/apache/log4j/**/*</exclude>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Our non-shaded JUnit library -->
+                    <exclude>org/junit/*</exclude>
+                    <exclude>org/junit/**/*</exclude>
+                    <!-- Not the org/ packages that are a part of the jdk -->
+                    <exclude>org/ietf/jgss/*</exclude>
+                    <exclude>org/omg/**/*</exclude>
+                    <exclude>org/w3c/dom/*</exclude>
+                    <exclude>org/w3c/dom/**/*</exclude>
+                    <exclude>org/xml/sax/*</exclude>
+                    <exclude>org/xml/sax/**/*</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>com/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.com.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Not the com/ packages that are a part of particular jdk implementations -->
+                    <exclude>com/sun/tools/*</exclude>
+                    <exclude>com/sun/javadoc/*</exclude>
+                    <exclude>com/sun/security/*</exclude>
+                    <exclude>com/sun/jndi/*</exclude>
+                    <exclude>com/sun/management/*</exclude>
+                    <exclude>com/sun/tools/**/*</exclude>
+                    <exclude>com/sun/javadoc/**/*</exclude>
+                    <exclude>com/sun/security/**/*</exclude>
+                    <exclude>com/sun/jndi/**/*</exclude>
+                    <exclude>com/sun/management/**/*</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>io/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.io.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Exclude config keys for Hadoop that look like package names -->
+                    <exclude>io/compression/*</exclude>
+                    <exclude>io/compression/**/*</exclude>
+                    <exclude>io/mapfile/*</exclude>
+                    <exclude>io/mapfile/**/*</exclude>
+                    <exclude>io/map/index/*</exclude>
+                    <exclude>io/seqfile/*</exclude>
+                    <exclude>io/seqfile/**/*</exclude>
+                    <exclude>io/file/buffer/size</exclude>
+                    <exclude>io/skip/checksum/errors</exclude>
+                    <exclude>io/sort/*</exclude>
+                    <exclude>io/serializations</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>javax/el/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.javax.el.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>javax/inject/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.javax.inject.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>javax/servlet/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.javax.servlet.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>net/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.net.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Exclude config keys for Hadoop that look like package names -->
+                    <exclude>net/topology/*</exclude>
+                    <exclude>net/topology/**/*</exclude>
+                  </excludes>
+                </relocation>
+              </relocations>
+              <transformers>
+                <!-- Needed until MSHADE-182 -->
+                <transformer implementation="org.apache.hadoop.maven.plugin.shade.resource.ServicesResourceTransformer"/>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"/>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
+                  <addHeader>false</addHeader>
+                </transformer>
+              </transformers>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>license-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-runtime/pom.xml b/hadoop-client-modules/hadoop-client-runtime/pom.xml
new file mode 100644
index 0000000..aa1a54b
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client-runtime/pom.xml
@@ -0,0 +1,318 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+<parent>
+   <groupId>org.apache.hadoop</groupId>
+   <artifactId>hadoop-project</artifactId>
+   <version>3.0.0-alpha2-SNAPSHOT</version>
+   <relativePath>../../hadoop-project</relativePath>
+</parent>
+  <artifactId>hadoop-client-runtime</artifactId>
+  <version>3.0.0-alpha2-SNAPSHOT</version>
+  <packaging>jar</packaging>
+
+  <description>Apache Hadoop Client</description>
+  <name>Apache Hadoop Client Runtime</name>
+
+  <properties>
+    <shaded.dependency.prefix>org.apache.hadoop.shaded</shaded.dependency.prefix>
+    <!-- We contain no source -->
+    <maven.javadoc.skip>true</maven.javadoc.skip>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <!-- We list this as optional because as a type-pom it won't get included in the shading.
+           Marking it optional means it doesn't count as a transitive dependency of this artifact.
+        -->
+      <optional>true</optional>
+      <exclusions>
+        <!-- these APIs are a part of the SE JDK -->
+        <exclusion>
+          <groupId>javax.xml.bind</groupId>
+          <artifactId>jaxb-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>xml-apis</groupId>
+          <artifactId>xml-apis</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- At runtime anyone using us must have the api present -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-api</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <!-- This comes from our parent pom. If we don't expressly change it here to get included,
+         downstream will get warnings at compile time. -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-annotations</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>jdk.tools</groupId>
+          <artifactId>jdk.tools</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- Since hadoop-client is listed as optional, we have to list transitive
+         dependencies that we still want to show up.
+         * HTrace
+         * Slf4j API
+         * commons-logging
+      -->
+    <dependency>
+      <groupId>org.apache.htrace</groupId>
+      <artifactId>htrace-core4</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <!-- Move log4j to optional, since it is needed for some pieces folks might not use:
+         * one of the three custom log4j appenders we have
+         * JobConf (?!) (so essentially any user of MapReduce)
+      -->
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <scope>runtime</scope>
+      <optional>true</optional>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <!-- We contain no source -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+          <configuration>
+            <skipSource>true</skipSource>
+          </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
+            <version>${project.version}</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <createSourceJar>true</createSourceJar>
+              <artifactSet>
+                <excludes>
+                  <!-- We need a filter that matches just those things that aer included in the api jar -->
+                  <exclude>org.apache.hadoop:hadoop-client-api</exclude>
+                  <!-- Leave HTrace as an unshaded dependency on purpose, since a static class member is used to trace within a given JVM instance -->
+                  <exclude>org.apache.htrace:htrace-core4</exclude>
+                  <!-- Leave slf4j unshaded so downstream users can configure logging. -->
+                  <exclude>org.slf4j:slf4j-api</exclude>
+                  <!-- Leave commons-logging unshaded so downstream users can configure logging. -->
+                  <exclude>commons-logging:commons-logging</exclude>
+                  <!-- Leave log4j unshaded so downstream users can configure logging. -->
+                  <exclude>log4j:log4j</exclude>
+                </excludes>
+              </artifactSet>
+              <filters>
+                <!-- We need a filter that matches just those things that are included in the api jar -->
+                <filter>
+                  <artifact>org.apache.hadoop:*</artifact>
+                  <excludes>
+                    <exclude>**/*</exclude>
+                    <exclude>*</exclude>
+                  </excludes>
+                </filter>
+                <!-- Some of our dependencies include source, so remove it. -->
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>**/*.java</exclude>
+                  </excludes>
+                </filter>
+                <!-- We only want one copy of the Localizer class. So long as we keep jasper compiler and runtime on the same version, which one doesn't matter -->
+                <filter>
+                  <artifact>tomcat:jasper-compiler</artifact>
+                  <excludes>
+                    <exclude>org/apache/jasper/compiler/Localizer.class</exclude>
+                  </excludes>
+                </filter>
+                <!-- We only have xerces as a dependency for XML output for the fsimage edits, we don't need anything specific to it for javax xml support -->
+                <filter>
+                  <artifact>xerces:xercesImpl</artifact>
+                  <excludes>
+                    <exclude>META-INF/services/*</exclude>
+                  </excludes>
+                </filter>
+                <!-- We rely on jersey for our web interfaces. We want to use its java services stuff only internal to jersey -->
+                <filter>
+                  <artifact>com.sun.jersey:*</artifact>
+                  <excludes>
+                    <exclude>META-INF/services/javax.*</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+              <relocations>
+                <relocation>
+                  <pattern>org/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.org.</shadedPattern>
+                  <excludes>
+                    <exclude>org/apache/hadoop/*</exclude>
+                    <exclude>org/apache/hadoop/**/*</exclude>
+                    <!-- Our non-shaded htrace and logging libraries -->
+                    <exclude>org/apache/htrace/*</exclude>
+                    <exclude>org/apache/htrace/**/*</exclude>
+                    <exclude>org/slf4j/*</exclude>
+                    <exclude>org/slf4j/**/*</exclude>
+                    <exclude>org/apache/commons/logging/*</exclude>
+                    <exclude>org/apache/commons/logging/**/*</exclude>
+                    <exclude>org/apache/log4j/*</exclude>
+                    <exclude>org/apache/log4j/**/*</exclude>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Not the org/ packages that are a part of the jdk -->
+                    <exclude>org/ietf/jgss/*</exclude>
+                    <exclude>org/omg/**/*</exclude>
+                    <exclude>org/w3c/dom/*</exclude>
+                    <exclude>org/w3c/dom/**/*</exclude>
+                    <exclude>org/xml/sax/*</exclude>
+                    <exclude>org/xml/sax/**/*</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>com/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.com.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Not the com/ packages that are a part of particular jdk implementations -->
+                    <exclude>com/sun/tools/*</exclude>
+                    <exclude>com/sun/javadoc/*</exclude>
+                    <exclude>com/sun/security/*</exclude>
+                    <exclude>com/sun/jndi/*</exclude>
+                    <exclude>com/sun/management/*</exclude>
+                    <exclude>com/sun/tools/**/*</exclude>
+                    <exclude>com/sun/javadoc/**/*</exclude>
+                    <exclude>com/sun/security/**/*</exclude>
+                    <exclude>com/sun/jndi/**/*</exclude>
+                    <exclude>com/sun/management/**/*</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>io/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.io.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Exclude config keys for Hadoop that look like package names -->
+                    <exclude>io/compression/*</exclude>
+                    <exclude>io/compression/**/*</exclude>
+                    <exclude>io/mapfile/*</exclude>
+                    <exclude>io/mapfile/**/*</exclude>
+                    <exclude>io/map/index/*</exclude>
+                    <exclude>io/seqfile/*</exclude>
+                    <exclude>io/seqfile/**/*</exclude>
+                    <exclude>io/file/buffer/size</exclude>
+                    <exclude>io/skip/checksum/errors</exclude>
+                    <exclude>io/sort/*</exclude>
+                    <exclude>io/serializations</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>javax/el/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.javax.el.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>javax/servlet/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.javax.servlet.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>net/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.net.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                    <!-- Exclude config keys for Hadoop that look like package names -->
+                    <exclude>net/topology/*</exclude>
+                    <exclude>net/topology/**/*</exclude>
+                  </excludes>
+                </relocation>
+                <!-- probably not. -->
+<!--
+                <relocation>
+                  <pattern>javax/</pattern>
+                  <shadedPattern>${shaded.dependency.prefix}.javax.</shadedPattern>
+                  <excludes>
+                    <exclude>**/pom.xml</exclude>
+                  </excludes>
+                </relocation>
+-->
+              </relocations>
+              <transformers>
+                <!-- Needed until MSHADE-182 -->
+                <transformer implementation="org.apache.hadoop.maven.plugin.shade.resource.ServicesResourceTransformer"/>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"/>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
+                  <addHeader>false</addHeader>
+                </transformer>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.XmlAppendingTransformer">
+                  <resource>META-INF/jboss-beans.xml</resource>
+                  <!-- Add this to enable loading of DTDs
+                  <ignoreDtd>false</ignoreDtd>
+                  -->
+                </transformer>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+                  <resource>META-INF/mailcap.default</resource>
+                </transformer>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+                  <resource>META-INF/mimetypes.default</resource>
+                </transformer>
+              </transformers>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>license-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>
+


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/29] hadoop git commit: HDFS-11204. Document the missing options of hdfs zkfc command. Contributed by Yiqun Lin.

Posted by as...@apache.org.
HDFS-11204. Document the missing options of hdfs zkfc command. Contributed by Yiqun Lin.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/72bff192
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/72bff192
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/72bff192

Branch: refs/heads/YARN-5085
Commit: 72bff192cd37ff97442e0f8dd477fbc2e58fc12d
Parents: 1f14f6d
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Dec 14 18:50:43 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Dec 14 18:50:43 2016 +0900

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ha/ZKFailoverController.java  | 7 +++++--
 .../hadoop-hdfs/src/site/markdown/HDFSCommands.md             | 2 +-
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/72bff192/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
index 30ec8f2..0ed9158 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
@@ -84,8 +84,11 @@ public abstract class ZKFailoverController {
     ZK_AUTH_KEY
   };
   
-  protected static final String USAGE = 
-      "Usage: hdfs zkfc [ -formatZK [-force] [-nonInteractive] ]";
+  protected static final String USAGE =
+      "Usage: hdfs zkfc [ -formatZK [-force] [-nonInteractive] ]\n"
+      + "\t-force: formats the znode if the znode exists.\n"
+      + "\t-nonInteractive: formats the znode aborts if the znode exists,\n"
+      + "\tunless -force option is specified.";
 
   /** Unable to format the parent znode in ZK */
   static final int ERR_CODE_FORMAT_DENIED = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72bff192/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 7f7dcde..a0d0ed7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -599,7 +599,7 @@ Usage: `hdfs zkfc [-formatZK [-force] [-nonInteractive]]`
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-| `-formatZK` | Format the Zookeeper instance |
+| `-formatZK` | Format the Zookeeper instance. -force: formats the znode if the znode exists. -nonInteractive: formats the znode aborts if the znode exists, unless -force option is specified. |
 | `-h` | Display help |
 
 This comamnd starts a Zookeeper Failover Controller process for use with [HDFS HA with QJM](./HDFSHighAvailabilityWithQJM.html#Administrative_commands).


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/29] hadoop git commit: Revert "HADOOP-13709. Ability to clean up subprocesses spawned by Shell when the process exits. Contributed by Eric Badger"

Posted by as...@apache.org.
Revert "HADOOP-13709. Ability to clean up subprocesses spawned by Shell when the process exits. Contributed by Eric Badger"

This reverts commit 9947aeb60c3dd075544866fd6e4dab0ad8b4afa2.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/169bfc09
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/169bfc09
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/169bfc09

Branch: refs/heads/YARN-5085
Commit: 169bfc09037595610eb000fd3a0cb63cc9deca06
Parents: f92913c
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Dec 15 20:30:45 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Dec 15 20:51:24 2016 +0000

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/util/Shell.java | 24 -------
 .../java/org/apache/hadoop/util/TestShell.java  | 68 --------------------
 2 files changed, 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/169bfc09/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
index 83877b7..5fc9869 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
@@ -26,11 +26,9 @@ import java.io.InputStream;
 import java.io.InterruptedIOException;
 import java.nio.charset.Charset;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.Map;
 import java.util.Timer;
 import java.util.TimerTask;
-import java.util.WeakHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -50,8 +48,6 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public abstract class Shell {
-  private static final Map <Process, Object> CHILD_PROCESSES =
-      Collections.synchronizedMap(new WeakHashMap<Process, Object>());
   public static final Logger LOG = LoggerFactory.getLogger(Shell.class);
 
   /**
@@ -920,7 +916,6 @@ public abstract class Shell {
     } else {
       process = builder.start();
     }
-    CHILD_PROCESSES.put(process, null);
 
     if (timeOutInterval > 0) {
       timeOutTimer = new Timer("Shell command timeout");
@@ -1017,7 +1012,6 @@ public abstract class Shell {
         LOG.warn("Error while closing the error stream", ioe);
       }
       process.destroy();
-      CHILD_PROCESSES.remove(process);
       lastTime = Time.monotonicNow();
     }
   }
@@ -1316,22 +1310,4 @@ public abstract class Shell {
       }
     }
   }
-
-  /**
-   * Static method to destroy all running <code>Shell</code> processes
-   * Iterates through a list of all currently running <code>Shell</code>
-   * processes and destroys them one by one. This method is thread safe and
-   * is intended to be used in a shutdown hook.
-   */
-  public static void destroyAllProcesses() {
-    synchronized (CHILD_PROCESSES) {
-      for (Process key : CHILD_PROCESSES.keySet()) {
-        Process process = key;
-        if (key != null) {
-          process.destroy();
-        }
-      }
-      CHILD_PROCESSES.clear();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/169bfc09/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
index 5cc011b..67903f7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.util;
 
-import com.google.common.base.Supplier;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.security.alias.AbstractJavaKeyStoreProvider;
 import org.junit.Assert;
@@ -472,71 +471,4 @@ public class TestShell extends Assert {
     assertEquals("'foo'\\''bar'", Shell.bashQuote("foo'bar"));
     assertEquals("''\\''foo'\\''bar'\\'''", Shell.bashQuote("'foo'bar'"));
   }
-
-  @Test(timeout=120000)
-  public void testShellKillAllProcesses() throws Throwable {
-    Assume.assumeFalse(WINDOWS);
-    StringBuffer sleepCommand = new StringBuffer();
-    sleepCommand.append("sleep 200");
-    String[] shellCmd = {"bash", "-c", sleepCommand.toString()};
-    final ShellCommandExecutor shexc1 = new ShellCommandExecutor(shellCmd);
-    final ShellCommandExecutor shexc2 = new ShellCommandExecutor(shellCmd);
-
-    Thread shellThread1 = new Thread() {
-      @Override
-      public void run() {
-        try {
-          shexc1.execute();
-        } catch(IOException ioe) {
-          //ignore IOException from thread interrupt
-        }
-      }
-    };
-    Thread shellThread2 = new Thread() {
-      @Override
-      public void run() {
-        try {
-          shexc2.execute();
-        } catch(IOException ioe) {
-          //ignore IOException from thread interrupt
-        }
-      }
-    };
-
-    shellThread1.start();
-    shellThread2.start();
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        return shexc1.getProcess() != null;
-      }
-    }, 10, 10000);
-
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        return shexc2.getProcess() != null;
-      }
-    }, 10, 10000);
-
-    Shell.destroyAllProcesses();
-    final Process process1 = shexc1.getProcess();
-    final Process process2 = shexc2.getProcess();
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        return !process1.isAlive();
-      }
-    }, 10, 10000);
-
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        return !process2.isAlive();
-      }
-    }, 10, 10000);
-
-    assertFalse("Process 1 was not killed within timeout", process1.isAlive());
-    assertFalse("Process 2 was not killed within timeout", process2.isAlive());
-  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/29] hadoop git commit: HADOOP-13831. Correct check for error code to detect Azure Storage Throttling and provide retries. Contributed by Gaurav Kanade

Posted by as...@apache.org.
HADOOP-13831. Correct check for error code to detect Azure Storage Throttling and provide retries. Contributed by Gaurav Kanade


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f92913c3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f92913c3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f92913c3

Branch: refs/heads/YARN-5085
Commit: f92913c35bfda0d565606f9fb9a301ddd4105fd8
Parents: 70ca1f1
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Dec 15 12:35:08 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Dec 15 12:35:08 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/fs/azure/AzureNativeFileSystemStore.java    | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f92913c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index ac6c514..d232a2d 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -26,6 +26,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URLDecoder;
@@ -2511,8 +2512,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       try {
         dstBlob.startCopyFromBlob(srcBlob, null, getInstrumentedContext());
       } catch (StorageException se) {
-        if (se.getErrorCode().equals(
-          StorageErrorCode.SERVER_BUSY.toString())) {
+        if (se.getHttpStatusCode() == HttpURLConnection.HTTP_UNAVAILABLE) {
           int copyBlobMinBackoff = sessionConfiguration.getInt(
             KEY_COPYBLOB_MIN_BACKOFF_INTERVAL,
 			DEFAULT_COPYBLOB_MIN_BACKOFF_INTERVAL);
@@ -2541,8 +2541,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       waitForCopyToComplete(dstBlob, getInstrumentedContext());
       safeDelete(srcBlob, lease);
     } catch (StorageException e) {
-      if (e.getErrorCode().equals(
-        StorageErrorCode.SERVER_BUSY.toString())) {
+      if (e.getHttpStatusCode() == HttpURLConnection.HTTP_UNAVAILABLE) {
         LOG.warn("Rename: CopyBlob: StorageException: ServerBusy: Retry complete, will attempt client side copy for page blob");
         InputStream ipStream = null;
         OutputStream opStream = null;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/29] hadoop git commit: HDFS-8411. Add bytes count metrics to datanode for ECWorker. Contributed by Sammi Chen and Andrew Wang

Posted by as...@apache.org.
HDFS-8411. Add bytes count metrics to datanode for ECWorker. Contributed by Sammi Chen and Andrew Wang


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1f14f6d0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1f14f6d0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1f14f6d0

Branch: refs/heads/YARN-5085
Commit: 1f14f6d038aecad55a5398c6fa4137c9d2f44729
Parents: ada876c
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Dec 14 14:50:50 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Dec 14 14:50:50 2016 +0800

----------------------------------------------------------------------
 .../erasurecode/StripedBlockReader.java         |   1 +
 .../erasurecode/StripedBlockReconstructor.java  |   6 +-
 .../erasurecode/StripedBlockWriter.java         |   1 +
 .../datanode/erasurecode/StripedReader.java     |   4 +
 .../erasurecode/StripedReconstructor.java       |  21 +++
 .../datanode/erasurecode/StripedWriter.java     |   4 +
 .../datanode/metrics/DataNodeMetrics.java       |  18 ++-
 .../apache/hadoop/hdfs/StripedFileTestUtil.java |  24 +++
 .../TestDataNodeErasureCodingMetrics.java       | 149 +++++++++----------
 9 files changed, 147 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f14f6d0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
index a27de9b..0f7c5c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
@@ -187,6 +187,7 @@ class StripedBlockReader {
         break;
       }
       n += nread;
+      stripedReader.getReconstructor().incrBytesRead(nread);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f14f6d0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java
index a8e9d30..5554d68 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 
 /**
  * StripedBlockReconstructor reconstruct one or more missed striped block in
@@ -66,7 +67,10 @@ class StripedBlockReconstructor extends StripedReconstructor
       getDatanode().getMetrics().incrECFailedReconstructionTasks();
     } finally {
       getDatanode().decrementXmitsInProgress();
-      getDatanode().getMetrics().incrECReconstructionTasks();
+      final DataNodeMetrics metrics = getDatanode().getMetrics();
+      metrics.incrECReconstructionTasks();
+      metrics.incrECReconstructionBytesRead(getBytesRead());
+      metrics.incrECReconstructionBytesWritten(getBytesWritten());
       getStripedReader().close();
       stripedWriter.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f14f6d0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockWriter.java
index 592be45..d999202 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockWriter.java
@@ -196,6 +196,7 @@ class StripedBlockWriter {
       packet.writeTo(targetOutputStream);
 
       blockOffset4Target += toWrite;
+      stripedWriter.getReconstructor().incrBytesWritten(toWrite);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f14f6d0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReader.java
index 238c628..f6f343a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReader.java
@@ -435,6 +435,10 @@ class StripedReader {
     }
   }
 
+  StripedReconstructor getReconstructor() {
+    return reconstructor;
+  }
+
   StripedBlockReader getReader(int i) {
     return readers.get(i);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f14f6d0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
index 5641c35..68769f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
@@ -41,6 +41,7 @@ import java.util.BitSet;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * StripedReconstructor reconstruct one or more missed striped block in the
@@ -114,6 +115,10 @@ abstract class StripedReconstructor {
   private long maxTargetLength = 0L;
   private final BitSet liveBitSet;
 
+  // metrics
+  private AtomicLong bytesRead = new AtomicLong(0);
+  private AtomicLong bytesWritten = new AtomicLong(0);
+
   StripedReconstructor(ErasureCodingWorker worker,
       StripedReconstructionInfo stripedReconInfo) {
     this.stripedReadPool = worker.getStripedReadPool();
@@ -133,6 +138,22 @@ abstract class StripedReconstructor {
     positionInBlock = 0L;
   }
 
+  public void incrBytesRead(long delta) {
+    bytesRead.addAndGet(delta);
+  }
+
+  public void incrBytesWritten(long delta) {
+    bytesWritten.addAndGet(delta);
+  }
+
+  public long getBytesRead() {
+    return bytesRead.get();
+  }
+
+  public long getBytesWritten() {
+    return bytesWritten.get();
+  }
+
   /**
    * Reconstruct one or more missed striped block in the striped block group,
    * the minimum number of live striped blocks should be no less than data

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f14f6d0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedWriter.java
index c099bc1..225a7ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedWriter.java
@@ -280,6 +280,10 @@ class StripedWriter {
     return reconstructor.getSocketAddress4Transfer(target);
   }
 
+  StripedReconstructor getReconstructor() {
+    return reconstructor;
+  }
+
   boolean hasValidTargets() {
     return hasValidTargets;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f14f6d0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
index 23e15a2..e09a85f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.datanode.metrics;
 
 import static org.apache.hadoop.metrics2.impl.MsInfo.SessionId;
-import static org.apache.hadoop.metrics2.lib.Interns.info;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -135,8 +134,12 @@ public class DataNodeMetrics {
   MutableCounterLong ecReconstructionTasks;
   @Metric("Count of erasure coding failed reconstruction tasks")
   MutableCounterLong ecFailedReconstructionTasks;
-  // Nanoseconds spent by decoding tasks.
+  @Metric("Nanoseconds spent by decoding tasks")
   MutableCounterLong ecDecodingTimeNanos;
+  @Metric("Bytes read by erasure coding worker")
+  MutableCounterLong ecReconstructionBytesRead;
+  @Metric("Bytes written by erasure coding worker")
+  MutableCounterLong ecReconstructionBytesWritten;
 
   final MetricsRegistry registry = new MetricsRegistry("datanode");
   final String name;
@@ -156,9 +159,6 @@ public class DataNodeMetrics {
     sendDataPacketTransferNanosQuantiles = new MutableQuantiles[len];
     ramDiskBlocksEvictionWindowMsQuantiles = new MutableQuantiles[len];
     ramDiskBlocksLazyPersistWindowMsQuantiles = new MutableQuantiles[len];
-    ecDecodingTimeNanos = registry.newCounter(
-        info("ecDecodingTimeNanos", "Nanoseconds spent by decoding tasks"),
-        (long) 0);
 
     for (int i = 0; i < len; i++) {
       int interval = intervals[i];
@@ -454,4 +454,12 @@ public class DataNodeMetrics {
   public void incrECDecodingTime(long decodingTimeNanos) {
     ecDecodingTimeNanos.incr(decodingTimeNanos);
   }
+
+  public void incrECReconstructionBytesRead(long bytes) {
+    ecReconstructionBytesRead.incr(bytes);
+  }
+
+  public void incrECReconstructionBytesWritten(long bytes) {
+    ecReconstructionBytesWritten.incr(bytes);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f14f6d0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
index 311ba7c..520d0e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
@@ -527,6 +527,30 @@ public class StripedFileTestUtil {
   }
 
   /**
+   * Wait for the reconstruction to be finished when the file has
+   * corrupted blocks. The function can take care file with any length.
+   */
+  public static void waitForAllReconstructionFinished(Path file,
+      DistributedFileSystem fs, long expectedBlocks) throws Exception {
+    LOG.info("Waiting for reconstruction to be finished for the file:" + file
+        + ", expectedBlocks:" + expectedBlocks);
+    final int attempts = 60;
+    for (int i = 0; i < attempts; i++) {
+      int totalBlocks = 0;
+      LocatedBlocks locatedBlocks = getLocatedBlocks(file, fs);
+      for (LocatedBlock locatedBlock: locatedBlocks.getLocatedBlocks()) {
+        DatanodeInfo[] storageInfos = locatedBlock.getLocations();
+        totalBlocks += storageInfos.length;
+      }
+      if (totalBlocks >= expectedBlocks) {
+        return;
+      }
+      Thread.sleep(1000);
+    }
+    throw new IOException("Time out waiting for EC block reconstruction.");
+  }
+
+  /**
    * Get the located blocks of a file.
    */
   public static LocatedBlocks getLocatedBlocks(Path file,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f14f6d0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java
index 1b0526b..64ddbd7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -28,7 +27,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@@ -38,21 +36,16 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
-import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
-import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-
 import java.io.IOException;
-import java.util.Arrays;
 
 /**
  * This file tests the erasure coding metrics in DataNode.
@@ -65,8 +58,9 @@ public class TestDataNodeErasureCodingMetrics {
   private final int dataBlocks = ecPolicy.getNumDataUnits();
   private final int parityBlocks = ecPolicy.getNumParityUnits();
   private final int cellSize = ecPolicy.getCellSize();
-  private final int blockSize = cellSize;
+  private final int blockSize = cellSize * 2;
   private final int groupSize = dataBlocks + parityBlocks;
+  private final int blockGroupSize = blockSize * dataBlocks;
   private final int numDNs = groupSize + 1;
 
   private MiniDFSCluster cluster;
@@ -76,7 +70,6 @@ public class TestDataNodeErasureCodingMetrics {
   @Before
   public void setup() throws IOException {
     conf = new Configuration();
-
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
@@ -93,82 +86,86 @@ public class TestDataNodeErasureCodingMetrics {
   }
 
   @Test(timeout = 120000)
-  public void testEcTasks() throws Exception {
-    DataNode workerDn = doTest("/testEcTasks");
-    MetricsRecordBuilder rb = getMetrics(workerDn.getMetrics().name());
-
-    // Ensure that reconstruction task is finished
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        long taskMetricValue = getLongCounter("EcReconstructionTasks", rb);
-        return (taskMetricValue > 0);
-      }
-    }, 500, 10000);
+  public void testFullBlock() throws Exception {
+    doTest("/testEcMetrics", blockGroupSize, 0);
+
+    Assert.assertEquals("EcReconstructionTasks should be ",
+        1, getLongMetric("EcReconstructionTasks"));
+    Assert.assertEquals("EcFailedReconstructionTasks should be ",
+        0, getLongMetric("EcFailedReconstructionTasks"));
+    Assert.assertTrue(getLongMetric("EcDecodingTimeNanos") > 0);
+    Assert.assertEquals("EcReconstructionBytesRead should be ",
+        blockGroupSize, getLongMetric("EcReconstructionBytesRead"));
+    Assert.assertEquals("EcReconstructionBytesWritten should be ",
+        blockSize, getLongMetric("EcReconstructionBytesWritten"));
+  }
 
-    assertCounter("EcReconstructionTasks", (long) 1, rb);
-    assertCounter("EcFailedReconstructionTasks", (long) 0, rb);
+  // A partial block, reconstruct the partial block
+  @Test(timeout = 120000)
+  public void testReconstructionBytesPartialGroup1() throws Exception {
+    final int fileLen = blockSize / 10;
+    doTest("/testEcBytes", fileLen, 0);
+
+    Assert.assertEquals("EcReconstructionBytesRead should be ",
+        fileLen,  getLongMetric("EcReconstructionBytesRead"));
+    Assert.assertEquals("EcReconstructionBytesWritten should be ",
+        fileLen, getLongMetric("EcReconstructionBytesWritten"));
   }
 
+  // 1 full block + 5 partial block, reconstruct the full block
   @Test(timeout = 120000)
-  public void testEcCodingTime() throws Exception {
-    DataNode workerDn = doTest("/testEcCodingTime");
-    MetricsRecordBuilder rb = getMetrics(workerDn.getMetrics().name());
-
-    // Ensure that reconstruction task is finished
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        long taskMetricValue = getLongCounter("EcReconstructionTasks", rb);
-        return (taskMetricValue > 0);
-      }
-    }, 500, 10000);
+  public void testReconstructionBytesPartialGroup2() throws Exception {
+    final int fileLen = cellSize * dataBlocks + cellSize + cellSize / 10;
+    doTest("/testEcBytes", fileLen, 0);
+
+    Assert.assertEquals("ecReconstructionBytesRead should be ",
+        cellSize * dataBlocks + cellSize + cellSize / 10,
+        getLongMetric("EcReconstructionBytesRead"));
+    Assert.assertEquals("ecReconstructionBytesWritten should be ",
+        blockSize, getLongMetric("EcReconstructionBytesWritten"));
+  }
 
-    long decodeTime = getLongCounter("ecDecodingTimeNanos", rb);
-    Assert.assertTrue(decodeTime > 0);
+  // 1 full block + 5 partial block, reconstruct the partial block
+  @Test(timeout = 120000)
+  public void testReconstructionBytesPartialGroup3() throws Exception {
+    final int fileLen = cellSize * dataBlocks + cellSize + cellSize / 10;
+    doTest("/testEcBytes", fileLen, 1);
+
+    Assert.assertEquals("ecReconstructionBytesRead should be ",
+        cellSize * dataBlocks + (cellSize / 10) * 2 ,
+        getLongMetric("EcReconstructionBytesRead"));
+    Assert.assertEquals("ecReconstructionBytesWritten should be ",
+        cellSize + cellSize / 10,
+        getLongMetric("EcReconstructionBytesWritten"));
   }
 
-  private DataNode doTest(String fileName) throws Exception {
+  private long getLongMetric(String metricName) {
+    long metricValue = 0;
+    // Add all reconstruction metric value from all data nodes
+    for (DataNode dn : cluster.getDataNodes()) {
+      MetricsRecordBuilder rb = getMetrics(dn.getMetrics().name());
+      metricValue += getLongCounter(metricName, rb);
+    }
+    return metricValue;
+  }
 
+  private void doTest(String fileName, int fileLen,
+      int deadNodeIndex) throws Exception {
+    assertTrue(fileLen > 0);
+    assertTrue(deadNodeIndex >= 0 && deadNodeIndex < numDNs);
     Path file = new Path(fileName);
-    long fileLen = dataBlocks * blockSize;
-    final byte[] data = StripedFileTestUtil.generateBytes((int) fileLen);
+    final byte[] data = StripedFileTestUtil.generateBytes(fileLen);
     DFSTestUtil.writeFile(fs, file, data);
     StripedFileTestUtil.waitBlockGroupsReported(fs, fileName);
 
-    LocatedBlocks locatedBlocks =
+    final LocatedBlocks locatedBlocks =
         StripedFileTestUtil.getLocatedBlocks(file, fs);
-    //only one block group
-    LocatedStripedBlock lastBlock =
+    final LocatedStripedBlock lastBlock =
         (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
-    DataNode workerDn = null;
-    DatanodeInfo[] locations = lastBlock.getLocations();
-    assertEquals(locations.length, groupSize);
-
-    // we have ONE extra datanode in addition to the GROUPSIZE datanodes, here
-    // is to find the extra datanode that the reconstruction task will run on,
-    // according to the current block placement logic for striped files.
-    // This can be improved later to be flexible regardless wherever the task
-    // runs.
-    for (DataNode dn : cluster.getDataNodes()) {
-      boolean appear = false;
-      for (DatanodeInfo info : locations) {
-        if (dn.getDatanodeUuid().equals(info.getDatanodeUuid())) {
-          appear = true;
-          break;
-        }
-      }
-      if (!appear) {
-        workerDn = dn;
-        break;
-      }
-    }
-    // Get a datanode from the block locations.
-    LOG.info("Block locations: " + Arrays.asList(locations));
-    LOG.info("Erasure coding worker datanode: " + workerDn);
-    assertNotNull("Failed to find a worker datanode", workerDn);
+    assertTrue(lastBlock.getLocations().length > deadNodeIndex);
 
-    DataNode toCorruptDn = cluster.getDataNode(locations[0].getIpcPort());
+    final DataNode toCorruptDn = cluster.getDataNode(
+        lastBlock.getLocations()[deadNodeIndex].getIpcPort());
     LOG.info("Datanode to be corrupted: " + toCorruptDn);
     assertNotNull("Failed to find a datanode to be corrupted", toCorruptDn);
     toCorruptDn.shutdown();
@@ -176,12 +173,15 @@ public class TestDataNodeErasureCodingMetrics {
     DFSTestUtil.waitForDatanodeState(cluster, toCorruptDn.getDatanodeUuid(),
         false, 10000);
 
-    int workCount = getComputedDatanodeWork();
+    final int workCount = getComputedDatanodeWork();
     assertTrue("Wrongly computed block reconstruction work", workCount > 0);
     cluster.triggerHeartbeats();
-    StripedFileTestUtil.waitForReconstructionFinished(file, fs, groupSize);
-
-    return workerDn;
+    int totalBlocks =  (fileLen / blockGroupSize) * groupSize;
+    final int remainder = fileLen % blockGroupSize;
+    totalBlocks += (remainder == 0) ? 0 :
+        (remainder % blockSize == 0) ? remainder / blockSize + parityBlocks :
+            remainder / blockSize + 1 + parityBlocks;
+    StripedFileTestUtil.waitForAllReconstructionFinished(file, fs, totalBlocks);
   }
 
   private int getComputedDatanodeWork()
@@ -209,5 +209,4 @@ public class TestDataNodeErasureCodingMetrics {
     BlockManagerTestUtil.checkHeartbeat(
         cluster.getNamesystem().getBlockManager());
   }
-
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/29] hadoop git commit: HDFS-11253. FileInputStream leak on failure path in BlockSender. Contributed by Arpit Agarwal

Posted by as...@apache.org.
HDFS-11253. FileInputStream leak on failure path in BlockSender. Contributed by Arpit Agarwal


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6fce1915
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6fce1915
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6fce1915

Branch: refs/heads/YARN-5085
Commit: 6fce19154906cb271b637b46896a34e7701a7bac
Parents: ac0d402
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Dec 15 15:39:15 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Dec 15 15:39:15 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java   | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fce1915/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index d7aebd8..99597dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -407,6 +407,8 @@ class BlockSender implements java.io.Closeable {
           blockIn, checksumIn, volumeRef, fileIoProvider);
     } catch (IOException ioe) {
       IOUtils.closeStream(this);
+      org.apache.commons.io.IOUtils.closeQuietly(blockIn);
+      org.apache.commons.io.IOUtils.closeQuietly(checksumIn);
       throw ioe;
     }
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/29] hadoop git commit: YARN-5646. Add documentation and update config parameter names for scheduling of OPPORTUNISTIC containers. (Konstantinos Karanasos via asuresh)

Posted by as...@apache.org.
YARN-5646. Add documentation and update config parameter names for scheduling of OPPORTUNISTIC containers. (Konstantinos Karanasos via asuresh)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2273a74c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2273a74c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2273a74c

Branch: refs/heads/YARN-5085
Commit: 2273a74c1f3895163046cca09ff5e983df301d22
Parents: cee0c46
Author: Arun Suresh <as...@apache.org>
Authored: Fri Dec 16 08:14:34 2016 -0800
Committer: Arun Suresh <as...@apache.org>
Committed: Fri Dec 16 08:14:34 2016 -0800

----------------------------------------------------------------------
 .../v2/app/rm/RMContainerAllocator.java         |   4 +-
 .../apache/hadoop/mapreduce/MRJobConfig.java    |   6 +-
 .../hadoop/mapred/TestMROpportunisticMaps.java  |   2 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  59 ++---
 .../api/impl/TestDistributedScheduling.java     |   1 -
 .../src/main/resources/yarn-default.xml         |  28 +--
 .../yarn/server/nodemanager/NodeManager.java    |   2 +-
 .../nodemanager/amrmproxy/AMRMProxyService.java |   2 +-
 .../containermanager/ContainerManagerImpl.java  |   4 +-
 .../scheduler/ContainerScheduler.java           |   2 +-
 ...pportunisticContainerAllocatorAMService.java |  16 +-
 .../yarn/server/resourcemanager/MockRM.java     |   2 +-
 .../hadoop/yarn/server/MiniYARNCluster.java     |   8 +-
 .../site/markdown/OpportunisticContainers.md    | 225 +++++++++++++++++++
 14 files changed, 296 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index 200fa1f..2b63b9e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -238,8 +238,8 @@ public class RMContainerAllocator extends RMContainerRequestor
     // first attempt to contact RM.
     retrystartTime = System.currentTimeMillis();
     this.scheduledRequests.setNumOpportunisticMapsPer100(
-        conf.getInt(MRJobConfig.MR_NUM_OPPORTUNISTIC_MAPS_PER_100,
-            MRJobConfig.DEFAULT_MR_NUM_OPPORTUNISTIC_MAPS_PER_100));
+        conf.getInt(MRJobConfig.MR_NUM_OPPORTUNISTIC_MAPS_PERCENTAGE,
+            MRJobConfig.DEFAULT_MR_NUM_OPPORTUNISTIC_MAPS_PERCENTAGE));
     LOG.info(this.scheduledRequests.getNumOpportunisticMapsPer100() +
         "% of the mappers will be scheduled using OPPORTUNISTIC containers");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 2ad86ef..32bbe96 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -1005,9 +1005,9 @@ public interface MRJobConfig {
    * requested by the AM will be opportunistic. If the total number of maps
    * for the job is less than 'x', then ALL maps will be OPPORTUNISTIC
    */
-  public static final String MR_NUM_OPPORTUNISTIC_MAPS_PER_100 =
-      "mapreduce.job.num-opportunistic-maps-per-100";
-  public static final int DEFAULT_MR_NUM_OPPORTUNISTIC_MAPS_PER_100 = 0;
+  public static final String MR_NUM_OPPORTUNISTIC_MAPS_PERCENTAGE =
+      "mapreduce.job.num-opportunistic-maps-percentage";
+  public static final int DEFAULT_MR_NUM_OPPORTUNISTIC_MAPS_PERCENTAGE = 0;
 
   /**
    * A comma-separated list of properties whose value will be redacted.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMROpportunisticMaps.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMROpportunisticMaps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMROpportunisticMaps.java
index d975fd0..462ff04 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMROpportunisticMaps.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMROpportunisticMaps.java
@@ -145,7 +145,7 @@ public class TestMROpportunisticMaps {
     job.setNumReduceTasks(numReducers);
 
     // All OPPORTUNISTIC
-    job.setInt(MRJobConfig.MR_NUM_OPPORTUNISTIC_MAPS_PER_100, percent);
+    job.setInt(MRJobConfig.MR_NUM_OPPORTUNISTIC_MAPS_PERCENTAGE, percent);
     job.setInt("mapreduce.map.maxattempts", 1);
     job.setInt("mapreduce.reduce.maxattempts", 1);
     job.setInt("mapred.test.num_lines", numLines);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index dc7c629..5adbfa7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -312,69 +312,65 @@ public class YarnConfiguration extends Configuration {
   /** ACL used in case none is found. Allows nothing. */
   public static final String DEFAULT_YARN_APP_ACL = " ";
 
-  /** Setting that controls whether distributed scheduling is enabled or not. */
-  public static final String DIST_SCHEDULING_ENABLED =
-      YARN_PREFIX + "distributed-scheduling.enabled";
-  public static final boolean DIST_SCHEDULING_ENABLED_DEFAULT = false;
-
   /** Setting that controls whether opportunistic container allocation
    *  is enabled or not. */
   public static final String OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED =
-      YARN_PREFIX + "opportunistic-container-allocation.enabled";
+      RM_PREFIX + "opportunistic-container-allocation.enabled";
   public static final boolean
-      OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED_DEFAULT = false;
+      DEFAULT_OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED = false;
 
   /** Number of nodes to be used by the Opportunistic Container allocator for
    * dispatching containers during container allocation. */
   public static final String OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED =
-      YARN_PREFIX + "opportunistic-container-allocation.nodes-used";
-  public static final int OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED_DEFAULT =
+      RM_PREFIX + "opportunistic-container-allocation.nodes-used";
+  public static final int DEFAULT_OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED =
       10;
 
   /** Frequency for computing least loaded NMs. */
   public static final String NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS =
-      YARN_PREFIX + "nm-container-queuing.sorting-nodes-interval-ms";
+      RM_PREFIX + "nm-container-queuing.sorting-nodes-interval-ms";
   public static final long
-      NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS_DEFAULT = 1000;
+      DEFAULT_NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS = 1000;
 
-  /** Comparator for determining node load for Distributed Scheduling. */
+  /** Comparator for determining node load for scheduling of opportunistic
+   * containers. */
   public static final String NM_CONTAINER_QUEUING_LOAD_COMPARATOR =
-      YARN_PREFIX + "nm-container-queuing.load-comparator";
-  public static final String NM_CONTAINER_QUEUING_LOAD_COMPARATOR_DEFAULT =
+      RM_PREFIX + "nm-container-queuing.load-comparator";
+  public static final String DEFAULT_NM_CONTAINER_QUEUING_LOAD_COMPARATOR =
       "QUEUE_LENGTH";
 
   /** Value of standard deviation used for calculation of queue limit
    * thresholds. */
   public static final String NM_CONTAINER_QUEUING_LIMIT_STDEV =
-      YARN_PREFIX + "nm-container-queuing.queue-limit-stdev";
-  public static final float NM_CONTAINER_QUEUING_LIMIT_STDEV_DEFAULT =
+      RM_PREFIX + "nm-container-queuing.queue-limit-stdev";
+  public static final float DEFAULT_NM_CONTAINER_QUEUING_LIMIT_STDEV =
       1.0f;
 
   /** Min length of container queue at NodeManager. This is a cluster-wide
    * configuration that acts as the lower-bound of optimal queue length
    * calculated by the NodeQueueLoadMonitor */
   public static final String NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH =
-      YARN_PREFIX + "nm-container-queuing.min-queue-length";
-  public static final int NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH_DEFAULT = 1;
+      RM_PREFIX + "nm-container-queuing.min-queue-length";
+  public static final int DEFAULT_NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH = 5;
 
   /** Max length of container queue at NodeManager. This is a cluster-wide
    * configuration that acts as the upper-bound of optimal queue length
    * calculated by the NodeQueueLoadMonitor */
   public static final String NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH =
-      YARN_PREFIX + "nm-container-queuing.max-queue-length";
-  public static final int NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH_DEFAULT = 10;
+      RM_PREFIX + "nm-container-queuing.max-queue-length";
+  public static final int DEFAULT_NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH = 15;
 
   /** Min queue wait time for a container at a NodeManager. */
   public static final String NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS =
-      YARN_PREFIX + "nm-container-queuing.min-queue-wait-time-ms";
-  public static final int NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS_DEFAULT =
-      1;
+      RM_PREFIX + "nm-container-queuing.min-queue-wait-time-ms";
+  public static final int DEFAULT_NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS =
+      10;
 
   /** Max queue wait time for a container queue at a NodeManager. */
   public static final String NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS =
-      YARN_PREFIX + "nm-container-queuing.max-queue-wait-time-ms";
-  public static final int NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS_DEFAULT =
-      10;
+      RM_PREFIX + "nm-container-queuing.max-queue-wait-time-ms";
+  public static final int DEFAULT_NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS =
+      100;
 
   /**
    * Enable/disable intermediate-data encryption at YARN level. For now, this
@@ -812,9 +808,14 @@ public class YarnConfiguration extends Configuration {
   /** Max Queue length of <code>OPPORTUNISTIC</code> containers on the NM. */
   public static final String NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH =
       NM_PREFIX + "opportunistic-containers-max-queue-length";
-  public static final int NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT =
+  public static final int DEFAULT_NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH =
       0;
 
+  /** Setting that controls whether distributed scheduling is enabled or not. */
+  public static final String DIST_SCHEDULING_ENABLED =
+      NM_PREFIX + "distributed-scheduling.enabled";
+  public static final boolean DEFAULT_DIST_SCHEDULING_ENABLED = false;
+
   /** Environment variables that will be sent to containers.*/
   public static final String NM_ADMIN_USER_ENV = NM_PREFIX + "admin-env";
   public static final String DEFAULT_NM_ADMIN_USER_ENV = "MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX";
@@ -2844,14 +2845,14 @@ public class YarnConfiguration extends Configuration {
 
   public static boolean isDistSchedulingEnabled(Configuration conf) {
     return conf.getBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED,
-        YarnConfiguration.DIST_SCHEDULING_ENABLED_DEFAULT);
+        YarnConfiguration.DEFAULT_DIST_SCHEDULING_ENABLED);
   }
 
   public static boolean isOpportunisticContainerAllocationEnabled(
       Configuration conf) {
     return conf.getBoolean(
         YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED,
-        YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED_DEFAULT);
+        YarnConfiguration.DEFAULT_OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED);
   }
 
   // helper methods for timeline service configuration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
index b552d19..e180f6d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
@@ -104,7 +104,6 @@ public class TestDistributedScheduling extends BaseAMRMProxyE2ETest {
     cluster = new MiniYARNCluster("testDistributedSchedulingE2E", 1, 1, 1);
 
     conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true);
     conf.setBoolean(YarnConfiguration.
         OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
     conf.setBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index c8c4edd..3e739ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2751,7 +2751,7 @@
     <description>
     Setting that controls whether distributed scheduling is enabled.
     </description>
-    <name>yarn.distributed-scheduling.enabled</name>
+    <name>yarn.nodemanager.distributed-scheduling.enabled</name>
     <value>false</value>
   </property>
 
@@ -2760,7 +2760,7 @@
       Setting that controls whether opportunistic container allocation
       is enabled.
     </description>
-    <name>yarn.opportunistic-container-allocation.enabled</name>
+    <name>yarn.resourcemanager.opportunistic-container-allocation.enabled</name>
     <value>false</value>
   </property>
 
@@ -2769,7 +2769,7 @@
     Number of nodes to be used by the Opportunistic Container Allocator for
     dispatching containers during container allocation.
     </description>
-    <name>yarn.opportunistic-container-allocation.nodes-used</name>
+    <name>yarn.resourcemanager.opportunistic-container-allocation.nodes-used</name>
     <value>10</value>
   </property>
 
@@ -2777,7 +2777,7 @@
     <description>
     Frequency for computing least loaded NMs.
     </description>
-    <name>yarn.nm-container-queuing.sorting-nodes-interval-ms</name>
+    <name>yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms</name>
     <value>1000</value>
   </property>
 
@@ -2785,7 +2785,7 @@
     <description>
     Comparator for determining node load for Distributed Scheduling.
     </description>
-    <name>yarn.nm-container-queuing.load-comparator</name>
+    <name>yarn.resourcemanager.nm-container-queuing.load-comparator</name>
     <value>QUEUE_LENGTH</value>
   </property>
 
@@ -2793,7 +2793,7 @@
     <description>
     Value of standard deviation used for calculation of queue limit thresholds.
     </description>
-    <name>yarn.nm-container-queuing.queue-limit-stdev</name>
+    <name>yarn.resourcemanager.nm-container-queuing.queue-limit-stdev</name>
     <value>1.0f</value>
   </property>
 
@@ -2801,32 +2801,32 @@
     <description>
     Min length of container queue at NodeManager.
     </description>
-    <name>yarn.nm-container-queuing.min-queue-length</name>
-    <value>1</value>
+    <name>yarn.resourcemanager.nm-container-queuing.min-queue-length</name>
+    <value>5</value>
   </property>
 
   <property>
     <description>
     Max length of container queue at NodeManager.
     </description>
-    <name>yarn.nm-container-queuing.max-queue-length</name>
-    <value>10</value>
+    <name>yarn.resourcemanager.nm-container-queuing.max-queue-length</name>
+    <value>15</value>
   </property>
 
   <property>
     <description>
     Min queue wait time for a container at a NodeManager.
     </description>
-    <name>yarn.nm-container-queuing.min-queue-wait-time-ms</name>
-    <value>1</value>
+    <name>yarn.resourcemanager.nm-container-queuing.min-queue-wait-time-ms</name>
+    <value>10</value>
   </property>
 
   <property>
     <description>
     Max queue wait time for a container queue at a NodeManager.
     </description>
-    <name>yarn.nm-container-queuing.max-queue-wait-time-ms</name>
-    <value>10</value>
+    <name>yarn.resourcemanager.nm-container-queuing.max-queue-wait-time-ms</name>
+    <value>100</value>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 72875a9..7de5317 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -329,7 +329,7 @@ public class NodeManager extends CompositeService
 
     boolean isDistSchedulingEnabled =
         conf.getBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED,
-            YarnConfiguration.DIST_SCHEDULING_ENABLED_DEFAULT);
+            YarnConfiguration.DEFAULT_DIST_SCHEDULING_ENABLED);
 
     this.context = createNMContext(containerTokenSecretManager,
         nmTokenSecretManager, nmStore, isDistSchedulingEnabled, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
index 79882aa..dc56090 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
@@ -79,7 +79,7 @@ import com.google.common.base.Preconditions;
  * to intercept and inspect messages from application master to the cluster
  * resource manager. It listens to messages from the application master and
  * creates a request intercepting pipeline instance for each application. The
- * pipeline is a chain of intercepter instances that can inspect and modify the
+ * pipeline is a chain of interceptor instances that can inspect and modify the
  * request/response as needed.
  */
 public class AMRMProxyService extends AbstractService implements

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index e4d9346..8f4478e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -305,7 +305,9 @@ public class ContainerManagerImpl extends CompositeService implements
   protected void createAMRMProxyService(Configuration conf) {
     this.amrmProxyEnabled =
         conf.getBoolean(YarnConfiguration.AMRM_PROXY_ENABLED,
-            YarnConfiguration.DEFAULT_AMRM_PROXY_ENABLED);
+            YarnConfiguration.DEFAULT_AMRM_PROXY_ENABLED) ||
+            conf.getBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED,
+                YarnConfiguration.DEFAULT_DIST_SCHEDULING_ENABLED);
 
     if (amrmProxyEnabled) {
       LOG.info("AMRMProxyService is enabled. "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
index 753bf04..24530b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
@@ -106,7 +106,7 @@ public class ContainerScheduler extends AbstractService implements
     this(context, dispatcher, metrics, context.getConf().getInt(
         YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH,
         YarnConfiguration.
-            NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT));
+            DEFAULT_NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH));
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
index 7b7963f..a527d04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
@@ -112,11 +112,11 @@ public class OpportunisticContainerAllocatorAMService
         rmContext.getContainerTokenSecretManager());
     this.k = rmContext.getYarnConfiguration().getInt(
         YarnConfiguration.OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED,
-        YarnConfiguration.OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED_DEFAULT);
+        YarnConfiguration.DEFAULT_OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED);
     long nodeSortInterval = rmContext.getYarnConfiguration().getLong(
         YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS,
         YarnConfiguration.
-            NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS_DEFAULT);
+            DEFAULT_NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS);
     this.cacheRefreshInterval = nodeSortInterval;
     this.lastCacheUpdateTime = System.currentTimeMillis();
     NodeQueueLoadMonitor.LoadComparator comparator =
@@ -124,14 +124,14 @@ public class OpportunisticContainerAllocatorAMService
             rmContext.getYarnConfiguration().get(
                 YarnConfiguration.NM_CONTAINER_QUEUING_LOAD_COMPARATOR,
                 YarnConfiguration.
-                    NM_CONTAINER_QUEUING_LOAD_COMPARATOR_DEFAULT));
+                    DEFAULT_NM_CONTAINER_QUEUING_LOAD_COMPARATOR));
 
     NodeQueueLoadMonitor topKSelector =
         new NodeQueueLoadMonitor(nodeSortInterval, comparator);
 
     float sigma = rmContext.getYarnConfiguration()
         .getFloat(YarnConfiguration.NM_CONTAINER_QUEUING_LIMIT_STDEV,
-            YarnConfiguration.NM_CONTAINER_QUEUING_LIMIT_STDEV_DEFAULT);
+            YarnConfiguration.DEFAULT_NM_CONTAINER_QUEUING_LIMIT_STDEV);
 
     int limitMin, limitMax;
 
@@ -139,22 +139,22 @@ public class OpportunisticContainerAllocatorAMService
       limitMin = rmContext.getYarnConfiguration()
           .getInt(YarnConfiguration.NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH,
               YarnConfiguration.
-                  NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH_DEFAULT);
+                  DEFAULT_NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH);
       limitMax = rmContext.getYarnConfiguration()
           .getInt(YarnConfiguration.NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH,
               YarnConfiguration.
-                  NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH_DEFAULT);
+                  DEFAULT_NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH);
     } else {
       limitMin = rmContext.getYarnConfiguration()
           .getInt(
               YarnConfiguration.NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS,
               YarnConfiguration.
-                  NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS_DEFAULT);
+                  DEFAULT_NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS);
       limitMax = rmContext.getYarnConfiguration()
           .getInt(
               YarnConfiguration.NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS,
               YarnConfiguration.
-                  NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS_DEFAULT);
+                  DEFAULT_NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS);
     }
 
     topKSelector.initThresholdCalculator(sigma, limitMin, limitMax);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index a66b093..02d3956 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -947,7 +947,7 @@ public class MockRM extends ResourceManager {
   protected ApplicationMasterService createApplicationMasterService() {
     if (this.rmContext.getYarnConfiguration().getBoolean(
         YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED,
-        YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED_DEFAULT)) {
+        YarnConfiguration.DEFAULT_OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED)) {
       return new OpportunisticContainerAllocatorAMService(getRMContext(),
           scheduler) {
         @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index 4f484ad..5ae74a6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -852,7 +852,9 @@ public class MiniYARNCluster extends CompositeService {
     protected void createAMRMProxyService(Configuration conf) {
       this.amrmProxyEnabled =
           conf.getBoolean(YarnConfiguration.AMRM_PROXY_ENABLED,
-              YarnConfiguration.DEFAULT_AMRM_PROXY_ENABLED);
+              YarnConfiguration.DEFAULT_AMRM_PROXY_ENABLED) ||
+              conf.getBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED,
+                  YarnConfiguration.DEFAULT_DIST_SCHEDULING_ENABLED);
 
       if (this.amrmProxyEnabled) {
         LOG.info("CustomAMRMProxyService is enabled. "
@@ -882,7 +884,9 @@ public class MiniYARNCluster extends CompositeService {
     protected void createAMRMProxyService(Configuration conf) {
       this.amrmProxyEnabled =
           conf.getBoolean(YarnConfiguration.AMRM_PROXY_ENABLED,
-              YarnConfiguration.DEFAULT_AMRM_PROXY_ENABLED);
+              YarnConfiguration.DEFAULT_AMRM_PROXY_ENABLED) ||
+              conf.getBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED,
+                  YarnConfiguration.DEFAULT_DIST_SCHEDULING_ENABLED);
 
       if (this.amrmProxyEnabled) {
         LOG.info("CustomAMRMProxyService is enabled. "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2273a74c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
new file mode 100644
index 0000000..ac26d88
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
@@ -0,0 +1,225 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+Opportunistic Containers
+========================
+
+* [Purpose](#Purpose)
+* [Quick Guide](#Quick_Guide)
+    * [Main Goal](#Main_Goal)
+    * [Enabling Opportunistic Containers](#Enabling_Opportunistic_Containers)
+    * [Running a Sample Job](Running_a_Sample_Job)
+    * [Opportunistic Containers in Web UI](Opportunistic_Containers_in_Web_UI)
+* [Overview](#Overview)
+* [Container Execution Types](#Container_Execution_Types)
+* [Execution of Opportunistic Containers](#Execution_of_Opportunistic_Containers)
+* [Allocation of Opportunistic Containers](#Allocation_of_Opportunistic_Containers)
+    * [Centralized Allocation](#Centralized_Allocation)
+    * [Distributed Allocation](#Distributed_Allocation)
+    * [Determining Nodes for Allocation](#Determining_Nodes_for_Allocation)
+    * [Rebalancing Node Load](#Rebalancing_Node_Load)
+* [Advanced Configuration](#Advanced_Configuration)
+* [Items for Future Work](#Items_for_Future_Work)
+
+
+<a name="Purpose"></a>Purpose
+-----------------------------
+
+This document introduces the notion of **opportunistic** container execution, and discusses how opportunistic containers are allocated and executed.
+
+
+<a name="Quick_Guide"></a>Quick Guide
+--------------------------------------------------------------------
+
+We start by providing a brief overview of opportunistic containers, including how a user can enable this feature and run a sample job using such containers.
+
+###<a name="Main_Goal"></a>Main Goal
+
+Unlike existing YARN containers that are scheduled in a node only if there are unallocated resources, opportunistic containers can be dispatched to an NM, even if their execution at that node cannot start immediately. In such a case, opportunistic containers will be queued at that NM until resources become available. 
+The main goal of opportunistic container execution is to improve cluster resource utilization, and therefore increase task throughput. Resource utilization and task throughput improvements are more pronounced for workloads that include relatively short tasks (in the order of seconds).
+
+
+###<a name="Enabling_Opportunistic_Containers"></a>Enabling Opportunistic Containers
+
+To enable opportunistic container allocation, the following two properties have to be present in **conf/yarn-site.xml**:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanager.opportunistic-container-allocation.enabled` | Enables opportunistic container allocation. | `false` |
+| `yarn.nodemanager.opportunistic-containers-max-queue-length` | Determines the max number of opportunistic containers that can be queued at an NM. | `0` |
+
+The first parameter above has to be set to `true`. The second one has to be set to a positive value to allow queuing of opportunistic containers at the NM. A value of `10` can be used to start experimenting with opportunistic containers. The optimal value depends on the jobs characteristics, the cluster configuration and the target utilization.
+
+By default, allocation of opportunistic containers is performed centrally through the RM. However, a user can choose to enable distributed allocation of opportunistic containers, which can further improve allocation latency for short tasks. Distributed scheduling can be enabling by setting to `true` the following parameter (note that non-opportunistic containers will continue being scheduled through the RM):
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.nodemanager.distributed-scheduling.enabled` | Enables distributed scheduling. | `false` |
+
+
+###<a name="Running_a_Sample_Job"></a>Running a Sample Job
+
+The following command can be used to run a sample pi map-reduce job, executing 40% of mappers using opportunistic containers (substitute `3.0.0-alpha2-SNAPSHOT` below with the version of Hadoop you are using):
+```
+$ hadoop jar hadoop-3.0.0-alpha2-SNAPSHOT/share/hadoop/mapreduce/hadoop-mapreduce-examples-3.0.0-alpha2-SNAPSHOT.jar pi -Dmapreduce.job.num-opportunistic-maps-percentage="40" 50 100
+```
+
+By changing the value of `mapreduce.job.num-opportunistic-maps-percentage` in the above command, we can specify the percentage of mappers that can be executed through opportunistic containers.
+
+
+###<a name="Opportunistic_Containers_in_Web_UI"></a>Opportunistic Containers in Web UI
+
+When opportunistic container allocation is enabled, the following new columns can be observed in the Nodes page of the Web UI (`rm-address:8088/cluster/nodes`):
+* Running Containers (O): number of running opportunistic containers on each node;
+* Mem Used (O): Total memory used by opportunistic containers on each node;
+* VCores Used (O): Total CPU virtual cores used by opportunistic containers on each node;
+* Queued Containers: Number of containers queued at each node.
+
+When clicking on a specific container running on a node, the execution type of the container is also shown.
+
+In the rest of the document, we provide an in-depth description of opportunistic containers, including details about their allocation and execution.
+
+
+Overview <a name="Overview"></a>
+--------------------------------
+
+The existing schedulers in YARN (Fair and Capacity Scheduler) allocate containers to a node only if there are unallocated resources at that node at the moment of scheduling the containers. This **guaranteed** type of execution has the advantage that once the AM dispatches a container to a node, the container execution will start immediately, since it is guaranteed that there will be available resources. Moreover, unless fairness or capacity constraints are violated, containers are guaranteed to run to completion without being preempted. 
+
+Although this design offers a more predictable task execution, it has two main drawbacks that can lead to suboptimal cluster resource utilization:
+
+* **Feedback delays.** When a container finishes its execution at a node, the RM gets notified that there are available resources through the next NM-RM heartbeat, then the RM schedules a new container at that node, the AM gets notified through the next AM-RM heartbeat, and finally the AM launches the new container at the node. These delays result in idle node resources, which in turn lead to lower resource utilization, especially when workloads involve tasks whose duration is relatively short.
+* **Allocated vs. utilized resources.** The RM allocates containers based on the *allocated* resources at each node, which might be significantly higher than the actually *utilized* resources (e.g., think of a container for which 4GB memory have been allocated, but only 2GB are being utilized). This lowers effective resource utilization, and can be avoided if the RM takes into account the utilized resources during scheduling. However, this has to be done in a way that allows resources to be reclaimed in case the utilized resources of a running container increase.
+
+To mitigate the above problems, in addition to the existing containers (which we term **guaranteed** containers hereafter), we introduce the notion of **opportunistic** containers. An opportunistic container can be dispatched to an NM, even if there are no available (unallocated) resources for it at the moment of scheduling. In such a case, the opportunistic container will be queued at the NM, waiting for resources to become available for its execution to start. The opportunistic containers are of lower priority than the guaranteed ones, which means that they can be preempted for guaranteed containers to start their execution. Therefore, they can be used to improve cluster resource utilization without impacting the execution of existing guaranteed containers.
+
+An additional advantage of opportunistic containers is that they introduce a notion of **execution priority at the NMs**. For instance, a lower priority job that does not require strict execution guarantees can use opportunistic containers or a mix of container execution types for its tasks.
+
+We have introduced two ways of allocating opportunistic containers: a **centralized** and a **distributed** one. In the centralized scheduling, opportunistic containers are allocated through the YARN RM, whereas in the distributed one, through local schedulers that reside at each NM. Centralized allocation allows for higher quality placement decisions and for implementing more involved sharing policies across applications (e.g., fairness). On the other hand, distributed scheduling can offer faster container allocation, which is useful for short tasks, as it avoids the round-trip to the RM. In both cases, the scheduling of guaranteed containers remains intact and happens through the YARN RM (using the existing Fair or Capacity Scheduler).
+
+Note that in the current implementation, we are allocating containers based on allocated (and not utilized) resources. Therefore, we tackle the "feedback delays" problem mentioned above, but not the "allocated vs. utilized resources" one. There is ongoing work (`YARN-1011`) that employs opportunistic containers to address the latter problem too.
+
+Below, we describe in more detail the [container execution types](#Container_Execution_Types), as well as the [execution](#Execution_of_Opportunistic_Containers) (including the container queuing at the NMs) and [allocation](#Allocation_of_Opportunistic_Containers) of opportunistic containers. Then we discuss how to fine-tune opportunistic containers through some [advanced configuration parameters](#Advanced_Configuration). Finally, we discuss open items for [future work](#Items_for_Future_Work).
+
+
+<a name="Container_Execution_Types"></a>Container Execution Types
+-----------------------------------------------------------------
+
+We introduce the following two types of containers:
+
+* **Guaranteed containers** correspond to the existing YARN containers. They are allocated by the Fair or Capacity Scheduler, and once dispatched to a node, it is guaranteed that there are available resources for their execution to start immediately. Moreover, these containers run to completion (as long as there are no failures). They can be preempted only in case the scheduler's queue to which they belong, violates fairness or capacity constraints.
+* **Opportunistic containers** are not guaranteed to have resources for their execution to start when they get dispatched to a node. Instead, they might be queued at the NM until resources become available. In case a guaranteed container arrives at a node and there are no resources available for it, one or more opportunistic containers will be preempted to execute the guaranteed one.
+
+When an AM submits its resource requests to the RM, it specifies the type for each container (default is guaranteed), determining the way the container will be [allocated](#Allocation_of_Opportunistic_Containers). Subsequently, when the container is launched by the AM at an NM, its type determines how it will be [executed](#Execution_of_Opportunistic_Containers) by the NM.
+
+
+<a name="Execution_of_Opportunistic_Containers"></a>Execution of Opportunistic Containers
+---------------------------------------------------------------------------
+
+When a container arrives at an NM, its execution is determined by the available resources at the NM and the container type. Guaranteed containers start their execution immediately, and if needed, the NM will kill running opportunistic containers to ensure there are sufficient resources for the guaranteed ones to start. On the other hand, opportunistic containers can be queued at the NM, if there are no resources available to start their execution when they arrive at the NM. To enable this, we extended the NM by allowing queuing of containers at each node. The NM monitors the local resources, and when there are sufficient resources available, it starts the execution of the opportunistic container that is at the head of the queue.
+
+In particular, when a container arrives at an NM, localization is performed (i.e., all required resources are downloaded), and then the container moves to a `SCHEDULED` state, in which the container is queued, waiting for its execution to begin:
+
+* If there are available resources, the execution of the container starts immediately, irrespective of its execution type.
+* If there are no available resources:
+    * If the container is guaranteed, we kill as many running opportunistic containers as required for the guaranteed container to be executed, and then start its execution.
+    * If the container is opportunistic, it remains at the queue until resources become available.
+* When a container (guaranteed or opportunistic) finishes its execution and resources get freed up, we examine the queued containers and if there are available resources we start their execution. We pick containers from the queue in a FIFO order.
+
+In the [future work items](#Items_for_Future_Work) below, we discuss different ways of prioritizing task execution (queue reordering) and of killing opportunistic containers to make space for guaranteed ones.
+
+
+<a name="Allocation_of_Opportunistic_Containers"></a>Allocation of Opportunistic Containers
+-----------------------------------------------------------------------------
+
+As mentioned above, we provide both a centralized and a distributed way of allocating opportunistic containers, which we describe below.
+
+###<a name="Centralized_Allocation"></a>Centralized Allocation
+
+We have introduced a new service at the RM, namely the `OpportunisticContainerAllocatorAMService`, which extends the `ApplicationMasterService`. When the centralized opportunistic allocation is enabled, the resource requests from the AMs are served at the RM side by the `OpportunisticContainerAllocatorAMService`, which splits them into two sets of resource requests: 
+
+* The guaranteed set is forwarded to the existing `ApplicationMasterService` and is subsequently handled by the Fair or Capacity Scheduler.
+* The opportunistic set is handled by the new `OpportunisticContainerAllocator`, which performs the scheduling of opportunistic containers to nodes.
+
+The `OpportunisticContainerAllocator` maintains a list with the [least loaded nodes](#Determining_Nodes_for_Allocation) of the cluster at each moment, and assigns containers to them in a round-robin fashion. Note that in the current implementation, we purposely do not take into account node locality constraints. Since an opportunistic container (unlike the guaranteed ones) might wait at the queue of an NM before its execution starts, it is more important to allocate it at a node that is less loaded (i.e., where queuing delay will be smaller) rather than respect its locality constraints. Moreover, we do not take into account sharing (fairness/capacity) constraints for opportunistic containers at the moment. Support for both locality and sharing constraints can be added in the future if required.
+
+
+###<a name="Distributed_Allocation"></a>Distributed Allocation
+
+In order to enable distributed scheduling of opportunistic containers, we have introduced a new service at each NM, called `AMRMProxyService`. The `AMRMProxyService` implements the `ApplicationMasterService` protocol, and acts as a proxy between the AMs running at that node and the RM. When the `AMRMProxyService` is enabled (through a parameter), we force all AMs running at a particular node to communicate with the `AMRMProxyService` of the same node, instead of going directly to the RM. Moreover, to ensure that the AMs will not talk directly with the RM, when a new AM gets initialized, we replace its `AMRMToken` with a token signed by the `AMRMProxyService`.
+
+A chain of interceptors can be registered with the `AMRMProxyService`. One of these interceptors is the `DistributedScheduler` that is responsible for allocating opportunistic containers in a distributed way, without needing to contact the RM. This modular design makes the `AMRMProxyService` instrumental in other scenarios too, such as YARN federation (`YARN-2915`) or throttling down misbehaving AMs, which can be enabled simply by adding additional interceptors at the interceptor chain.
+
+When distributed opportunistic scheduling is enabled, each AM sends its resource requests to the `AMRMProxyService` running at the same node. The `AMRMProxyService` splits the resource requests into two sets:
+
+* The guaranteed set is forwarded to the RM. In this case the `AMRMProxyService` simply acts as a proxy between the AM and the RM, and the container allocation remains intact (using the Fair or Capacity Scheduler).
+* The opportunistic set is not forwarded to the RM. Instead, it is handled by the `DistributedScheduler` that is running locally at the node. In particular, the `DistributedScheduler` maintains a list with the least loaded nodes in the cluster, and allocates containers to them in a round-robin fashion. The RM informs the `DistributedScheduler` about the least loaded nodes at regular intervals through the NM-RM heartbeats.
+
+The above procedure is similar to the one performed by the `OpportunisticContainerAllocatorAMService` in the case of centralized opportunistic scheduling described above. The main difference is that in the distributed case, the splitting of requests into guaranteed and opportunistic happens locally at the node, and only the guaranteed requests are forwarded to the RM, while the opportunistic ones are handled without contacting the RM.
+
+
+###<a name="Determining_Nodes_for_Allocation"></a>Determining Nodes for Allocation
+
+Each NM informs the RM periodically through the NM-RM heartbeats about the number of running guaranteed and opportunistic containers, as well as the number of queued opportunistic containers. The RM gathers this information from all nodes and determines the least loaded ones.
+
+In the case of centralized allocation of opportunistic containers, this information is immediately available, since the allocation happens centrally. In the case of distributed scheduling, the list with the least loaded nodes is propagated to all NMs (and thus becomes available to the `DistributedSchedulers`) through the heartbeat responses from the RM to the NMs. The number of least loaded nodes sent to the NMs is configurable.
+
+At the moment, we take into account only the number of queued opportunistic containers at each node in order to estimate the time an opportunistic container would have to wait if sent to that node and, thus, determine the least loaded nodes. If the AM provided us with information about the estimated task durations, we could take them into account in order to have better estimates of the queue waiting times.
+
+
+###<a name="Rebalancing_Node_Load"></a>Rebalancing Node Load
+
+Occasionally poor placement choices for opportunistic containers may be made (due to stale queue length estimates), which can lead to load imbalance between nodes. The problem is more pronounced under high cluster load, and also in the case of distributed scheduling (multiple `DistributedSchedulers` may place containers at the same NM, since they do not coordinate with each other). To deal with this load imbalance between the NM queues, we perform load shedding to dynamically re-balance the load between NMs. In particular, while aggregating at the RM the queue time estimates published by each NM, we construct a distribution and find a targeted maximal value for the length of the NM queues (based on the mean and standard deviation of the distribution). Then the RM disseminates this value to the various NMs through the heartbeat responses. Subsequently, using this information, an NM on a node whose queue length is above the threshold discards opportunistic containers to meet this maxi
 mal value. This forces the associated individual AMs to reschedule those containers elsewhere.
+
+
+<a name="Advanced_Configuration"></a>Advanced Configuration
+--------------------------------------------------
+
+The main properties for enabling opportunistic container allocation and choosing between centralized and distributed allocation were described in the [quick guide](#Quick_Guide) in the beginning of this document. Here we present more advanced configuration. Note that using default values for those parameters should be sufficient in most cases. All parameters below have to be defined in the **conf/yarn-site.xml** file.
+
+To determine the number of [least loaded nodes](#Determining_Nodes_for_Allocation) that will be used when scheduling opportunistic containers and how often this list will be refreshed, we use the following parameters:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanager.opportunistic-container-allocation.nodes-used` | Number of least loaded nodes to be used by the Opportunistic Container allocator for dispatching containers during container allocation. A higher value can improve load balance in large clusters. | `10` |
+| `yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms` | Frequency for computing least loaded nodes. | `1000` |
+
+
+As discussed in the [node load rebalancing](#Rebalancing_Node_Load) section above, at regular intervals, the RM gathers all NM queue lengths and computes their mean value (`avg`) and standard deviation (`stdev`), as well as the value `avg + k*stdev` (where `k` a float). This value gets propagated through the NM-RM heartbeats to all NMs, who should respect that value by dequeuing containers (if required), as long as their current queue length is between a `queue_min_length` and a `queue_max_length` value (these values are used to avoid dequeuing tasks from very short queues and to aggressively dequeue tasks from long queues, respectively). 
+The parameters `k`, `queue_min_length` and `queue_max_length` can be specified as follows:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanager.nm-container-queuing.queue-limit-stdev` | The `k` parameter. | `1.0f` |
+| `yarn.resourcemanager.nm-container-queuing.min-queue-length` | The `queue_min_length` parameter. | `5` |
+| `yarn.resourcemanager.nm-container-queuing.max-queue-length` | The `queue_max_length` parameter. | `15` |
+
+
+Finally, two more properties can further tune the `AMRMProxyService` in case distributed scheduling is used:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.nodemanager.amrmproxy.address` | The address/port to which the `AMRMProxyService` is bound to. | `0.0.0.0:8049` |
+| `yarn.nodemanager.amrmproxy.client.thread-count` | The number of threads that are used at each NM for serving the interceptors register to the `AMRMProxyService` by different jobs. | `3` |
+
+
+<a name="Items_for_Future_Work"></a>Items for Future Work
+-----------------------------------------------
+
+Here we describe multiple ways in which we can extend/enhance the allocation and execution of opportunistic containers. We also provide the JIRAs that track each item.
+
+* **Resource overcommitment** (`YARN-1011`). As already discussed, in order to further improve the cluster resource utilization, we can schedule containers not based on the allocated resources but on the actually utilized ones. When over-committing resources, there is the risk of running out of resources in case we have an increase in the utilized resources of the already running containers. Therefore, opportunistic execution should be used for containers whose allocation goes beyond the capacity of a node. This way, we can choose opportunistic containers to kill for reclaiming resources.
+* **NM Queue reordering** (`YARN-5886`). Instead of executing queued containers in a FIFO order, we can employ reordering strategies that dynamically determine which opportunistic container will be executed next. For example, we can prioritize containers that are expected to be short-running or which belong to applications that are close to completion.
+* **Out of order killing at NMs** (`YARN-5887`). As described above, when we need to free up resources for a guaranteed container to start its execution, we kill opportunistic containers in reverse order of arrival (first the most recently started ones). This might not always be the right decision. For example, we might want to minimize the number of containers killed or to refrain from killing containers of jobs that are very close to completion.
+* **Container pausing** (`YARN-5292`): At the moment we kill opportunistic containers to make room for guaranteed in case of resource contention. In busy clusters this can lower the effective cluster utilization: whenever we kill a running opportunistic container, it has to be restarted, and thus we lose work. To this end, we can instead pause running opportunistic containers. Note that this will require support from the container executor (e.g., the container technology used) and from the application.
+* **Container promotion** (`YARN-5085`). There are cases where changing the execution type of a container during its execution can be beneficial. For instance, an application might submit a container as opportunistic, and when its execution starts, it can request its promotion to a guaranteed container to avoid it getting killed.


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/29] hadoop git commit: YARN-5999. AMRMClientAsync will stop if any exceptions thrown on allocate call. Contributed by Jian He

Posted by as...@apache.org.
YARN-5999. AMRMClientAsync will stop if any exceptions thrown on allocate call. Contributed by Jian He


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/64a2d5be
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/64a2d5be
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/64a2d5be

Branch: refs/heads/YARN-5085
Commit: 64a2d5be91a7f344aadf820481c542c967bc46a8
Parents: f5e0bd3
Author: Xuan <xg...@apache.org>
Authored: Wed Dec 14 14:33:23 2016 -0800
Committer: Xuan <xg...@apache.org>
Committed: Wed Dec 14 14:33:23 2016 -0800

----------------------------------------------------------------------
 .../api/async/impl/AMRMClientAsyncImpl.java     | 30 ++++++++------------
 .../api/async/impl/TestAMRMClientAsync.java     |  2 +-
 2 files changed, 13 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/64a2d5be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
index bc6cadd..3dd53d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
@@ -61,7 +61,7 @@ extends AMRMClientAsync<T> {
   private final HeartbeatThread heartbeatThread;
   private final CallbackHandlerThread handlerThread;
 
-  private final BlockingQueue<AllocateResponse> responseQueue;
+  private final BlockingQueue<Object> responseQueue;
   
   private final Object unregisterHeartbeatLock = new Object();
   
@@ -70,8 +70,6 @@ extends AMRMClientAsync<T> {
   
   private volatile String collectorAddr;
 
-  private volatile Throwable savedException;
-
   /**
    *
    * @param intervalMs heartbeat interval in milliseconds between AM and RM
@@ -90,7 +88,6 @@ extends AMRMClientAsync<T> {
     handlerThread = new CallbackHandlerThread();
     responseQueue = new LinkedBlockingQueue<>();
     keepRunning = true;
-    savedException = null;
   }
 
   /**
@@ -111,9 +108,8 @@ extends AMRMClientAsync<T> {
     super(client, intervalMs, callbackHandler);
     heartbeatThread = new HeartbeatThread();
     handlerThread = new CallbackHandlerThread();
-    responseQueue = new LinkedBlockingQueue<AllocateResponse>();
+    responseQueue = new LinkedBlockingQueue<Object>();
     keepRunning = true;
-    savedException = null;
   }
 
   @Override
@@ -265,7 +261,7 @@ extends AMRMClientAsync<T> {
     
     public void run() {
       while (true) {
-        AllocateResponse response = null;
+        Object response = null;
         // synchronization ensures we don't send heartbeats after unregistering
         synchronized (unregisterHeartbeatLock) {
           if (!keepRunning) {
@@ -280,10 +276,7 @@ extends AMRMClientAsync<T> {
             return;
           } catch (Throwable ex) {
             LOG.error("Exception on heartbeat", ex);
-            savedException = ex;
-            // interrupt handler thread in case it waiting on the queue
-            handlerThread.interrupt();
-            return;
+            response = ex;
           }
           if (response != null) {
             while (true) {
@@ -316,19 +309,20 @@ extends AMRMClientAsync<T> {
           return;
         }
         try {
-          AllocateResponse response;
-          if(savedException != null) {
-            LOG.error("Stopping callback due to: ", savedException);
-            handler.onError(savedException);
-            return;
-          }
+          Object object;
           try {
-            response = responseQueue.take();
+            object = responseQueue.take();
           } catch (InterruptedException ex) {
             LOG.info("Interrupted while waiting for queue", ex);
             continue;
           }
+          if (object instanceof Throwable) {
+            progress = handler.getProgress();
+            handler.onError((Throwable) object);
+            continue;
+          }
 
+          AllocateResponse response = (AllocateResponse) object;
           String collectorAddress = response.getCollectorAddr();
           TimelineClient timelineClient = client.getRegisteredTimelineClient();
           if (timelineClient != null && collectorAddress != null

http://git-wip-us.apache.org/repos/asf/hadoop/blob/64a2d5be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
index dac82e4..ba38340 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
@@ -213,7 +213,7 @@ public class TestAMRMClientAsync {
     
     asyncClient.stop();
     // stopping should have joined all threads and completed all callbacks
-    Assert.assertTrue(callbackHandler.callbackCount == 0);
+    Assert.assertTrue(callbackHandler.callbackCount > 0);
   }
 
   @Test (timeout = 10000)


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/29] hadoop git commit: HADOOP-13709. Ability to clean up subprocesses spawned by Shell when the process exits. Contributed by Eric Badger

Posted by as...@apache.org.
HADOOP-13709. Ability to clean up subprocesses spawned by Shell when the process exits. Contributed by Eric Badger


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/631f1dae
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/631f1dae
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/631f1dae

Branch: refs/heads/YARN-5085
Commit: 631f1daee3507a1adbc68b937cca31c27dbe8d3d
Parents: 169bfc0
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Dec 15 20:52:40 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Dec 15 20:52:40 2016 +0000

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/util/Shell.java | 24 +++++++++
 .../java/org/apache/hadoop/util/TestShell.java  | 52 ++++++++++++++++++++
 2 files changed, 76 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/631f1dae/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
index 5fc9869..83877b7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
@@ -26,9 +26,11 @@ import java.io.InputStream;
 import java.io.InterruptedIOException;
 import java.nio.charset.Charset;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Map;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.WeakHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -48,6 +50,8 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public abstract class Shell {
+  private static final Map <Process, Object> CHILD_PROCESSES =
+      Collections.synchronizedMap(new WeakHashMap<Process, Object>());
   public static final Logger LOG = LoggerFactory.getLogger(Shell.class);
 
   /**
@@ -916,6 +920,7 @@ public abstract class Shell {
     } else {
       process = builder.start();
     }
+    CHILD_PROCESSES.put(process, null);
 
     if (timeOutInterval > 0) {
       timeOutTimer = new Timer("Shell command timeout");
@@ -1012,6 +1017,7 @@ public abstract class Shell {
         LOG.warn("Error while closing the error stream", ioe);
       }
       process.destroy();
+      CHILD_PROCESSES.remove(process);
       lastTime = Time.monotonicNow();
     }
   }
@@ -1310,4 +1316,22 @@ public abstract class Shell {
       }
     }
   }
+
+  /**
+   * Static method to destroy all running <code>Shell</code> processes
+   * Iterates through a list of all currently running <code>Shell</code>
+   * processes and destroys them one by one. This method is thread safe and
+   * is intended to be used in a shutdown hook.
+   */
+  public static void destroyAllProcesses() {
+    synchronized (CHILD_PROCESSES) {
+      for (Process key : CHILD_PROCESSES.keySet()) {
+        Process process = key;
+        if (key != null) {
+          process.destroy();
+        }
+      }
+      CHILD_PROCESSES.clear();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/631f1dae/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
index 67903f7..88859b5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.util;
 
+import com.google.common.base.Supplier;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.security.alias.AbstractJavaKeyStoreProvider;
 import org.junit.Assert;
@@ -471,4 +472,55 @@ public class TestShell extends Assert {
     assertEquals("'foo'\\''bar'", Shell.bashQuote("foo'bar"));
     assertEquals("''\\''foo'\\''bar'\\'''", Shell.bashQuote("'foo'bar'"));
   }
+
+  @Test(timeout=120000)
+  public void testShellKillAllProcesses() throws Throwable {
+    Assume.assumeFalse(WINDOWS);
+    StringBuffer sleepCommand = new StringBuffer();
+    sleepCommand.append("sleep 200");
+    String[] shellCmd = {"bash", "-c", sleepCommand.toString()};
+    final ShellCommandExecutor shexc1 = new ShellCommandExecutor(shellCmd);
+    final ShellCommandExecutor shexc2 = new ShellCommandExecutor(shellCmd);
+
+    Thread shellThread1 = new Thread() {
+      @Override
+      public void run() {
+        try {
+          shexc1.execute();
+        } catch(IOException ioe) {
+          //ignore IOException from thread interrupt
+        }
+      }
+    };
+    Thread shellThread2 = new Thread() {
+      @Override
+      public void run() {
+        try {
+          shexc2.execute();
+        } catch(IOException ioe) {
+          //ignore IOException from thread interrupt
+        }
+      }
+    };
+
+    shellThread1.start();
+    shellThread2.start();
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return shexc1.getProcess() != null;
+      }
+    }, 10, 10000);
+
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return shexc2.getProcess() != null;
+      }
+    }, 10, 10000);
+
+    Shell.destroyAllProcesses();
+    shexc1.getProcess().waitFor();
+    shexc2.getProcess().waitFor();
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/29] hadoop git commit: HADOOP-13900. Remove snapshot version of SDK dependency from Azure Data Lake Store File System. Contributed by Vishwajeet Dusane

Posted by as...@apache.org.
HADOOP-13900. Remove snapshot version of SDK dependency from Azure Data Lake Store File System. Contributed by Vishwajeet Dusane


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ef34bf2b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ef34bf2b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ef34bf2b

Branch: refs/heads/YARN-5085
Commit: ef34bf2bb92a4e8def6617b185ae72db81450de8
Parents: b0b033e
Author: Mingliang Liu <li...@apache.org>
Authored: Tue Dec 13 10:34:32 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Tue Dec 13 10:34:32 2016 -0800

----------------------------------------------------------------------
 hadoop-tools/hadoop-azure-datalake/pom.xml | 10 +---------
 1 file changed, 1 insertion(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef34bf2b/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index 7f872c3..b2d419f 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -34,14 +34,6 @@
     <file.encoding>UTF-8</file.encoding>
     <downloadSources>true</downloadSources>
   </properties>
-  <repositories>
-    <repository>
-      <id>snapshots-repo</id>
-      <url>https://oss.sonatype.org/content/repositories/snapshots</url>
-      <releases><enabled>false</enabled></releases>
-      <snapshots><enabled>true</enabled></snapshots>
-    </repository>
-  </repositories>
   <build>
     <plugins>
       <plugin>
@@ -128,7 +120,7 @@
     <dependency>
       <groupId>com.microsoft.azure</groupId>
       <artifactId>azure-data-lake-store-sdk</artifactId>
-      <version>2.0.4-SNAPSHOT</version>
+      <version>2.0.11</version>
     </dependency>
     <!--  ENDS HERE-->
     <dependency>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/29] hadoop git commit: HDFS-11094. Send back HAState along with NamespaceInfo during a versionRequest as an optional parameter. Contributed by Eric Badger

Posted by as...@apache.org.
HDFS-11094. Send back HAState along with NamespaceInfo during a versionRequest as an optional parameter. Contributed by Eric Badger


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ac0d402c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ac0d402c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ac0d402c

Branch: refs/heads/YARN-5085
Commit: ac0d402c6a3af784aa8bdceecf3293c30760346f
Parents: 631f1da
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Dec 15 12:48:27 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Dec 15 14:53:53 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 72 +++++++++++++-------
 .../hdfs/server/datanode/BPOfferService.java    | 10 ++-
 .../hdfs/server/datanode/BPServiceActor.java    |  4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  8 ++-
 .../hdfs/server/protocol/NamespaceInfo.java     | 26 +++++++
 .../src/main/proto/DatanodeProtocol.proto       | 13 ----
 .../hadoop-hdfs/src/main/proto/HdfsServer.proto | 15 ++++
 .../server/datanode/TestBPOfferService.java     | 32 +++++++++
 .../hdfs/server/namenode/TestFSNamesystem.java  | 21 ++++++
 9 files changed, 158 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac0d402c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 4f6a04e..d97708f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeComm
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHeartbeatProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
@@ -66,6 +65,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeCommandPro
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto.NamenodeRoleProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamespaceInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NNHAStatusHeartbeatProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RecoveringBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogManifestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogProto;
@@ -337,7 +337,8 @@ public class PBHelper {
     StorageInfoProto storage = info.getStorageInfo();
     return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
         info.getBlockPoolID(), storage.getCTime(), info.getBuildVersion(),
-        info.getSoftwareVersion(), info.getCapabilities());
+        info.getSoftwareVersion(), info.getCapabilities(),
+        convert(info.getState()));
   }
 
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
@@ -743,43 +744,64 @@ public class PBHelper {
   }
   
   public static NamespaceInfoProto convert(NamespaceInfo info) {
-    return NamespaceInfoProto.newBuilder()
-        .setBlockPoolID(info.getBlockPoolID())
+    NamespaceInfoProto.Builder builder = NamespaceInfoProto.newBuilder();
+    builder.setBlockPoolID(info.getBlockPoolID())
         .setBuildVersion(info.getBuildVersion())
         .setUnused(0)
         .setStorageInfo(PBHelper.convert((StorageInfo)info))
         .setSoftwareVersion(info.getSoftwareVersion())
-        .setCapabilities(info.getCapabilities())
-        .build();
+        .setCapabilities(info.getCapabilities());
+    HAServiceState state = info.getState();
+    if(state != null) {
+      builder.setState(convert(info.getState()));
+    }
+    return builder.build();
   }
 
-  public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
-    if (s == null) return null;
-    switch (s.getState()) {
+  public static HAServiceState convert(NNHAStatusHeartbeatProto.State s) {
+    if (s == null) {
+      return null;
+    }
+    switch (s) {
     case ACTIVE:
-      return new NNHAStatusHeartbeat(HAServiceState.ACTIVE, s.getTxid());
+      return HAServiceState.ACTIVE;
     case STANDBY:
-      return new NNHAStatusHeartbeat(HAServiceState.STANDBY, s.getTxid());
+      return HAServiceState.STANDBY;
     default:
-      throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" + s.getState());
+      throw new IllegalArgumentException("Unexpected HAServiceStateProto:"
+          + s);
     }
   }
 
+  public static NNHAStatusHeartbeatProto.State convert(HAServiceState s) {
+    if (s == null) {
+      return null;
+    }
+    switch (s) {
+    case ACTIVE:
+      return NNHAStatusHeartbeatProto.State.ACTIVE;
+    case STANDBY:
+      return NNHAStatusHeartbeatProto.State.STANDBY;
+    default:
+      throw new IllegalArgumentException("Unexpected HAServiceState:"
+          + s);
+    }
+  }
+
+  public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
+    if (s == null) {
+      return null;
+    }
+    return new NNHAStatusHeartbeat(convert(s.getState()), s.getTxid());
+  }
+
   public static NNHAStatusHeartbeatProto convert(NNHAStatusHeartbeat hb) {
-    if (hb == null) return null;
-    NNHAStatusHeartbeatProto.Builder builder =
-      NNHAStatusHeartbeatProto.newBuilder();
-    switch (hb.getState()) {
-      case ACTIVE:
-        builder.setState(NNHAStatusHeartbeatProto.State.ACTIVE);
-        break;
-      case STANDBY:
-        builder.setState(NNHAStatusHeartbeatProto.State.STANDBY);
-        break;
-      default:
-        throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" +
-            hb.getState());
+    if (hb == null) {
+      return null;
     }
+    NNHAStatusHeartbeatProto.Builder builder =
+        NNHAStatusHeartbeatProto.newBuilder();
+    builder.setState(convert(hb.getState()));
     builder.setTxid(hb.getTxId());
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac0d402c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 00102eb..00e6b3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -307,8 +307,16 @@ class BPOfferService {
    * verifies that this namespace matches (eg to prevent a misconfiguration
    * where a StandbyNode from a different cluster is specified)
    */
-  void verifyAndSetNamespaceInfo(NamespaceInfo nsInfo) throws IOException {
+  void verifyAndSetNamespaceInfo(BPServiceActor actor, NamespaceInfo nsInfo)
+    throws IOException {
     writeLock();
+
+    if(nsInfo.getState() == HAServiceState.ACTIVE
+        && bpServiceToActive == null) {
+      LOG.info("Acknowledging ACTIVE Namenode during handshake" + actor);
+      bpServiceToActive = actor;
+    }
+
     try {
       if (this.bpNSInfo == null) {
         this.bpNSInfo = nsInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac0d402c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index bb2b792..25ab3b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -269,11 +269,11 @@ class BPServiceActor implements Runnable {
     // First phase of the handshake with NN - get the namespace
     // info.
     NamespaceInfo nsInfo = retrieveNamespaceInfo();
-    
+
     // Verify that this matches the other NN in this HA pair.
     // This also initializes our block pool in the DN if we are
     // the first NN connection for this BP.
-    bpos.verifyAndSetNamespaceInfo(nsInfo);
+    bpos.verifyAndSetNamespaceInfo(this, nsInfo);
     
     // Second phase of the handshake with the NN.
     register(nsInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac0d402c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 8a750a0..90fb924 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1594,7 +1594,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   NamespaceInfo unprotectedGetNamespaceInfo() {
     return new NamespaceInfo(getFSImage().getStorage().getNamespaceID(),
         getClusterId(), getBlockPoolId(),
-        getFSImage().getStorage().getCTime());
+        getFSImage().getStorage().getCTime(), getState());
   }
 
   /**
@@ -4531,12 +4531,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       return 0;
     }
   }
-  
+
   @Metric
   public int getBlockCapacity() {
     return blockManager.getCapacity();
   }
 
+  public HAServiceState getState() {
+    return haContext == null ? null : haContext.getState().getServiceState();
+  }
+
   @Override // FSNamesystemMBean
   public String getFSState() {
     return isInSafeMode() ? "safeMode" : "Operational";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac0d402c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
index 90d0aac..66ce9ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -44,6 +45,7 @@ public class NamespaceInfo extends StorageInfo {
   String blockPoolID = "";    // id of the block pool
   String softwareVersion;
   long capabilities;
+  HAServiceState state;
 
   // only authoritative on the server-side to determine advertisement to
   // clients.  enum will update the supported values
@@ -88,6 +90,14 @@ public class NamespaceInfo extends StorageInfo {
         CAPABILITIES_SUPPORTED);
   }
 
+  public NamespaceInfo(int nsID, String clusterID, String bpID,
+      long cT, String buildVersion, String softwareVersion,
+      long capabilities, HAServiceState st) {
+    this(nsID, clusterID, bpID, cT, buildVersion, softwareVersion,
+        capabilities);
+    this.state = st;
+  }
+
   // for use by server and/or client
   public NamespaceInfo(int nsID, String clusterID, String bpID,
       long cT, String buildVersion, String softwareVersion,
@@ -105,6 +115,13 @@ public class NamespaceInfo extends StorageInfo {
     this(nsID, clusterID, bpID, cT, Storage.getBuildVersion(),
         VersionInfo.getVersion());
   }
+
+  public NamespaceInfo(int nsID, String clusterID, String bpID,
+      long cT, HAServiceState st) {
+    this(nsID, clusterID, bpID, cT, Storage.getBuildVersion(),
+        VersionInfo.getVersion());
+    this.state = st;
+  }
   
   public long getCapabilities() {
     return capabilities;
@@ -115,6 +132,11 @@ public class NamespaceInfo extends StorageInfo {
     this.capabilities = capabilities;
   }
 
+  @VisibleForTesting
+  public void setState(HAServiceState state) {
+    this.state = state;
+  }
+
   public boolean isCapabilitySupported(Capability capability) {
     Preconditions.checkArgument(capability != Capability.UNKNOWN,
         "cannot test for unknown capability");
@@ -134,6 +156,10 @@ public class NamespaceInfo extends StorageInfo {
     return softwareVersion;
   }
 
+  public HAServiceState getState() {
+    return state;
+  }
+
   @Override
   public String toString(){
     return super.toString() + ";bpid=" + blockPoolID;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac0d402c/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 016eae2..7423b33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -209,19 +209,6 @@ message HeartbeatRequestProto {
 }
 
 /**
- * state - State the NN is in when returning response to the DN
- * txid - Highest transaction ID this NN has seen
- */
-message NNHAStatusHeartbeatProto {
-  enum State {
-    ACTIVE = 0;
-    STANDBY = 1;
-  }
-  required State state = 1; 
-  required uint64 txid = 2;
-}
-
-/**
  * cmds - Commands from namenode to datanode.
  * haStatus - Status (from an HA perspective) of the NN sending this response
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac0d402c/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
index 910e03b..e50883a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
@@ -32,6 +32,7 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
 import "hdfs.proto";
+import "HAServiceProtocol.proto";
 
 /**
  * Block access token information
@@ -101,6 +102,7 @@ message NamespaceInfoProto {
   required StorageInfoProto storageInfo = 4;// Node information
   required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
   optional uint64 capabilities = 6 [default = 0]; // feature flags
+  optional NNHAStatusHeartbeatProto.State state = 7;
 }
 
 /**
@@ -201,3 +203,16 @@ message NamenodeRegistrationProto {
   required StorageInfoProto storageInfo = 3;  // Node information
   optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
 }
+
+/**
+ * state - State the NN is in when returning response to the DN
+ * txid - Highest transaction ID this NN has seen
+ */
+message NNHAStatusHeartbeatProto {
+  enum State {
+    ACTIVE = 0;
+    STANDBY = 1;
+  }
+  required State state = 1;
+  required uint64 txid = 2;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac0d402c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
index 2d50c75..b7b8966 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
@@ -799,4 +800,35 @@ public class TestBPOfferService {
     }
     return -1;
   }
+
+   /*
+    *
+    */
+  @Test
+  public void testNNHAStateUpdateFromVersionRequest() throws Exception {
+    final BPOfferService bpos = setupBPOSForNNs(mockNN1, mockNN2);
+    Mockito.doReturn(true).when(mockDn).areHeartbeatsDisabledForTests();
+    BPServiceActor actor = bpos.getBPServiceActors().get(0);
+    bpos.start();
+    waitForInitialization(bpos);
+    // Should start with neither NN as active.
+    assertNull(bpos.getActiveNN());
+
+    // getNamespaceInfo() will not include HAServiceState
+    NamespaceInfo nsInfo = mockNN1.versionRequest();
+    bpos.verifyAndSetNamespaceInfo(actor, nsInfo);
+
+    assertNull(bpos.getActiveNN());
+
+    // Change mock so getNamespaceInfo() will include HAServiceState
+    Mockito.doReturn(new NamespaceInfo(1, FAKE_CLUSTERID, FAKE_BPID, 0,
+        HAServiceState.ACTIVE)).when(mockNN1).versionRequest();
+
+    // Update the bpos NamespaceInfo
+    nsInfo = mockNN1.versionRequest();
+    bpos.verifyAndSetNamespaceInfo(actor, nsInfo);
+
+    assertNotNull(bpos.getActiveNN());
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac0d402c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
index f02c679..6a0dd6f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
@@ -33,6 +33,7 @@ import java.util.Collection;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -43,6 +44,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.junit.After;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -155,6 +157,25 @@ public class TestFSNamesystem {
   }
 
   @Test
+  public void testHAStateInNamespaceInfo() throws IOException {
+    Configuration conf = new Configuration();
+
+    FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);
+    FSImage fsImage = Mockito.mock(FSImage.class);
+    Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog);
+    NNStorage nnStorage = Mockito.mock(NNStorage.class);
+    Mockito.when(fsImage.getStorage()).thenReturn(nnStorage);
+
+    FSNamesystem fsNamesystem = new FSNamesystem(conf, fsImage);
+    FSNamesystem fsn = Mockito.spy(fsNamesystem);
+    Mockito.when(fsn.getState()).thenReturn(
+        HAServiceProtocol.HAServiceState.ACTIVE);
+
+    NamespaceInfo nsInfo = fsn.unprotectedGetNamespaceInfo();
+    assertNotNull(nsInfo.getState());
+  }
+
+  @Test
   public void testReset() throws Exception {
     Configuration conf = new Configuration();
     FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/29] hadoop git commit: HDFS-11188. Change min supported DN and NN versions back to 2.x. Contributed by Andrew Wang.

Posted by as...@apache.org.
HDFS-11188. Change min supported DN and NN versions back to 2.x. Contributed by Andrew Wang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cee0c468
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cee0c468
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cee0c468

Branch: refs/heads/YARN-5085
Commit: cee0c468b0ebe2ae8727502620e345745ba11f77
Parents: aebb912
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Dec 15 16:48:50 2016 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Dec 15 16:48:50 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cee0c468/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 80ceaaf..15bb0bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -290,7 +290,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_MAX_EXTRA_EDITS_SEGMENTS_RETAINED_KEY = "dfs.namenode.max.extra.edits.segments.retained";
   public static final int     DFS_NAMENODE_MAX_EXTRA_EDITS_SEGMENTS_RETAINED_DEFAULT = 10000; // 10k
   public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY = "dfs.namenode.min.supported.datanode.version";
-  public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_DEFAULT = "3.0.0-alpha1-SNAPSHOT";
+  public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_DEFAULT = "2.1.0-beta";
 
   public static final String  DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
   public static final int     DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
@@ -613,7 +613,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_DATANODE_IPC_DEFAULT_PORT = 9867;
   public static final String  DFS_DATANODE_IPC_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_IPC_DEFAULT_PORT;
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY = "dfs.datanode.min.supported.namenode.version";
-  public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "3.0.0-alpha1-SNAPSHOT";
+  public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "2.1.0-beta";
   public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY = "dfs.namenode.inode.attributes.provider.class";
   public static final String  DFS_DATANODE_BP_READY_TIMEOUT_KEY = "dfs.datanode.bp-ready.timeout";
   public static final long    DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT = 20;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/29] hadoop git commit: HADOOP-11804. Shaded Hadoop client artifacts and minicluster. Contributed by Sean Busbey.

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/hadoop-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client/pom.xml b/hadoop-client-modules/hadoop-client/pom.xml
new file mode 100644
index 0000000..0394cae
--- /dev/null
+++ b/hadoop-client-modules/hadoop-client/pom.xml
@@ -0,0 +1,338 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+<parent>
+   <groupId>org.apache.hadoop</groupId>
+   <artifactId>hadoop-project-dist</artifactId>
+   <version>3.0.0-alpha2-SNAPSHOT</version>
+   <relativePath>../../hadoop-project-dist</relativePath>
+</parent>
+  <artifactId>hadoop-client</artifactId>
+  <version>3.0.0-alpha2-SNAPSHOT</version>
+  <packaging>pom</packaging>
+
+  <description>Apache Hadoop Client aggregation pom with dependencies exposed</description>
+  <name>Apache Hadoop Client Aggregator</name>
+
+<properties>
+   <hadoop.component>client</hadoop.component>
+ </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>javax.servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>jetty</groupId>
+          <artifactId>org.eclipse.jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-server</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-util</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>servlet-api-2.5</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-json</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-server</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jdt</groupId>
+          <artifactId>core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro-ipc</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>net.sf.kosmosfs</groupId>
+          <artifactId>kfs</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>net.java.dev.jets3t</groupId>
+          <artifactId>jets3t</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.jcraft</groupId>
+          <artifactId>jsch</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.zookeeper</groupId>
+          <artifactId>zookeeper</artifactId>
+        </exclusion>
+        <!-- No slf4j backends for downstream clients -->
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs-client</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-server</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-server</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>javax.servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-app</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>javax.servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-server-nodemanager</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-server-web-proxy</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject.extensions</groupId>
+          <artifactId>guice-servlet</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>jline</groupId>
+          <artifactId>jline</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+        <!-- No slf4j backends for downstream clients -->
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.zookeeper</groupId>
+          <artifactId>zookeeper</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject</groupId>
+          <artifactId>guice</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-server</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.contribs</groupId>
+          <artifactId>jersey-guice</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject.extensions</groupId>
+          <artifactId>guice-servlet</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-json</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject</groupId>
+          <artifactId>guice</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-server</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.contribs</groupId>
+          <artifactId>jersey-guice</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject.extensions</groupId>
+          <artifactId>guice-servlet</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-json</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+        <!-- No slf4j backends for downstream clients -->
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject.extensions</groupId>
+          <artifactId>guice-servlet</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+        <!-- No slf4j backends for downstream clients -->
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-annotations</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>jdk.tools</groupId>
+          <artifactId>jdk.tools</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+
+</project>
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client-modules/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/pom.xml b/hadoop-client-modules/pom.xml
new file mode 100644
index 0000000..eea4c8f
--- /dev/null
+++ b/hadoop-client-modules/pom.xml
@@ -0,0 +1,45 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-alpha2-SNAPSHOT</version>
+    <relativePath>../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-client-modules</artifactId>
+  <packaging>pom</packaging>
+  <description>multi-module for Apache Hadoop client artifacts</description>
+  <name>Apache Hadoop Client Modules</name>
+
+  <modules>
+    <!-- Left as an empty artifact w/dep for compat -->
+    <module>hadoop-client</module>
+    <!-- Should be used at compile scope for access to IA.Public classes -->
+    <module>hadoop-client-api</module>
+    <!-- Should be used at runtime scope for remaining classes necessary for hadoop-client-api to function -->
+    <module>hadoop-client-runtime</module>
+    <!-- Should be used at test scope for those that need access to mini cluster that works with above api and runtime -->
+    <module>hadoop-client-minicluster</module>
+    <!-- Checks invariants above -->
+    <module>hadoop-client-check-invariants</module>
+    <module>hadoop-client-check-test-invariants</module>
+    <!-- Attempt to use the created libraries -->
+    <module>hadoop-client-integration-tests</module>
+  </modules>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client/pom.xml b/hadoop-client/pom.xml
deleted file mode 100644
index dbd4fac..0000000
--- a/hadoop-client/pom.xml
+++ /dev/null
@@ -1,319 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
-   http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-<parent>
-   <groupId>org.apache.hadoop</groupId>
-   <artifactId>hadoop-project-dist</artifactId>
-   <version>3.0.0-alpha2-SNAPSHOT</version>
-   <relativePath>../hadoop-project-dist</relativePath>
- </parent>
-  <artifactId>hadoop-client</artifactId>
-  <version>3.0.0-alpha2-SNAPSHOT</version>
-  <packaging>jar</packaging>
-
-  <description>Apache Hadoop Client</description>
-  <name>Apache Hadoop Client</name>
-
-<properties>
-   <hadoop.component>client</hadoop.component>
- </properties>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>compile</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>javax.servlet-api</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>commons-logging</groupId>
-          <artifactId>commons-logging-api</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>jetty</groupId>
-          <artifactId>org.eclipse.jetty</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.eclipse.jetty</groupId>
-          <artifactId>jetty-server</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.eclipse.jetty</groupId>
-          <artifactId>jetty-util</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.eclipse.jetty</groupId>
-          <artifactId>servlet-api-2.5</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-core</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-json</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-server</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.eclipse.jdt</groupId>
-          <artifactId>core</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.avro</groupId>
-          <artifactId>avro-ipc</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>net.sf.kosmosfs</groupId>
-          <artifactId>kfs</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>net.java.dev.jets3t</groupId>
-          <artifactId>jets3t</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.jcraft</groupId>
-          <artifactId>jsch</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.zookeeper</groupId>
-          <artifactId>zookeeper</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs-client</artifactId>
-      <scope>compile</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.avro</groupId>
-          <artifactId>avro</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.eclipse.jetty</groupId>
-          <artifactId>jetty-server</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-core</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-server</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>javax.servlet-api</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-app</artifactId>
-      <scope>compile</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>javax.servlet-api</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-yarn-server-nodemanager</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-yarn-server-web-proxy</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.inject.extensions</groupId>
-          <artifactId>guice-servlet</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>junit</groupId>
-          <artifactId>junit</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.avro</groupId>
-          <artifactId>avro</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>jline</groupId>
-          <artifactId>jline</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>io.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.zookeeper</groupId>
-          <artifactId>zookeeper</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-      <scope>compile</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.inject</groupId>
-          <artifactId>guice</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey.jersey-test-framework</groupId>
-          <artifactId>jersey-test-framework-grizzly2</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-server</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey.contribs</groupId>
-          <artifactId>jersey-guice</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.inject.extensions</groupId>
-          <artifactId>guice-servlet</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.avro</groupId>
-          <artifactId>avro</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-core</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-json</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>io.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-core</artifactId>
-      <scope>compile</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>junit</groupId>
-          <artifactId>junit</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.inject</groupId>
-          <artifactId>guice</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey.jersey-test-framework</groupId>
-          <artifactId>jersey-test-framework-grizzly2</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-server</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey.contribs</groupId>
-          <artifactId>jersey-guice</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.avro</groupId>
-          <artifactId>avro</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.inject.extensions</groupId>
-          <artifactId>guice-servlet</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-json</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>io.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-      <scope>compile</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>junit</groupId>
-          <artifactId>junit</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.avro</groupId>
-          <artifactId>avro</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.inject.extensions</groupId>
-          <artifactId>guice-servlet</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>io.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-annotations</artifactId>
-      <scope>compile</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>jdk.tools</groupId>
-          <artifactId>jdk.tools</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    
-  </dependencies>
-
-</project>
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-dist/pom.xml b/hadoop-dist/pom.xml
index 30e6472..ae8c4ce 100644
--- a/hadoop-dist/pom.xml
+++ b/hadoop-dist/pom.xml
@@ -51,6 +51,23 @@
       <artifactId>hadoop-yarn-api</artifactId>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-check-invariants</artifactId>
+      <type>pom</type>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-check-test-invariants</artifactId>
+      <type>pom</type>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-integration-tests</artifactId>
+      <scope>provided</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-maven-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/pom.xml b/hadoop-maven-plugins/pom.xml
index eae03a1..6a2bd73 100644
--- a/hadoop-maven-plugins/pom.xml
+++ b/hadoop-maven-plugins/pom.xml
@@ -54,6 +54,28 @@
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.maven.plugins</groupId>
+      <artifactId>maven-shade-plugin</artifactId>
+      <version>${maven-shade-plugin.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <!-- shade conflicts with itself and our other maven internals
+             by including a combination of 3.x and 2.x
+          -->
+        <exclusion>
+          <groupId>org.apache.maven.shared</groupId>
+          <artifactId>maven-dependency-tree</artifactId>
+        </exclusion>
+        <!-- again, shade conflicts with itself on the transitives of this
+             dependency
+          -->
+        <exclusion>
+          <groupId>org.vafer</groupId>
+          <artifactId>jdependency</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
   </dependencies>
   <build>
     <plugins>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/shade/resource/ServicesResourceTransformer.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/shade/resource/ServicesResourceTransformer.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/shade/resource/ServicesResourceTransformer.java
new file mode 100644
index 0000000..a54b5ba
--- /dev/null
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/shade/resource/ServicesResourceTransformer.java
@@ -0,0 +1,164 @@
+package org.apache.hadoop.maven.plugin.shade.resource;
+/*
+ * 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.
+ *
+ */
+import java.io.BufferedReader;
+import org.apache.maven.plugins.shade.relocation.Relocator;
+import org.apache.maven.plugins.shade.resource.ResourceTransformer;
+import org.codehaus.plexus.util.IOUtil;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.jar.JarEntry;
+import java.util.jar.JarOutputStream;
+
+/**
+ * Resources transformer that appends entries in META-INF/services resources
+ * into a single resource. For example, if there are several
+ * META-INF/services/org.apache.maven.project.ProjectBuilder resources spread
+ * across many JARs the individual entries will all be concatenated into a
+ * single META-INF/services/org.apache.maven.project.ProjectBuilder resource
+ * packaged into the resultant JAR produced by the shading process.
+ *
+ * From following sources, only needed until MSHADE-182 gets released
+ * * https://s.apache.org/vwjl (source in maven-shade-plugin repo)
+ * * https://issues.apache.org/jira/secure/attachment/12718938/MSHADE-182.patch
+ *
+ * Has been reformatted according to Hadoop checkstyle rules and modified
+ * to meet Hadoop's threshold for Findbugs problems.
+ */
+public class ServicesResourceTransformer
+    implements ResourceTransformer {
+
+  private static final String SERVICES_PATH = "META-INF/services";
+
+  private Map<String, ServiceStream> serviceEntries = new HashMap<>();
+
+  private List<Relocator> relocators;
+
+  public boolean canTransformResource(String resource) {
+    if (resource.startsWith(SERVICES_PATH)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  public void processResource(String resource, InputStream is,
+      List<Relocator> relocatorz) throws IOException {
+    ServiceStream out = serviceEntries.get(resource);
+    if (out == null) {
+      out = new ServiceStream();
+      serviceEntries.put(resource, out);
+    }
+
+    out.append(is);
+    is.close();
+
+    if (this.relocators == null) {
+      this.relocators = relocatorz;
+    }
+  }
+
+  public boolean hasTransformedResource() {
+    return serviceEntries.size() > 0;
+  }
+
+  public void modifyOutputStream(JarOutputStream jos)
+      throws IOException {
+    for (Map.Entry<String, ServiceStream> entry : serviceEntries.entrySet()) {
+      String key = entry.getKey();
+      ServiceStream data = entry.getValue();
+
+      if (relocators != null) {
+        key = key.substring(SERVICES_PATH.length() + 1);
+        for (Relocator relocator : relocators) {
+          if (relocator.canRelocateClass(key)) {
+            key = relocator.relocateClass(key);
+            break;
+          }
+        }
+
+        key = SERVICES_PATH + '/' + key;
+      }
+
+      jos.putNextEntry(new JarEntry(key));
+
+      //read the content of service file for candidate classes for relocation
+      //presume everything is UTF8, because Findbugs barfs on default
+      //charset and this seems no worse a choice �\_(\u30c4)_/�
+      PrintWriter writer = new PrintWriter(new OutputStreamWriter(jos,
+          StandardCharsets.UTF_8));
+      InputStreamReader streamReader =
+          new InputStreamReader(data.toInputStream(), StandardCharsets.UTF_8);
+      BufferedReader reader = new BufferedReader(streamReader);
+      String className;
+
+      while ((className = reader.readLine()) != null) {
+
+        if (relocators != null) {
+          for (Relocator relocator : relocators) {
+            //if the class can be relocated then relocate it
+            if (relocator.canRelocateClass(className)) {
+              className = relocator.applyToSourceContent(className);
+              break;
+            }
+          }
+        }
+
+        writer.println(className);
+        writer.flush();
+      }
+
+      reader.close();
+      data.reset();
+    }
+  }
+
+  static class ServiceStream extends ByteArrayOutputStream {
+
+    public ServiceStream() {
+      super(1024);
+    }
+
+    public void append(InputStream is)
+        throws IOException {
+      if (count > 0 && buf[count - 1] != '\n' && buf[count - 1] != '\r') {
+        write('\n');
+      }
+
+      IOUtil.copy(is, this);
+    }
+
+    public InputStream toInputStream() {
+      return new ByteArrayInputStream(buf, 0, count);
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/shade/resource/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/shade/resource/package-info.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/shade/resource/package-info.java
new file mode 100644
index 0000000..571491c
--- /dev/null
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/shade/resource/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * Resource handling plugins used internal to the Hadoop build.
+ * IA.Private (build structure encourages not using the actual annotations)
+ */
+package org.apache.hadoop.maven.plugin.shade.resource;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 9ec24ea..36172f2 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -27,6 +27,7 @@
   <description>Apache Hadoop Project POM</description>
   <name>Apache Hadoop Project POM</name>
   <packaging>pom</packaging>
+  <inceptionYear>2008</inceptionYear>
 
   <properties>
     <!-- Set the Release year during release -->
@@ -114,6 +115,7 @@
     <maven-compiler-plugin.version>3.1</maven-compiler-plugin.version>
     <maven-install-plugin.version>2.5.1</maven-install-plugin.version>
     <maven-resources-plugin.version>2.6</maven-resources-plugin.version>
+    <maven-shade-plugin.version>2.4.3</maven-shade-plugin.version>
     <maven-jar-plugin.version>2.5</maven-jar-plugin.version>
     <maven-war-plugin.version>2.4</maven-war-plugin.version>
     <maven-source-plugin.version>2.3</maven-source-plugin.version>
@@ -150,6 +152,44 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-client-modules</artifactId>
+        <version>${project.version}</version>
+        <type>pom</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-client-api</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-client-check-invariants</artifactId>
+        <version>${project.version}</version>
+        <type>pom</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-client-check-test-invariants</artifactId>
+        <version>${project.version}</version>
+        <type>pom</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-client-integration-tests</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-client-runtime</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-client-minicluster</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-common</artifactId>
         <version>${project.version}</version>
       </dependency>
@@ -1245,6 +1285,40 @@
     <pluginManagement>
       <plugins>
         <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>build-helper-maven-plugin</artifactId>
+          <version>${build-helper-maven-plugin.version}</version>
+        </plugin>
+        <!-- We include the configuration for license-maven-plugin to correct
+             maven-shade-plugin generated poms because it's always the same. We
+             can't simply configure the plugin because we must ensure execution
+             happens in the package phase after the shade plugin runs.
+          -->
+        <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>license-maven-plugin</artifactId>
+          <version>1.10</version>
+          <configuration>
+            <canUpdateCopyright>false</canUpdateCopyright>
+            <roots><root>${project.basedir}</root></roots>
+          </configuration>
+          <executions>
+            <execution>
+              <id>update-pom-license</id>
+              <goals>
+                <goal>update-file-header</goal>
+              </goals>
+              <phase>package</phase>
+              <configuration>
+                <licenseName>apache_v2</licenseName>
+                <includes>
+                  <include>dependency-reduced-pom.xml</include>
+                </includes>
+              </configuration>
+            </execution>
+          </executions>
+        </plugin>
+        <plugin>
           <artifactId>maven-clean-plugin</artifactId>
           <version>${maven-clean-plugin.version}</version>
         </plugin>
@@ -1260,13 +1334,8 @@
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-dependency-plugin</artifactId>
-          <version>2.2</version>
-        </plugin>
-        <plugin>
-          <groupId>org.codehaus.mojo</groupId>
-          <artifactId>build-helper-maven-plugin</artifactId>
-          <version>${build-helper-maven-plugin.version}</version>
+          <artifactId>maven-shade-plugin</artifactId>
+          <version>${maven-shade-plugin.version}</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -1356,6 +1425,19 @@
 
     <plugins>
       <plugin>
+        <artifactId>maven-clean-plugin</artifactId>
+        <configuration>
+          <filesets>
+            <fileset>
+              <directory>${project.basedir}</directory>
+              <includes>
+                <include>dependency-reduced-pom.xml</include>
+              </includes>
+            </fileset>
+          </filesets>
+        </configuration>
+      </plugin>
+      <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>
       </plugin>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70ca1f1e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8776534..9de7b36 100644
--- a/pom.xml
+++ b/pom.xml
@@ -96,7 +96,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     <maven-stylus-skin.version>1.5</maven-stylus-skin.version>
     <maven-antrun-plugin.version>1.7</maven-antrun-plugin.version>
     <maven-assembly-plugin.version>2.4</maven-assembly-plugin.version>
-    <maven-dependency-plugin.version>2.8</maven-dependency-plugin.version>
+    <maven-dependency-plugin.version>2.10</maven-dependency-plugin.version>
     <maven-enforcer-plugin.version>1.4.1</maven-enforcer-plugin.version>
     <maven-javadoc-plugin.version>2.9.1</maven-javadoc-plugin.version>
     <maven-gpg-plugin.version>1.5</maven-gpg-plugin.version>
@@ -125,8 +125,8 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     <module>hadoop-mapreduce-project</module>
     <module>hadoop-tools</module>
     <module>hadoop-dist</module>
-    <module>hadoop-client</module>
     <module>hadoop-minicluster</module>
+    <module>hadoop-client-modules</module>
     <module>hadoop-build-tools</module>
   </modules>
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/29] hadoop git commit: MAPREDUCE-6821. Fix javac warning related to the deprecated APIs after upgrading Jackson. Contributed by Yiqin Lin.

Posted by as...@apache.org.
MAPREDUCE-6821. Fix javac warning related to the deprecated APIs after upgrading Jackson. Contributed by Yiqin Lin.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b0b033ea
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b0b033ea
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b0b033ea

Branch: refs/heads/YARN-5085
Commit: b0b033ea2e462356b8bbcf7790953ac09c712430
Parents: 754f15b
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Dec 13 14:20:02 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Dec 13 14:22:07 2016 +0900

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b033ea/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
index a842649..09edd94 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
@@ -72,7 +72,7 @@ import com.google.common.base.Charsets;
 class JobSubmitter {
   protected static final Log LOG = LogFactory.getLog(JobSubmitter.class);
   private static final ObjectReader READER =
-      new ObjectMapper().reader(Map.class);
+      new ObjectMapper().readerFor(Map.class);
   private static final String SHUFFLE_KEYGEN_ALGORITHM = "HmacSHA1";
   private static final int SHUFFLE_KEY_LENGTH = 64;
   private FileSystem jtFs;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/29] hadoop git commit: Revert YARN-4126. RM should not issue delegation tokens in unsecure mode.

Posted by as...@apache.org.
Revert YARN-4126. RM should not issue delegation tokens in unsecure mode.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ada876cd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ada876cd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ada876cd

Branch: refs/heads/YARN-5085
Commit: ada876cd1d22b61f237603cf339bbed65285dab8
Parents: fbdbbd5
Author: Jian He <ji...@apache.org>
Authored: Tue Dec 13 20:49:54 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Tue Dec 13 20:49:54 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/yarn/server/resourcemanager/ClientRMService.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ada876cd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index 1bc40c4..3dc7e38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -1238,7 +1238,7 @@ public class ClientRMService extends AbstractService implements
           .contains(UserGroupInformation.getCurrentUser()
                   .getRealAuthenticationMethod());
     } else {
-      return false;
+      return true;
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/29] hadoop git commit: HDFS-9911. TestDataNodeLifeline Fails intermittently. Contributed by Yiqun Lin

Posted by as...@apache.org.
HDFS-9911. TestDataNodeLifeline Fails intermittently. Contributed by Yiqun Lin


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a9563906
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a9563906
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a9563906

Branch: refs/heads/YARN-5085
Commit: a95639068c99ebcaefe8b6c4268449d12a6577d6
Parents: 2273a74
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Dec 16 09:46:21 2016 -0800
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Dec 16 09:46:21 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9563906/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index 25ab3b7..5294799 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -1068,7 +1068,7 @@ class BPServiceActor implements Runnable {
     volatile long nextHeartbeatTime = monotonicNow();
 
     @VisibleForTesting
-    volatile long nextLifelineTime = monotonicNow();
+    volatile long nextLifelineTime;
 
     @VisibleForTesting
     volatile long lastBlockReportTime = monotonicNow();
@@ -1091,6 +1091,7 @@ class BPServiceActor implements Runnable {
       this.heartbeatIntervalMs = heartbeatIntervalMs;
       this.lifelineIntervalMs = lifelineIntervalMs;
       this.blockReportIntervalMs = blockReportIntervalMs;
+      scheduleNextLifeline(nextHeartbeatTime);
     }
 
     // This is useful to make sure NN gets Heartbeat before Blockreport


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/29] hadoop git commit: HDFS-11164: Mover should avoid unnecessary retries if the block is pinned. Contributed by Rakesh R

Posted by as...@apache.org.
HDFS-11164: Mover should avoid unnecessary retries if the block is pinned. Contributed by Rakesh R


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e24a923d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e24a923d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e24a923d

Branch: refs/heads/YARN-5085
Commit: e24a923db50879f7dbe5d2afac0e6757089fb07d
Parents: 9947aeb
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Tue Dec 13 17:09:58 2016 -0800
Committer: Uma Maheswara Rao G <um...@intel.com>
Committed: Tue Dec 13 17:09:58 2016 -0800

----------------------------------------------------------------------
 .../datatransfer/BlockPinningException.java     |  33 ++++
 .../datatransfer/DataTransferProtoUtil.java     |  17 +-
 .../src/main/proto/datatransfer.proto           |   1 +
 .../hadoop/hdfs/server/balancer/Dispatcher.java |  62 ++++++-
 .../hdfs/server/datanode/DataXceiver.java       |   8 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |  26 ++-
 .../hdfs/server/datanode/DataNodeTestUtils.java |  28 ++++
 .../server/datanode/TestBlockReplacement.java   |  70 +++++++-
 .../hadoop/hdfs/server/mover/TestMover.java     | 163 ++++++++++++++++++-
 9 files changed, 395 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e24a923d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/BlockPinningException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/BlockPinningException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/BlockPinningException.java
new file mode 100644
index 0000000..c2f12f9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/BlockPinningException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.protocol.datatransfer;
+
+import java.io.IOException;
+
+/**
+ * Indicates a failure due to block pinning.
+ */
+public class BlockPinningException extends IOException {
+
+  // Required by {@link java.io.Serializable}.
+  private static final long serialVersionUID = 1L;
+
+  public BlockPinningException(String errMsg) {
+    super(errMsg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e24a923d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
index 6801149..287928c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
@@ -24,11 +24,11 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ChecksumTypeProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -107,6 +107,11 @@ public abstract class DataTransferProtoUtil {
   public static void checkBlockOpStatus(
           BlockOpResponseProto response,
           String logInfo) throws IOException {
+    checkBlockOpStatus(response, logInfo, false);
+  }
+
+  public static void checkBlockOpStatus(BlockOpResponseProto response,
+      String logInfo, boolean checkBlockPinningErr) throws IOException {
     if (response.getStatus() != Status.SUCCESS) {
       if (response.getStatus() == Status.ERROR_ACCESS_TOKEN) {
         throw new InvalidBlockTokenException(
@@ -114,6 +119,14 @@ public abstract class DataTransferProtoUtil {
           + ", status message " + response.getMessage()
           + ", " + logInfo
         );
+      } else if (checkBlockPinningErr
+          && response.getStatus() == Status.ERROR_BLOCK_PINNED) {
+        throw new BlockPinningException(
+            "Got error"
+            + ", status=" + response.getStatus().name()
+            + ", status message " + response.getMessage()
+            + ", " + logInfo
+          );
       } else {
         throw new IOException(
           "Got error"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e24a923d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto
index 290b158..889361a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto
@@ -243,6 +243,7 @@ enum Status {
   OOB_RESERVED2 = 10;         // Reserved
   OOB_RESERVED3 = 11;         // Reserved
   IN_PROGRESS = 12;
+  ERROR_BLOCK_PINNED = 13;
 }
 
 enum ShortCircuitFdResponse {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e24a923d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index eb3ed87..0e62da2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -36,6 +36,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
@@ -55,6 +56,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockPinningException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -224,6 +226,10 @@ public class Dispatcher {
       this.target = target;
     }
 
+    public DatanodeInfo getSource() {
+      return source.getDatanodeInfo();
+    }
+
     @Override
     public String toString() {
       final Block b = reportedBlock != null ? reportedBlock.getBlock() : null;
@@ -367,6 +373,15 @@ public class Dispatcher {
       } catch (IOException e) {
         LOG.warn("Failed to move " + this, e);
         target.getDDatanode().setHasFailure();
+        // Check that the failure is due to block pinning errors.
+        if (e instanceof BlockPinningException) {
+          // Pinned block can't be moved. Add this block into failure list.
+          // Later in the next iteration mover will exclude these blocks from
+          // pending moves.
+          target.getDDatanode().addBlockPinningFailures(this);
+          return;
+        }
+
         // Proxy or target may have some issues, delay before using these nodes
         // further in order to avoid a potential storm of "threads quota
         // exceeded" warnings when the dispatcher gets out of sync with work
@@ -419,7 +434,7 @@ public class Dispatcher {
         }
       }
       String logInfo = "reportedBlock move is failed";
-      DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
+      DataTransferProtoUtil.checkBlockOpStatus(response, logInfo, true);
     }
 
     /** reset the object */
@@ -600,6 +615,7 @@ public class Dispatcher {
     /** blocks being moved but not confirmed yet */
     private final List<PendingMove> pendings;
     private volatile boolean hasFailure = false;
+    private Map<Long, Set<DatanodeInfo>> blockPinningFailures = new HashMap<>();
     private volatile boolean hasSuccess = false;
     private ExecutorService moveExecutor;
 
@@ -685,6 +701,22 @@ public class Dispatcher {
       this.hasFailure = true;
     }
 
+    void addBlockPinningFailures(PendingMove pendingBlock) {
+      synchronized (blockPinningFailures) {
+        long blockId = pendingBlock.reportedBlock.getBlock().getBlockId();
+        Set<DatanodeInfo> pinnedLocations = blockPinningFailures.get(blockId);
+        if (pinnedLocations == null) {
+          pinnedLocations = new HashSet<>();
+          blockPinningFailures.put(blockId, pinnedLocations);
+        }
+        pinnedLocations.add(pendingBlock.getSource());
+      }
+    }
+
+    Map<Long, Set<DatanodeInfo>> getBlockPinningFailureList() {
+      return blockPinningFailures;
+    }
+
     void setHasSuccess() {
       this.hasSuccess = true;
     }
@@ -1155,6 +1187,34 @@ public class Dispatcher {
   }
 
   /**
+   * Check any of the block movements are failed due to block pinning errors. If
+   * yes, add the failed blockId and its respective source node location to the
+   * excluded list.
+   */
+  public static void checkForBlockPinningFailures(
+      Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks,
+      Iterable<? extends StorageGroup> targets) {
+    for (StorageGroup t : targets) {
+      Map<Long, Set<DatanodeInfo>> blockPinningFailureList = t.getDDatanode()
+          .getBlockPinningFailureList();
+      Set<Entry<Long, Set<DatanodeInfo>>> entrySet = blockPinningFailureList
+          .entrySet();
+      for (Entry<Long, Set<DatanodeInfo>> entry : entrySet) {
+        Long blockId = entry.getKey();
+        Set<DatanodeInfo> locs = excludedPinnedBlocks.get(blockId);
+        if (locs == null) {
+          // blockId doesn't exists in the excluded list.
+          locs = entry.getValue();
+          excludedPinnedBlocks.put(blockId, locs);
+        } else {
+          // blockId already exists in the excluded list, add the pinned node.
+          locs.addAll(entry.getValue());
+        }
+      }
+    }
+  }
+
+  /**
    * @return true if some moves are success.
    */
   public static boolean checkForSuccess(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e24a923d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index fee16b3..a35a5b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.StripedBlockInfo;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockPinningException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
@@ -1022,7 +1023,7 @@ class DataXceiver extends Receiver implements Runnable {
       String msg = "Not able to copy block " + block.getBlockId() + " " +
           "to " + peer.getRemoteAddressString() + " because it's pinned ";
       LOG.info(msg);
-      sendResponse(ERROR, msg);
+      sendResponse(Status.ERROR_BLOCK_PINNED, msg);
       return;
     }
     
@@ -1156,7 +1157,7 @@ class DataXceiver extends Receiver implements Runnable {
 
         String logInfo = "copy block " + block + " from "
             + proxySock.getRemoteSocketAddress();
-        DataTransferProtoUtil.checkBlockOpStatus(copyResponse, logInfo);
+        DataTransferProtoUtil.checkBlockOpStatus(copyResponse, logInfo, true);
 
         // get checksum info about the block we're copying
         ReadOpChecksumInfoProto checksumInfo = copyResponse.getReadOpChecksumInfo();
@@ -1183,6 +1184,9 @@ class DataXceiver extends Receiver implements Runnable {
       }
     } catch (IOException ioe) {
       opStatus = ERROR;
+      if (ioe instanceof BlockPinningException) {
+        opStatus = Status.ERROR_BLOCK_PINNED;
+      }
       errMsg = "opReplaceBlock " + block + " received exception " + ioe; 
       LOG.info(errMsg);
       if (!IoeDuringCopyBlockOperation) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e24a923d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 4ab55d3..bc75f0f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.SecurityUtil;
@@ -117,10 +116,12 @@ public class Mover {
   private final List<Path> targetPaths;
   private final int retryMaxAttempts;
   private final AtomicInteger retryCount;
+  private final Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks;
 
   private final BlockStoragePolicy[] blockStoragePolicies;
 
-  Mover(NameNodeConnector nnc, Configuration conf, AtomicInteger retryCount) {
+  Mover(NameNodeConnector nnc, Configuration conf, AtomicInteger retryCount,
+      Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks) {
     final long movedWinWidth = conf.getLong(
         DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY,
         DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_DEFAULT);
@@ -144,6 +145,7 @@ public class Mover {
     this.targetPaths = nnc.getTargetPaths();
     this.blockStoragePolicies = new BlockStoragePolicy[1 <<
         BlockStoragePolicySuite.ID_BIT_LENGTH];
+    this.excludedPinnedBlocks = excludedPinnedBlocks;
   }
 
   void init() throws IOException {
@@ -292,6 +294,8 @@ public class Mover {
       // wait for pending move to finish and retry the failed migration
       boolean hasFailed = Dispatcher.waitForMoveCompletion(storages.targets
           .values());
+      Dispatcher.checkForBlockPinningFailures(excludedPinnedBlocks,
+          storages.targets.values());
       boolean hasSuccess = Dispatcher.checkForSuccess(storages.targets
           .values());
       if (hasFailed && !hasSuccess) {
@@ -461,6 +465,19 @@ public class Mover {
         return true;
       }
 
+      // Check the given block is pinned in the source datanode. A pinned block
+      // can't be moved to a different datanode. So we can skip adding these
+      // blocks to different nodes.
+      long blockId = db.getBlock().getBlockId();
+      if (excludedPinnedBlocks.containsKey(blockId)) {
+        Set<DatanodeInfo> locs = excludedPinnedBlocks.get(blockId);
+        for (DatanodeInfo dn : locs) {
+          if (source.getDatanodeInfo().equals(dn)) {
+            return false;
+          }
+        }
+      }
+
       if (dispatcher.getCluster().isNodeGroupAware()) {
         if (chooseTarget(db, source, targetTypes, Matcher.SAME_NODE_GROUP)) {
           return true;
@@ -614,6 +631,8 @@ public class Mover {
             DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT,
             TimeUnit.SECONDS) * 1000;
     AtomicInteger retryCount = new AtomicInteger(0);
+    // TODO: Need to limit the size of the pinned blocks to limit memory usage
+    Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks = new HashMap<>();
     LOG.info("namenodes = " + namenodes);
 
     checkKeytabAndInit(conf);
@@ -628,7 +647,8 @@ public class Mover {
         Iterator<NameNodeConnector> iter = connectors.iterator();
         while (iter.hasNext()) {
           NameNodeConnector nnc = iter.next();
-          final Mover m = new Mover(nnc, conf, retryCount);
+          final Mover m = new Mover(nnc, conf, retryCount,
+              excludedPinnedBlocks);
           final ExitStatus r = m.run();
 
           if (r == ExitStatus.SUCCESS) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e24a923d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
index e2755f9..3501ed3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
@@ -25,10 +25,17 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
 
 /**
  * Utility class for accessing package-private DataNode information during tests.
@@ -175,4 +182,25 @@ public class DataNodeTestUtils {
       dn.getDirectoryScanner().reconcile();
     }
   }
+
+  /**
+   * This method is used to mock the data node block pinning API.
+   *
+   * @param dn datanode
+   * @param pinned true if the block is pinned, false otherwise
+   * @throws IOException
+   */
+  public static void mockDatanodeBlkPinning(final DataNode dn,
+      final boolean pinned) throws IOException {
+    final FsDatasetSpi<? extends FsVolumeSpi> data = dn.data;
+    dn.data = Mockito.spy(data);
+
+    doAnswer(new Answer<Object>() {
+      public Object answer(InvocationOnMock invocation) throws IOException {
+        // Bypass the argument to FsDatasetImpl#getPinning to show that
+        // the block is pinned.
+        return pinned;
+      }
+    }).when(dn.data).getPinning(any(ExtendedBlock.class));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e24a923d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
index 597dc46..f811bd8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
@@ -208,6 +208,67 @@ public class TestBlockReplacement {
     }
   }
 
+  /**
+   * Test to verify that the copying of pinned block to a different destination
+   * datanode will throw IOException with error code Status.ERROR_BLOCK_PINNED.
+   *
+   */
+  @Test(timeout = 90000)
+  public void testBlockReplacementWithPinnedBlocks() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+
+    // create only one datanode in the cluster with DISK and ARCHIVE storage
+    // types.
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+        .storageTypes(
+            new StorageType[] {StorageType.DISK, StorageType.ARCHIVE})
+        .build();
+
+    try {
+      cluster.waitActive();
+
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      String fileName = "/testBlockReplacementWithPinnedBlocks/file";
+      final Path file = new Path(fileName);
+      DFSTestUtil.createFile(dfs, file, 1024, (short) 1, 1024);
+
+      LocatedBlock lb = dfs.getClient().getLocatedBlocks(fileName, 0).get(0);
+      DatanodeInfo[] oldNodes = lb.getLocations();
+      assertEquals("Wrong block locations", oldNodes.length, 1);
+      DatanodeInfo source = oldNodes[0];
+      ExtendedBlock b = lb.getBlock();
+
+      DatanodeInfo[] datanodes = dfs.getDataNodeStats();
+      DatanodeInfo destin = null;
+      for (DatanodeInfo datanodeInfo : datanodes) {
+        // choose different destination node
+        if (!oldNodes[0].equals(datanodeInfo)) {
+          destin = datanodeInfo;
+          break;
+        }
+      }
+
+      assertNotNull("Failed to choose destination datanode!", destin);
+
+      assertFalse("Source and destin datanode should be different",
+          source.equals(destin));
+
+      // Mock FsDatasetSpi#getPinning to show that the block is pinned.
+      for (int i = 0; i < cluster.getDataNodes().size(); i++) {
+        DataNode dn = cluster.getDataNodes().get(i);
+        LOG.info("Simulate block pinning in datanode " + dn);
+        DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
+      }
+
+      // Block movement to a different datanode should fail as the block is
+      // pinned.
+      assertTrue("Status code mismatches!", replaceBlock(b, source, source,
+          destin, StorageType.ARCHIVE, Status.ERROR_BLOCK_PINNED));
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   @Test
   public void testBlockMoveAcrossStorageInSameNode() throws Exception {
     final Configuration conf = new HdfsConfiguration();
@@ -236,7 +297,7 @@ public class TestBlockReplacement {
       // move block to ARCHIVE by using same DataNodeInfo for source, proxy and
       // destination so that movement happens within datanode 
       assertTrue(replaceBlock(block, source, source, source,
-          StorageType.ARCHIVE));
+          StorageType.ARCHIVE, Status.SUCCESS));
       
       // wait till namenode notified
       Thread.sleep(3000);
@@ -311,7 +372,7 @@ public class TestBlockReplacement {
   private boolean replaceBlock( ExtendedBlock block, DatanodeInfo source,
       DatanodeInfo sourceProxy, DatanodeInfo destination) throws IOException {
     return replaceBlock(block, source, sourceProxy, destination,
-        StorageType.DEFAULT);
+        StorageType.DEFAULT, Status.SUCCESS);
   }
 
   /*
@@ -322,7 +383,8 @@ public class TestBlockReplacement {
       DatanodeInfo source,
       DatanodeInfo sourceProxy,
       DatanodeInfo destination,
-      StorageType targetStorageType) throws IOException, SocketException {
+      StorageType targetStorageType,
+      Status opStatus) throws IOException, SocketException {
     Socket sock = new Socket();
     try {
       sock.connect(NetUtils.createSocketAddr(destination.getXferAddr()),
@@ -342,7 +404,7 @@ public class TestBlockReplacement {
       while (proto.getStatus() == Status.IN_PROGRESS) {
         proto = BlockOpResponseProto.parseDelimitedFrom(reply);
       }
-      return proto.getStatus() == Status.SUCCESS;
+      return proto.getStatus() == opStatus;
     } finally {
       sock.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e24a923d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index 20a6959..d565548 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@ -37,11 +37,13 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSF
 
 import java.io.File;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -64,6 +66,7 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -72,6 +75,8 @@ import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DBlock;
 import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.mover.Mover.MLocation;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
@@ -121,7 +126,7 @@ public class TestMover {
     final List<NameNodeConnector> nncs = NameNodeConnector.newNameNodeConnectors(
         nnMap, Mover.class.getSimpleName(), Mover.MOVER_ID_PATH, conf,
         NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
-    return new Mover(nncs.get(0), conf, new AtomicInteger(0));
+    return new Mover(nncs.get(0), conf, new AtomicInteger(0), new HashMap<>());
   }
 
   @Test
@@ -705,4 +710,160 @@ public class TestMover {
       UserGroupInformation.setConfiguration(new Configuration());
     }
   }
+
+  /**
+   * Test to verify that mover can't move pinned blocks.
+   */
+  @Test(timeout = 90000)
+  public void testMoverWithPinnedBlocks() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    initConf(conf);
+
+    // Sets bigger retry max attempts value so that test case will timed out if
+    // block pinning errors are not handled properly during block movement.
+    conf.setInt(DFSConfigKeys.DFS_MOVER_RETRY_MAX_ATTEMPTS_KEY, 10000);
+
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3)
+        .build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final String file = "/testMoverWithPinnedBlocks/file";
+      Path dir = new Path("/testMoverWithPinnedBlocks");
+      dfs.mkdirs(dir);
+
+      // write to DISK
+      dfs.setStoragePolicy(dir, "HOT");
+      final FSDataOutputStream out = dfs.create(new Path(file));
+      byte[] fileData = StripedFileTestUtil
+          .generateBytes(DEFAULT_BLOCK_SIZE * 3);
+      out.write(fileData);
+      out.close();
+
+      // verify before movement
+      LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+      StorageType[] storageTypes = lb.getStorageTypes();
+      for (StorageType storageType : storageTypes) {
+        Assert.assertTrue(StorageType.DISK == storageType);
+      }
+
+      // Adding one SSD based data node to the cluster.
+      StorageType[][] newtypes = new StorageType[][] {{StorageType.SSD}};
+      startAdditionalDNs(conf, 1, newtypes, cluster);
+
+      // Mock FsDatasetSpi#getPinning to show that the block is pinned.
+      for (int i = 0; i < cluster.getDataNodes().size(); i++) {
+        DataNode dn = cluster.getDataNodes().get(i);
+        LOG.info("Simulate block pinning in datanode {}", dn);
+        DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
+      }
+
+      // move file blocks to ONE_SSD policy
+      dfs.setStoragePolicy(dir, "ONE_SSD");
+      int rc = ToolRunner.run(conf, new Mover.Cli(),
+          new String[] {"-p", dir.toString()});
+
+      int exitcode = ExitStatus.NO_MOVE_BLOCK.getExitCode();
+      Assert.assertEquals("Movement should fail", exitcode, rc);
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test to verify that mover should work well with pinned blocks as well as
+   * failed blocks. Mover should continue retrying the failed blocks only.
+   */
+  @Test(timeout = 90000)
+  public void testMoverFailedRetryWithPinnedBlocks() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    initConf(conf);
+    conf.set(DFSConfigKeys.DFS_MOVER_RETRY_MAX_ATTEMPTS_KEY, "2");
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(2)
+        .storageTypes(
+            new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE}}).build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final String parenDir = "/parent";
+      dfs.mkdirs(new Path(parenDir));
+      final String file1 = "/parent/testMoverFailedRetryWithPinnedBlocks1";
+      // write to DISK
+      final FSDataOutputStream out = dfs.create(new Path(file1), (short) 2);
+      byte[] fileData = StripedFileTestUtil
+          .generateBytes(DEFAULT_BLOCK_SIZE * 2);
+      out.write(fileData);
+      out.close();
+
+      // Adding pinned blocks.
+      createFileWithFavoredDatanodes(conf, cluster, dfs);
+
+      // Delete block file so, block move will fail with FileNotFoundException
+      LocatedBlocks locatedBlocks = dfs.getClient().getLocatedBlocks(file1, 0);
+      Assert.assertEquals("Wrong block count", 2,
+          locatedBlocks.locatedBlockCount());
+      LocatedBlock lb = locatedBlocks.get(0);
+      cluster.corruptBlockOnDataNodesByDeletingBlockFile(lb.getBlock());
+
+      // move to ARCHIVE
+      dfs.setStoragePolicy(new Path(parenDir), "COLD");
+      int rc = ToolRunner.run(conf, new Mover.Cli(),
+          new String[] {"-p", parenDir.toString()});
+      Assert.assertEquals("Movement should fail after some retry",
+          ExitStatus.NO_MOVE_PROGRESS.getExitCode(), rc);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private void createFileWithFavoredDatanodes(final Configuration conf,
+      final MiniDFSCluster cluster, final DistributedFileSystem dfs)
+          throws IOException {
+    // Adding two DISK based data node to the cluster.
+    // Also, ensure that blocks are pinned in these new data nodes.
+    StorageType[][] newtypes =
+        new StorageType[][] {{StorageType.DISK}, {StorageType.DISK}};
+    startAdditionalDNs(conf, 2, newtypes, cluster);
+    ArrayList<DataNode> dataNodes = cluster.getDataNodes();
+    InetSocketAddress[] favoredNodes = new InetSocketAddress[2];
+    int j = 0;
+    for (int i = dataNodes.size() - 1; i >= 2; i--) {
+      favoredNodes[j++] = dataNodes.get(i).getXferAddress();
+    }
+    final String file = "/parent/testMoverFailedRetryWithPinnedBlocks2";
+    final FSDataOutputStream out = dfs.create(new Path(file),
+        FsPermission.getDefault(), true, DEFAULT_BLOCK_SIZE, (short) 2,
+        DEFAULT_BLOCK_SIZE, null, favoredNodes);
+    byte[] fileData = StripedFileTestUtil.generateBytes(DEFAULT_BLOCK_SIZE * 2);
+    out.write(fileData);
+    out.close();
+
+    // Mock FsDatasetSpi#getPinning to show that the block is pinned.
+    LocatedBlocks locatedBlocks = dfs.getClient().getLocatedBlocks(file, 0);
+    Assert.assertEquals("Wrong block count", 2,
+        locatedBlocks.locatedBlockCount());
+    LocatedBlock lb = locatedBlocks.get(0);
+    DatanodeInfo datanodeInfo = lb.getLocations()[0];
+    for (DataNode dn : cluster.getDataNodes()) {
+      if (dn.getDatanodeId().getDatanodeUuid()
+          .equals(datanodeInfo.getDatanodeUuid())) {
+        LOG.info("Simulate block pinning in datanode {}", datanodeInfo);
+        DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
+        break;
+      }
+    }
+  }
+
+  private void startAdditionalDNs(final Configuration conf,
+      int newNodesRequired, StorageType[][] newTypes,
+      final MiniDFSCluster cluster) throws IOException {
+
+    cluster.startDataNodes(conf, newNodesRequired, newTypes, true, null, null,
+        null, null, null, false, false, false, null);
+    cluster.triggerHeartbeats();
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org