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 xg...@apache.org on 2016/12/05 18:47:19 UTC

[01/29] hadoop git commit: MAPREDUCE-6565. Configuration to use host name in delegation token service is not read from job.xml during MapReduce job execution. Contributed by Li Lu.

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5734 6d8b4f6c2 -> 291df5c7f


MAPREDUCE-6565. Configuration to use host name in delegation token service is not read from job.xml during MapReduce job execution. Contributed by Li Lu.


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

Branch: refs/heads/YARN-5734
Commit: 8f6e14399a3e77e1bdcc5034f7601e9f62163dea
Parents: 6d8b4f6
Author: Junping Du <ju...@apache.org>
Authored: Tue Nov 29 15:51:27 2016 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Tue Nov 29 15:51:27 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/mapred/YarnChild.java         | 2 ++
 .../main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java | 3 +++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f6e1439/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
index 164f19d..97642a5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
@@ -78,6 +78,8 @@ class YarnChild {
     // Initing with our JobConf allows us to avoid loading confs twice
     Limits.init(job);
     UserGroupInformation.setConfiguration(job);
+    // MAPREDUCE-6565: need to set configuration for SecurityUtil.
+    SecurityUtil.setConfiguration(job);
 
     String host = args[0];
     int port = Integer.parseInt(args[1]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f6e1439/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index 4a8a90e..b383a02 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -123,6 +123,7 @@ import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.service.AbstractService;
@@ -1690,6 +1691,8 @@ public class MRAppMaster extends CompositeService {
       final JobConf conf, String jobUserName) throws IOException,
       InterruptedException {
     UserGroupInformation.setConfiguration(conf);
+    // MAPREDUCE-6565: need to set configuration for SecurityUtil.
+    SecurityUtil.setConfiguration(conf);
     // Security framework already loaded the tokens into current UGI, just use
     // them
     Credentials credentials =


---------------------------------------------------------------------
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: YARN-5746. The state of the parentQueue and its childQueues should be synchronized. Contributed by Xuan Gong

Posted by xg...@apache.org.
YARN-5746. The state of the parentQueue and its childQueues should be synchronized. Contributed by Xuan Gong


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

Branch: refs/heads/YARN-5734
Commit: f885160f4ac56a0999e3b051eb7bccce928c1c33
Parents: 4113ec5
Author: Jian He <ji...@apache.org>
Authored: Fri Dec 2 16:17:31 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Fri Dec 2 16:17:31 2016 -0800

----------------------------------------------------------------------
 .../scheduler/capacity/AbstractCSQueue.java     | 26 +++++-
 .../CapacitySchedulerConfiguration.java         | 22 ++++-
 .../scheduler/capacity/TestQueueState.java      | 96 ++++++++++++++++++++
 3 files changed, 139 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f885160f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.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/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index 3daabaf..dd2f0d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -291,7 +291,8 @@ public abstract class AbstractCSQueue implements CSQueue {
 
       authorizer = YarnAuthorizationProvider.getInstance(csContext.getConf());
 
-      this.state = csContext.getConfiguration().getState(getQueuePath());
+      initializeQueueState();
+
       this.acls = csContext.getConfiguration().getAcls(getQueuePath());
 
       // Update metrics
@@ -330,6 +331,29 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
   }
 
+  private void initializeQueueState() {
+    // inherit from parent if state not set, only do this when we are not root
+    if (parent != null) {
+      QueueState configuredState = csContext.getConfiguration()
+          .getConfiguredState(getQueuePath());
+      QueueState parentState = parent.getState();
+      if (configuredState == null) {
+        this.state = parentState;
+      } else if (configuredState == QueueState.RUNNING
+          && parentState == QueueState.STOPPED) {
+        throw new IllegalArgumentException(
+            "The parent queue:" + parent.getQueueName() + " state is STOPPED, "
+            + "child queue:" + queueName + " state cannot be RUNNING.");
+      } else {
+        this.state = configuredState;
+      }
+    } else {
+      // if this is the root queue, get the state from the configuration.
+      // if the state is not set, use RUNNING as default state.
+      this.state = csContext.getConfiguration().getState(getQueuePath());
+    }
+  }
+
   protected QueueInfo getQueueInfo() {
     // Deliberately doesn't use lock here, because this method will be invoked
     // from schedulerApplicationAttempt, to avoid deadlock, sacrifice

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f885160f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
index f8335a8..bfaeba4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
@@ -448,12 +448,26 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     setFloat(getQueuePrefix(queue) + USER_LIMIT_FACTOR, userLimitFactor); 
   }
   
-  public QueueState getState(String queue) {
+  public QueueState getConfiguredState(String queue) {
     String state = get(getQueuePrefix(queue) + STATE);
-    return (state != null) ? 
-        QueueState.valueOf(StringUtils.toUpperCase(state)) : QueueState.RUNNING;
+    if (state == null) {
+      return null;
+    } else {
+      return QueueState.valueOf(StringUtils.toUpperCase(state));
+    }
   }
-  
+
+  public QueueState getState(String queue) {
+    QueueState state = getConfiguredState(queue);
+    return (state == null) ? QueueState.RUNNING : state;
+  }
+
+  @Private
+  @VisibleForTesting
+  public void setState(String queue, QueueState state) {
+    set(getQueuePrefix(queue) + STATE, state.name());
+  }
+
   public void setAccessibleNodeLabels(String queue, Set<String> labels) {
     if (labels == null) {
       return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f885160f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueState.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/scheduler/capacity/TestQueueState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueState.java
new file mode 100644
index 0000000..bd878b7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueState.java
@@ -0,0 +1,96 @@
+/**
+ * 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.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.io.IOException;
+import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test Queue States.
+ */
+public class TestQueueState {
+
+  private static final String Q1 = "q1";
+  private static final String Q2 = "q2";
+
+  private final static String Q1_PATH =
+      CapacitySchedulerConfiguration.ROOT + "." + Q1;
+  private final static String Q2_PATH =
+      Q1_PATH + "." + Q2;
+  private CapacityScheduler cs;
+  private YarnConfiguration conf;
+
+  @Test (timeout = 15000)
+  public void testQueueState() throws IOException {
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {Q1});
+    csConf.setQueues(Q1_PATH, new String[] {Q2});
+
+    csConf.setCapacity(Q1_PATH, 100);
+    csConf.setCapacity(Q2_PATH, 100);
+
+    conf = new YarnConfiguration(csConf);
+    cs = new CapacityScheduler();
+
+    RMContext rmContext = TestUtils.getMockRMContext();
+    cs.setConf(conf);
+    cs.setRMContext(rmContext);
+    cs.init(conf);
+
+    //by default, the state of both queues should be RUNNING
+    Assert.assertEquals(QueueState.RUNNING, cs.getQueue(Q1).getState());
+    Assert.assertEquals(QueueState.RUNNING, cs.getQueue(Q2).getState());
+
+    // Change the state of Q1 to STOPPED, and re-initiate the CS
+    csConf.setState(Q1_PATH, QueueState.STOPPED);
+    conf = new YarnConfiguration(csConf);
+    cs.reinitialize(conf, rmContext);
+    // The state of Q1 and its child: Q2 should be STOPPED
+    Assert.assertEquals(QueueState.STOPPED, cs.getQueue(Q1).getState());
+    Assert.assertEquals(QueueState.STOPPED, cs.getQueue(Q2).getState());
+
+    // Change the state of Q1 to RUNNING, and change the state of Q2 to STOPPED
+    csConf.setState(Q1_PATH, QueueState.RUNNING);
+    csConf.setState(Q2_PATH, QueueState.STOPPED);
+    conf = new YarnConfiguration(csConf);
+    // reinitialize the CS, the operation should be successful
+    cs.reinitialize(conf, rmContext);
+    Assert.assertEquals(QueueState.RUNNING, cs.getQueue(Q1).getState());
+    Assert.assertEquals(QueueState.STOPPED, cs.getQueue(Q2).getState());
+
+    // Change the state of Q1 to STOPPED, and change the state of Q2 to RUNNING
+    csConf.setState(Q1_PATH, QueueState.STOPPED);
+    csConf.setState(Q2_PATH, QueueState.RUNNING);
+    conf = new YarnConfiguration(csConf);
+    // reinitialize the CS, the operation should be failed.
+    try {
+      cs.reinitialize(conf, rmContext);
+      Assert.fail("Should throw an Exception.");
+    } catch (Exception ex) {
+      Assert.assertTrue(ex.getCause().getMessage().contains(
+          "The parent queue:q1 state is STOPPED, "
+          + "child queue:q2 state cannot be RUNNING."));
+    }
+  }
+}


---------------------------------------------------------------------
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: MAPREDUCE-6810. Fix hadoop-mapreduce-client-nativetask compilation with GCC-6.2.1. Contributed by Ravi Prakash.

Posted by xg...@apache.org.
MAPREDUCE-6810. Fix hadoop-mapreduce-client-nativetask compilation with GCC-6.2.1. Contributed by Ravi Prakash.


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

Branch: refs/heads/YARN-5734
Commit: 7c848719de778929258f1f9e2778e56f267c90ed
Parents: b3befc0
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Wed Nov 30 10:47:41 2016 -0800
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Wed Nov 30 10:47:41 2016 -0800

----------------------------------------------------------------------
 .../src/main/native/src/lib/Log.h                                  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c848719/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Log.h
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Log.h b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Log.h
index a0c17f3..a84b055 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Log.h
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Log.h
@@ -32,7 +32,7 @@ extern FILE * LOG_DEVICE;
 #define LOG(_fmt_, args...)   if (LOG_DEVICE) { \
     time_t log_timer; struct tm log_tm; \
     time(&log_timer); localtime_r(&log_timer, &log_tm); \
-    fprintf(LOG_DEVICE, "%02d/%02d/%02d %02d:%02d:%02d INFO "_fmt_"\n", \
+    fprintf(LOG_DEVICE, "%02d/%02d/%02d %02d:%02d:%02d INFO " _fmt_ "\n", \
     log_tm.tm_year%100, log_tm.tm_mon+1, log_tm.tm_mday, \
     log_tm.tm_hour, log_tm.tm_min, log_tm.tm_sec, \
     ##args);}


---------------------------------------------------------------------
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-11156. Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API. Contributed by Weiwei Yang

Posted by xg...@apache.org.
HDFS-11156. Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API. Contributed by Weiwei Yang


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

Branch: refs/heads/YARN-5734
Commit: c7ff34f8dcca3a2024230c5383abd9299daa1b20
Parents: 0cfd7ad
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Dec 2 11:10:09 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Fri Dec 2 11:10:13 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  | 32 ++++++++++++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 13 +++--
 .../hadoop/hdfs/web/resources/GetOpParam.java   | 12 ++++-
 .../web/resources/NamenodeWebHdfsMethods.java   | 17 +++++++
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    | 30 ++++++++++++
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java | 51 ++++++++++++++++++++
 6 files changed, 151 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/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 a75f4f1..12899f4 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
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary.Builder;
 import org.apache.hadoop.fs.FileChecksum;
@@ -588,4 +589,35 @@ class JsonUtilClient {
         lastLocatedBlock, isLastBlockComplete, null, null);
   }
 
+  /** Convert a Json map to BlockLocation. **/
+  static BlockLocation toBlockLocation(Map<String, Object> m)
+      throws IOException{
+    long length = ((Number) m.get("length")).longValue();
+    long offset = ((Number) m.get("offset")).longValue();
+    boolean corrupt = Boolean.
+        getBoolean(m.get("corrupt").toString());
+    String[] storageIds = toStringArray(getList(m, "storageIds"));
+    String[] cachedHosts = toStringArray(getList(m, "cachedHosts"));
+    String[] hosts = toStringArray(getList(m, "hosts"));
+    String[] names = toStringArray(getList(m, "names"));
+    String[] topologyPaths = toStringArray(getList(m, "topologyPaths"));
+    StorageType[] storageTypes = toStorageTypeArray(
+        getList(m, "storageTypes"));
+    return new BlockLocation(names, hosts, cachedHosts,
+        topologyPaths, storageIds, storageTypes,
+        offset, length, corrupt);
+  }
+
+  static String[] toStringArray(List<?> list) {
+    if (list == null) {
+      return null;
+    } else {
+      final String[] array = new String[list.size()];
+      int i = 0;
+      for (Object object : list) {
+        array[i++] = object.toString();
+      }
+      return array;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/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 23804b7..e82e9f6 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
@@ -1610,13 +1610,20 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.GET_FILE_BLOCK_LOCATIONS);
 
-    final HttpOpParam.Op op = GetOpParam.Op.GET_BLOCK_LOCATIONS;
+    final HttpOpParam.Op op = GetOpParam.Op.GETFILEBLOCKLOCATIONS;
     return new FsPathResponseRunner<BlockLocation[]>(op, p,
         new OffsetParam(offset), new LengthParam(length)) {
       @Override
+      @SuppressWarnings("unchecked")
       BlockLocation[] decodeResponse(Map<?,?> json) throws IOException {
-        return DFSUtilClient.locatedBlocks2Locations(
-            JsonUtilClient.toLocatedBlocks(json));
+        List<?> list = JsonUtilClient.getList(json, "BlockLocations");
+        BlockLocation[] locations = new BlockLocation[list.size()];
+        for(int i=0; i<locations.length; i++) {
+          BlockLocation bl = JsonUtilClient.
+              toBlockLocation((Map<String, Object>) list.get(i));
+          locations[i] = bl;
+        }
+        return locations;
       }
     }.run();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
index 635e6d7..ccb0bb3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
@@ -33,8 +33,18 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     GETHOMEDIRECTORY(false, HttpURLConnection.HTTP_OK),
     GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
 
-    /** GET_BLOCK_LOCATIONS is a private unstable op. */
+    /**
+     * GET_BLOCK_LOCATIONS is a private/stable API op. It returns a
+     * {@link org.apache.hadoop.hdfs.protocol.LocatedBlocks}
+     * json object.
+     */
     GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
+    /**
+     * GETFILEBLOCKLOCATIONS is the public op that complies with
+     * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations}
+     * interface.
+     */
+    GETFILEBLOCKLOCATIONS(false, HttpURLConnection.HTTP_OK),
     GETACLSTATUS(false, HttpURLConnection.HTTP_OK),
     GETXATTRS(false, HttpURLConnection.HTTP_OK),
     GETTRASHROOT(false, HttpURLConnection.HTTP_OK),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 5d9b12a..107d4ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -54,6 +54,7 @@ import javax.ws.rs.core.StreamingOutput;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -975,6 +976,22 @@ public class NamenodeWebHdfsMethods {
         return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       }
     }
+    case GETFILEBLOCKLOCATIONS:
+    {
+      final long offsetValue = offset.getValue();
+      final Long lengthValue = length.getValue();
+
+      try (final FileSystem fs = FileSystem.get(conf != null ?
+          conf : new Configuration())) {
+        BlockLocation[] locations = fs.getFileBlockLocations(
+            new org.apache.hadoop.fs.Path(fullpath),
+            offsetValue,
+            lengthValue != null? lengthValue: Long.MAX_VALUE);
+        final String js = JsonUtil.toJsonString("BlockLocations",
+            JsonUtil.toJsonArray(locations));
+        return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+      }
+    }
     case GET_BLOCK_LOCATIONS:
     {
       final long offsetValue = offset.getValue();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index a0dadbd..affa861 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -436,4 +436,34 @@ public class JsonUtil {
     return MAPPER.writeValueAsString(obj);
   }
 
+  public static Object[] toJsonArray(BlockLocation[] locations)
+      throws IOException {
+    if(locations == null) {
+      return null;
+    }
+    Object[] blockLocations = new Object[locations.length];
+    for(int i=0; i<locations.length; i++) {
+      blockLocations[i] = toJsonMap(locations[i]);
+    }
+    return blockLocations;
+  }
+
+  public static Map<String, Object> toJsonMap(
+      final BlockLocation blockLocation) throws IOException {
+    if (blockLocation == null) {
+      return null;
+    }
+
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("length", blockLocation.getLength());
+    m.put("offset", blockLocation.getOffset());
+    m.put("corrupt", blockLocation.isCorrupt());
+    m.put("storageTypes", toJsonArray(blockLocation.getStorageTypes()));
+    m.put("storageIds", blockLocation.getStorageIds());
+    m.put("cachedHosts", blockLocation.getCachedHosts());
+    m.put("hosts", blockLocation.getHosts());
+    m.put("names", blockLocation.getNames());
+    m.put("topologyPaths", blockLocation.getTopologyPaths());
+    return m;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index 5386a45..82b708a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -37,6 +37,7 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
+import java.util.Map;
 import java.util.Random;
 
 import org.apache.commons.io.IOUtils;
@@ -88,6 +89,8 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.type.MapType;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Mockito.doReturn;
@@ -852,6 +855,46 @@ public class TestWebHDFS {
         Assert.assertTrue(storageTypes != null && storageTypes.length > 0 &&
             storageTypes[0] == StorageType.DISK);
       }
+
+      // Query webhdfs REST API to get block locations
+      InetSocketAddress addr = cluster.getNameNode().getHttpAddress();
+      URL url = new URL("http", addr.getHostString(), addr.getPort(),
+          WebHdfsFileSystem.PATH_PREFIX + "/foo?op=GETFILEBLOCKLOCATIONS");
+      LOG.info("Sending GETFILEBLOCKLOCATIONS request " + url);
+
+      String response = getResponse(url, "GET");
+      LOG.info("The output of GETFILEBLOCKLOCATIONS request " + response);
+      // Expected output from rest API
+      // { "BlockLoactions" : [{Block_Loation_Json}, ...] }
+      ObjectMapper mapper = new ObjectMapper();
+      MapType jsonType = mapper.getTypeFactory().constructMapType(
+          Map.class,
+          String.class,
+          BlockLocation[].class);
+      Map<String, BlockLocation[]> jsonMap = mapper.readValue(response,
+          jsonType);
+      BlockLocation[] array = jsonMap.get("BlockLocations");
+
+      for(int i=0; i<locations.length; i++) {
+        BlockLocation raw = locations[i];
+        BlockLocation rest = array[i];
+        Assert.assertEquals(raw.getLength(),
+            rest.getLength());
+        Assert.assertEquals(raw.getOffset(),
+            rest.getOffset());
+        Assert.assertArrayEquals(raw.getCachedHosts(),
+            rest.getCachedHosts());
+        Assert.assertArrayEquals(raw.getHosts(),
+            rest.getHosts());
+        Assert.assertArrayEquals(raw.getNames(),
+            rest.getNames());
+        Assert.assertArrayEquals(raw.getStorageIds(),
+            rest.getStorageIds());
+        Assert.assertArrayEquals(raw.getTopologyPaths(),
+            rest.getTopologyPaths());
+        Assert.assertArrayEquals(raw.getStorageTypes(),
+            rest.getStorageTypes());
+      }
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -859,6 +902,14 @@ public class TestWebHDFS {
     }
   }
 
+  private static String getResponse(URL url, String httpRequestType)
+      throws IOException {
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod(httpRequestType);
+    conn.setInstanceFollowRedirects(false);
+    return IOUtils.toString(conn.getInputStream());
+  }
+
   private WebHdfsFileSystem createWebHDFSAsTestUser(final Configuration conf,
       final URI uri, final String userName) throws Exception {
 


---------------------------------------------------------------------
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: YARN-5901. Fix race condition in TestGetGroups beforeclass setup() (Contributed by Haibo Chen via Daniel Templeton)

Posted by xg...@apache.org.
YARN-5901. Fix race condition in TestGetGroups beforeclass setup() (Contributed by Haibo Chen via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: 2d77dc727d9b5e56009bbc36643d85500efcbca5
Parents: 19f373a
Author: Daniel Templeton <te...@apache.org>
Authored: Thu Dec 1 15:57:39 2016 -0800
Committer: Daniel Templeton <te...@apache.org>
Committed: Thu Dec 1 15:57:39 2016 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/client/TestGetGroups.java       | 36 +++++++++++++-------
 1 file changed, 24 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d77dc72/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
index e947ece..da0258c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
@@ -20,16 +20,21 @@ package org.apache.hadoop.yarn.client;
 
 import java.io.IOException;
 import java.io.PrintStream;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
 import org.apache.hadoop.service.Service.STATE;
+import org.apache.hadoop.service.ServiceStateChangeListener;
 import org.apache.hadoop.tools.GetGroupsTestBase;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 
@@ -42,30 +47,37 @@ public class TestGetGroups extends GetGroupsTestBase {
   private static Configuration conf;
   
   @BeforeClass
-  public static void setUpResourceManager() throws IOException, InterruptedException {
+  public static void setUpResourceManager() throws InterruptedException {
     conf = new YarnConfiguration();
     resourceManager = new ResourceManager() {
       @Override
       protected void doSecureLogin() throws IOException {
       };
     };
+
+    // a reliable way to wait for resource manager to start
+    CountDownLatch rmStartedSignal = new CountDownLatch(1);
+    ServiceStateChangeListener rmStateChangeListener =
+        new ServiceStateChangeListener() {
+          @Override
+          public void stateChanged(Service service) {
+            if (service.getServiceState() == STATE.STARTED) {
+              rmStartedSignal.countDown();
+            }
+          }
+        };
+    resourceManager.registerServiceListener(rmStateChangeListener);
+
     resourceManager.init(conf);
     new Thread() {
       public void run() {
         resourceManager.start();
       };
     }.start();
-    int waitCount = 0;
-    while (resourceManager.getServiceState() == STATE.INITED
-        && waitCount++ < 10) {
-      LOG.info("Waiting for RM to start...");
-      Thread.sleep(1000);
-    }
-    if (resourceManager.getServiceState() != STATE.STARTED) {
-      throw new IOException(
-          "ResourceManager failed to start. Final state is "
-              + resourceManager.getServiceState());
-    }
+
+    boolean rmStarted = rmStartedSignal.await(60000L, TimeUnit.MILLISECONDS);
+    Assert.assertTrue("ResourceManager failed to start up.", rmStarted);
+
     LOG.info("ResourceManager RMAdmin address: " +
         conf.get(YarnConfiguration.RM_ADMIN_ADDRESS));
   }


---------------------------------------------------------------------
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: HDFS-11181. Fuse wrapper has a typo. Contributed by Wei-Chiu Chuang.

Posted by xg...@apache.org.
HDFS-11181. Fuse wrapper has a typo. Contributed by Wei-Chiu Chuang.


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

Branch: refs/heads/YARN-5734
Commit: c51bfd29cd1e6ec619742f2c47ebfc8bbfb231b6
Parents: f885160
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Mon Dec 5 08:44:40 2016 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Mon Dec 5 08:44:40 2016 -0800

----------------------------------------------------------------------
 .../src/main/native/fuse-dfs/fuse_dfs_wrapper.sh                   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c51bfd29/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh
index c52c5f9..d5bfd09 100755
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh
@@ -43,7 +43,7 @@ done < <(find "$HADOOP_HOME/hadoop-client" -name "*.jar" -print0)
 while IFS= read -r -d '' file
 do
   export CLASSPATH=$CLASSPATH:$file
-done < <(find "$HADOOP_HOME/hhadoop-hdfs-project" -name "*.jar" -print0)
+done < <(find "$HADOOP_HOME/hadoop-hdfs-project" -name "*.jar" -print0)
 
 export CLASSPATH=$HADOOP_CONF_DIR:$CLASSPATH
 export PATH=$FUSEDFS_PATH:$PATH


---------------------------------------------------------------------
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: MAPREDUCE-6787. Allow job_conf.xml to be downloadable on the job overview page in JHS (haibochen via rkanter)

Posted by xg...@apache.org.
MAPREDUCE-6787. Allow job_conf.xml to be downloadable on the job overview page in JHS (haibochen via rkanter)


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

Branch: refs/heads/YARN-5734
Commit: c87b3a448a00df97149a4e93a8c39d9ad0268bdb
Parents: 2d77dc7
Author: Robert Kanter <rk...@apache.org>
Authored: Thu Dec 1 17:29:16 2016 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Thu Dec 1 17:29:38 2016 -0800

----------------------------------------------------------------------
 .../mapreduce/v2/app/webapp/AppController.java  | 34 ++++++++++++++++++++
 .../mapreduce/v2/app/webapp/ConfBlock.java      |  2 +-
 .../v2/app/webapp/TestAppController.java        | 14 ++++++++
 .../hadoop/mapreduce/v2/hs/webapp/HsWebApp.java |  2 ++
 .../org/apache/hadoop/yarn/webapp/Router.java   | 23 ++++++++++---
 .../org/apache/hadoop/yarn/webapp/WebApp.java   | 13 ++++++++
 6 files changed, 83 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c87b3a44/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
index 305ec7e..e30e1b9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
@@ -324,6 +324,40 @@ public class AppController extends Controller implements AMParams {
   }
 
   /**
+   * Handle requests to download the job configuration.
+   */
+  public void downloadConf() {
+    try {
+      requireJob();
+    } catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
+    writeJobConf();
+  }
+
+  private void writeJobConf() {
+    String jobId = $(JOB_ID);
+    assert(!jobId.isEmpty());
+
+    JobId jobID = MRApps.toJobID($(JOB_ID));
+    Job job = app.context.getJob(jobID);
+    assert(job != null);
+
+    try {
+      Configuration jobConf = job.loadConfFile();
+      response().setContentType("text/xml");
+      response().setHeader("Content-Disposition",
+          "attachment; filename=" + jobId + ".xml");
+      jobConf.writeXml(writer());
+    } catch (IOException e) {
+      LOG.error("Error reading/writing job" +
+          " conf file for job: " + jobId, e);
+      renderText(e.getMessage());
+    }
+  }
+
+  /**
    * Render a BAD_REQUEST error.
    * @param s the error message to include.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c87b3a44/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
index 4cb79bf..532c2bd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
@@ -70,7 +70,7 @@ public class ConfBlock extends HtmlBlock {
     try {
       ConfInfo info = new ConfInfo(job);
 
-      html.div().h3(confPath.toString())._();
+      html.div().a("/jobhistory/downloadconf/" + jid, confPath.toString());
       TBODY<TABLE<Hamlet>> tbody = html.
         // Tasks table
       table("#conf").

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c87b3a44/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
index 92786e3..3f685b0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
@@ -59,6 +60,8 @@ public class TestAppController {
     Task task = mock(Task.class);
 
     when(job.getTask(any(TaskId.class))).thenReturn(task);
+    when(job.loadConfFile()).thenReturn(new Configuration());
+    when(job.getConfFile()).thenReturn(new Path("/"));
 
     JobId jobID = MRApps.toJobID("job_01_01");
     when(context.getJob(jobID)).thenReturn(job);
@@ -266,6 +269,17 @@ public class TestAppController {
   }
 
   /**
+   * Test downloadConf request handling.
+   */
+  @Test
+  public void testDownloadConfiguration() {
+    appController.downloadConf();
+    String jobConfXml = appController.getData();
+    assertTrue("Error downloading the job configuration file.",
+        !jobConfXml.contains("Error"));
+  }
+
+  /**
    *   Test method 'conf'. Should set AttemptsPage class for rendering or print information about error
    */
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c87b3a44/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java
index ebc6d46..d130910 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java
@@ -50,6 +50,8 @@ public class HsWebApp extends WebApp implements AMParams {
     route("/app", HsController.class);
     route(pajoin("/job", JOB_ID), HsController.class, "job");
     route(pajoin("/conf", JOB_ID), HsController.class, "conf");
+    routeWithoutDefaultView(pajoin("/downloadconf", JOB_ID),
+        HsController.class, "downloadConf");
     route(pajoin("/jobcounters", JOB_ID), HsController.class, "jobCounters");
     route(pajoin("/singlejobcounter",JOB_ID, COUNTER_GROUP, COUNTER_NAME),
         HsController.class, "singleJobCounter");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c87b3a44/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Router.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Router.java
index c46b50e..f2eca04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Router.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Router.java
@@ -74,17 +74,32 @@ class Router {
 
   final TreeMap<String, Dest> routes = Maps.newTreeMap(); // path->dest
 
+  synchronized Dest add(WebApp.HTTP httpMethod, String path,
+                        Class<? extends Controller> cls,
+                        String action, List<String> names){
+    return addWithOptionalDefaultView(
+        httpMethod, path, cls, action, names, true);
+  }
+
+  synchronized Dest addWithoutDefaultView(WebApp.HTTP httpMethod,
+      String path, Class<? extends Controller> cls, String action,
+      List<String> names){
+    return addWithOptionalDefaultView(httpMethod, path, cls, action,
+        names, false);
+  }
   /**
    * Add a route to the router.
    * e.g., add(GET, "/foo/show", FooController.class, "show", [name...]);
    * The name list is from /foo/show/:name/...
    */
-  synchronized Dest add(WebApp.HTTP httpMethod, String path,
-                        Class<? extends Controller> cls,
-                        String action, List<String> names) {
+  synchronized Dest addWithOptionalDefaultView(WebApp.HTTP httpMethod,
+      String path, Class<? extends Controller> cls,
+      String action, List<String> names, boolean defaultViewNeeded) {
     LOG.debug("adding {}({})->{}#{}", new Object[]{path, names, cls, action});
     Dest dest = addController(httpMethod, path, cls, action, names);
-    addDefaultView(dest);
+    if (defaultViewNeeded) {
+      addDefaultView(dest);
+    }
     return dest;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c87b3a44/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java
index fe800f0..de6a52b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java
@@ -210,6 +210,19 @@ public abstract class WebApp extends ServletModule {
                res.subList(R_PARAMS, res.size()));
   }
 
+  /**
+   * Setup of a webapp serving route without default views added to the page.
+   * @param pathSpec  the path spec in the form of /controller/action/:args etc.
+   * @param cls the controller class
+   * @param action the controller method
+   */
+  public void routeWithoutDefaultView(String pathSpec,
+                    Class<? extends Controller> cls, String action) {
+    List<String> res = parseRoute(pathSpec);
+    router.addWithoutDefaultView(HTTP.GET, res.get(R_PATH), cls, action,
+        res.subList(R_PARAMS, res.size()));
+  }
+
   public void route(String pathSpec, Class<? extends Controller> cls,
                     String action) {
     route(HTTP.GET, pathSpec, cls, action);


---------------------------------------------------------------------
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-11149. Support for parallel checking of FsVolumes.

Posted by xg...@apache.org.
HDFS-11149. Support for parallel checking of FsVolumes.


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

Branch: refs/heads/YARN-5734
Commit: eaaa32950cbae42a74e28e3db3f0cdb1ff158119
Parents: 8f6e143
Author: Arpit Agarwal <ar...@apache.org>
Authored: Tue Nov 29 20:31:02 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Tue Nov 29 20:31:02 2016 -0800

----------------------------------------------------------------------
 .../datanode/checker/DatasetVolumeChecker.java  | 442 +++++++++++++++++++
 .../server/datanode/fsdataset/FsDatasetSpi.java |   7 +
 .../server/datanode/fsdataset/FsVolumeSpi.java  |  12 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |  15 +-
 .../src/main/resources/hdfs-default.xml         |  10 +-
 .../server/datanode/SimulatedFSDataset.java     |   7 +
 .../server/datanode/TestDirectoryScanner.java   |   7 +
 .../checker/TestDatasetVolumeChecker.java       | 261 +++++++++++
 .../TestDatasetVolumeCheckerFailures.java       | 193 ++++++++
 .../datanode/extdataset/ExternalVolumeImpl.java |   7 +
 10 files changed, 953 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
new file mode 100644
index 0000000..8a57812
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
@@ -0,0 +1,442 @@
+/**
+ * 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.checker;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.VolumeCheckContext;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.Timer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.nio.channels.ClosedChannelException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DISK_CHECK_MIN_GAP_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DISK_CHECK_TIMEOUT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DISK_CHECK_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY;
+
+/**
+ * A class that encapsulates running disk checks against each volume of an
+ * {@link FsDatasetSpi} and allows retrieving a list of failed volumes.
+ *
+ * This splits out behavior that was originally implemented across
+ * DataNode, FsDatasetImpl and FsVolumeList.
+ */
+public class DatasetVolumeChecker {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DatasetVolumeChecker.class);
+
+  private AsyncChecker<VolumeCheckContext, VolumeCheckResult> delegateChecker;
+
+  private final AtomicLong numVolumeChecks = new AtomicLong(0);
+  private final AtomicLong numSyncDatasetChecks = new AtomicLong(0);
+  private final AtomicLong numAsyncDatasetChecks = new AtomicLong(0);
+  private final AtomicLong numSkippedChecks = new AtomicLong(0);
+
+  /**
+   * Max allowed time for a disk check in milliseconds. If the check
+   * doesn't complete within this time we declare the disk as dead.
+   */
+  private final long maxAllowedTimeForCheckMs;
+
+  /**
+   * Maximum number of volume failures that can be tolerated without
+   * declaring a fatal error.
+   */
+  private final int maxVolumeFailuresTolerated;
+
+  /**
+   * Minimum time between two successive disk checks of a volume.
+   */
+  private final long minDiskCheckGapMs;
+
+  /**
+   * Timestamp of the last check of all volumes.
+   */
+  private long lastAllVolumesCheck;
+
+  private final Timer timer;
+
+  private static final VolumeCheckContext IGNORED_CONTEXT =
+      new VolumeCheckContext();
+
+  /**
+   * @param conf Configuration object.
+   * @param timer {@link Timer} object used for throttling checks.
+   */
+  public DatasetVolumeChecker(Configuration conf, Timer timer)
+      throws DiskErrorException {
+    maxAllowedTimeForCheckMs = conf.getTimeDuration(
+        DFS_DATANODE_DISK_CHECK_TIMEOUT_KEY,
+        DFS_DATANODE_DISK_CHECK_TIMEOUT_DEFAULT,
+        TimeUnit.MILLISECONDS);
+
+    if (maxAllowedTimeForCheckMs <= 0) {
+      throw new DiskErrorException("Invalid value configured for "
+          + DFS_DATANODE_DISK_CHECK_TIMEOUT_KEY + " - "
+          + maxAllowedTimeForCheckMs + " (should be > 0)");
+    }
+
+    this.timer = timer;
+
+    maxVolumeFailuresTolerated = conf.getInt(
+        DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
+        DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
+
+    minDiskCheckGapMs = conf.getTimeDuration(
+        DFSConfigKeys.DFS_DATANODE_DISK_CHECK_MIN_GAP_KEY,
+        DFSConfigKeys.DFS_DATANODE_DISK_CHECK_MIN_GAP_DEFAULT,
+        TimeUnit.MILLISECONDS);
+
+    if (minDiskCheckGapMs < 0) {
+      throw new DiskErrorException("Invalid value configured for "
+          + DFS_DATANODE_DISK_CHECK_MIN_GAP_KEY + " - "
+          + minDiskCheckGapMs + " (should be >= 0)");
+    }
+
+    lastAllVolumesCheck = timer.monotonicNow() - minDiskCheckGapMs;
+
+    if (maxVolumeFailuresTolerated < 0) {
+      throw new DiskErrorException("Invalid value configured for "
+          + DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY + " - "
+          + maxVolumeFailuresTolerated + " (should be non-negative)");
+    }
+
+    delegateChecker = new ThrottledAsyncChecker<>(
+        timer, minDiskCheckGapMs, Executors.newCachedThreadPool(
+            new ThreadFactoryBuilder()
+                .setNameFormat("DataNode DiskChecker thread %d")
+                .setDaemon(true)
+                .build()));
+  }
+
+  /**
+   * Run checks against all volumes of a dataset.
+   *
+   * This check may be performed at service startup and subsequently at
+   * regular intervals to detect and handle failed volumes.
+   *
+   * @param dataset - FsDatasetSpi to be checked.
+   * @return set of failed volumes.
+   */
+  public Set<StorageLocation> checkAllVolumes(
+      final FsDatasetSpi<? extends FsVolumeSpi> dataset)
+      throws InterruptedException {
+
+    if (timer.monotonicNow() - lastAllVolumesCheck < minDiskCheckGapMs) {
+      numSkippedChecks.incrementAndGet();
+      return Collections.emptySet();
+    }
+
+    lastAllVolumesCheck = timer.monotonicNow();
+    final Set<StorageLocation> healthyVolumes = new HashSet<>();
+    final Set<StorageLocation> failedVolumes = new HashSet<>();
+    final Set<StorageLocation> allVolumes = new HashSet<>();
+
+    final FsDatasetSpi.FsVolumeReferences references =
+        dataset.getFsVolumeReferences();
+    final CountDownLatch resultsLatch = new CountDownLatch(references.size());
+
+    for (int i = 0; i < references.size(); ++i) {
+      final FsVolumeReference reference = references.getReference(i);
+      allVolumes.add(reference.getVolume().getStorageLocation());
+      ListenableFuture<VolumeCheckResult> future =
+          delegateChecker.schedule(reference.getVolume(), IGNORED_CONTEXT);
+      LOG.info("Scheduled health check for volume {}", reference.getVolume());
+      Futures.addCallback(future, new ResultHandler(
+          reference, healthyVolumes, failedVolumes, resultsLatch, null));
+    }
+
+    // Wait until our timeout elapses, after which we give up on
+    // the remaining volumes.
+    if (!resultsLatch.await(maxAllowedTimeForCheckMs, TimeUnit.MILLISECONDS)) {
+      LOG.warn("checkAllVolumes timed out after {} ms" +
+          maxAllowedTimeForCheckMs);
+    }
+
+    numSyncDatasetChecks.incrementAndGet();
+    synchronized (this) {
+      // All volumes that have not been detected as healthy should be
+      // considered failed. This is a superset of 'failedVolumes'.
+      //
+      // Make a copy under the mutex as Sets.difference() returns a view
+      // of a potentially changing set.
+      return new HashSet<>(Sets.difference(allVolumes, healthyVolumes));
+    }
+  }
+
+  /**
+   * Start checks against all volumes of a dataset, invoking the
+   * given callback when the operation has completed. The function
+   * does not wait for the checks to complete.
+   *
+   * If a volume cannot be referenced then it is already closed and
+   * cannot be checked. No error is propagated to the callback for that
+   * volume.
+   *
+   * @param dataset - FsDatasetSpi to be checked.
+   * @param callback - Callback to be invoked when the checks are complete.
+   * @return true if the check was scheduled and the callback will be invoked.
+   *         false if the check was not scheduled and the callback will not be
+   *         invoked.
+   */
+  public boolean checkAllVolumesAsync(
+      final FsDatasetSpi<? extends FsVolumeSpi> dataset,
+      Callback callback) {
+
+    if (timer.monotonicNow() - lastAllVolumesCheck < minDiskCheckGapMs) {
+      numSkippedChecks.incrementAndGet();
+      return false;
+    }
+
+    lastAllVolumesCheck = timer.monotonicNow();
+    final Set<StorageLocation> healthyVolumes = new HashSet<>();
+    final Set<StorageLocation> failedVolumes = new HashSet<>();
+    final FsDatasetSpi.FsVolumeReferences references =
+        dataset.getFsVolumeReferences();
+    final CountDownLatch latch = new CountDownLatch(references.size());
+
+    LOG.info("Checking {} volumes", references.size());
+    for (int i = 0; i < references.size(); ++i) {
+      final FsVolumeReference reference = references.getReference(i);
+      // The context parameter is currently ignored.
+      ListenableFuture<VolumeCheckResult> future =
+          delegateChecker.schedule(reference.getVolume(), IGNORED_CONTEXT);
+      Futures.addCallback(future, new ResultHandler(
+          reference, healthyVolumes, failedVolumes, latch, callback));
+    }
+    numAsyncDatasetChecks.incrementAndGet();
+    return true;
+  }
+
+  /**
+   * A callback interface that is supplied the result of running an
+   * async disk check on multiple volumes.
+   */
+  public interface Callback {
+    /**
+     * @param healthyVolumes set of volumes that passed disk checks.
+     * @param failedVolumes set of volumes that failed disk checks.
+     */
+    void call(Set<StorageLocation> healthyVolumes,
+              Set<StorageLocation> failedVolumes);
+  }
+
+  /**
+   * Check a single volume, returning a {@link ListenableFuture}
+   * that can be used to retrieve the final result.
+   *
+   * If the volume cannot be referenced then it is already closed and
+   * cannot be checked. No error is propagated to the callback.
+   *
+   * @param volume the volume that is to be checked.
+   * @param callback callback to be invoked when the volume check completes.
+   */
+  public void checkVolume(
+      final FsVolumeSpi volume,
+      Callback callback) {
+    FsVolumeReference volumeReference;
+    try {
+      volumeReference = volume.obtainReference();
+    } catch (ClosedChannelException e) {
+      // The volume has already been closed.
+      callback.call(new HashSet<>(), new HashSet<>());
+      return;
+    }
+    ListenableFuture<VolumeCheckResult> future =
+        delegateChecker.schedule(volume, IGNORED_CONTEXT);
+    numVolumeChecks.incrementAndGet();
+    Futures.addCallback(future, new ResultHandler(
+        volumeReference, new HashSet<>(), new HashSet<>(),
+        new CountDownLatch(1), callback));
+  }
+
+  /**
+   * A callback to process the results of checking a volume.
+   */
+  private class ResultHandler
+      implements FutureCallback<VolumeCheckResult> {
+    private final FsVolumeReference reference;
+    private final Set<StorageLocation> failedVolumes;
+    private final Set<StorageLocation> healthyVolumes;
+    private final CountDownLatch latch;
+    private final AtomicLong numVolumes;
+
+    @Nullable
+    private final Callback callback;
+
+    ResultHandler(FsVolumeReference reference,
+                  Set<StorageLocation> healthyVolumes,
+                  Set<StorageLocation> failedVolumes,
+                  CountDownLatch latch,
+                  @Nullable Callback callback) {
+      Preconditions.checkState(reference != null);
+      this.reference = reference;
+      this.healthyVolumes = healthyVolumes;
+      this.failedVolumes = failedVolumes;
+      this.latch = latch;
+      this.callback = callback;
+      numVolumes = new AtomicLong(latch.getCount());
+    }
+
+    @Override
+    public void onSuccess(@Nonnull VolumeCheckResult result) {
+      switch(result) {
+      case HEALTHY:
+      case DEGRADED:
+        LOG.debug("Volume {} is {}.", reference.getVolume(), result);
+        markHealthy();
+        break;
+      case FAILED:
+        LOG.warn("Volume {} detected as being unhealthy",
+            reference.getVolume());
+        markFailed();
+        break;
+      default:
+        LOG.error("Unexpected health check result {} for volume {}",
+            result, reference.getVolume());
+        markHealthy();
+        break;
+      }
+      cleanup();
+    }
+
+    @Override
+    public void onFailure(@Nonnull Throwable t) {
+      Throwable exception = (t instanceof ExecutionException) ?
+          t.getCause() : t;
+      LOG.warn("Exception running disk checks against volume " +
+          reference.getVolume(), exception);
+      markFailed();
+      cleanup();
+    }
+
+    private void markHealthy() {
+      synchronized (DatasetVolumeChecker.this) {
+        healthyVolumes.add(reference.getVolume().getStorageLocation());
+      }
+    }
+
+    private void markFailed() {
+      synchronized (DatasetVolumeChecker.this) {
+        failedVolumes.add(reference.getVolume().getStorageLocation());
+      }
+    }
+
+    private void cleanup() {
+      IOUtils.cleanup(null, reference);
+      invokeCallback();
+    }
+
+    private void invokeCallback() {
+      try {
+        latch.countDown();
+
+        if (numVolumes.decrementAndGet() == 0 &&
+            callback != null) {
+          callback.call(healthyVolumes, failedVolumes);
+        }
+      } catch(Exception e) {
+        // Propagating this exception is unlikely to be helpful.
+        LOG.warn("Unexpected exception", e);
+      }
+    }
+  }
+
+  /**
+   * Shutdown the checker and its associated ExecutorService.
+   *
+   * See {@link ExecutorService#awaitTermination} for the interpretation
+   * of the parameters.
+   */
+  public void shutdownAndWait(int gracePeriod, TimeUnit timeUnit) {
+    try {
+      delegateChecker.shutdownAndWait(gracePeriod, timeUnit);
+    } catch (InterruptedException e) {
+      LOG.warn("DatasetVolumeChecker interrupted during shutdown.");
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  /**
+   * This method is for testing only.
+   *
+   * @param testDelegate
+   */
+  @VisibleForTesting
+  void setDelegateChecker(
+      AsyncChecker<VolumeCheckContext, VolumeCheckResult> testDelegate) {
+    delegateChecker = testDelegate;
+  }
+
+  /**
+   * Return the number of {@link #checkVolume} invocations.
+   */
+  public long getNumVolumeChecks() {
+    return numVolumeChecks.get();
+  }
+
+  /**
+   * Return the number of {@link #checkAllVolumes} invocations.
+   */
+  public long getNumSyncDatasetChecks() {
+    return numSyncDatasetChecks.get();
+  }
+
+  /**
+   * Return the number of {@link #checkAllVolumesAsync} invocations.
+   */
+  public long getNumAsyncDatasetChecks() {
+    return numAsyncDatasetChecks.get();
+  }
+
+  /**
+   * Return the number of checks skipped because the minimum gap since the
+   * last check had not elapsed.
+   */
+  public long getNumSkippedChecks() {
+    return numSkippedChecks.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 547392f..57ec2b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -159,6 +159,13 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
       return references.get(index).getVolume();
     }
 
+    /**
+     * Get the reference for a given index.
+     */
+    public FsVolumeReference getReference(int index) {
+      return references.get(index);
+    }
+
     @Override
     public void close() throws IOException {
       IOException ioe = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/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 dbba31d..a11a207 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
@@ -34,11 +34,15 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
+import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
 
 /**
  * This is an interface for the underlying volume.
  */
-public interface FsVolumeSpi {
+public interface FsVolumeSpi
+    extends Checkable<FsVolumeSpi.VolumeCheckContext, VolumeCheckResult> {
+
   /**
    * Obtain a reference object that had increased 1 reference count of the
    * volume.
@@ -408,4 +412,10 @@ public interface FsVolumeSpi {
   LinkedList<ScanInfo> compileReport(String bpid,
       LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
       throws InterruptedException, IOException;
+
+  /**
+   * Context for the {@link #check} call.
+   */
+  class VolumeCheckContext {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/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 5880b3e..a231e03 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
@@ -49,6 +49,8 @@ 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.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
@@ -69,7 +71,6 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.BlockDirFilter;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
@@ -914,7 +915,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   }
 
   @Override
-  public FsDatasetSpi getDataset() {
+  public FsDatasetSpi<? extends FsVolumeSpi> getDataset() {
     return dataset;
   }
 
@@ -962,6 +963,16 @@ public class FsVolumeImpl implements FsVolumeSpi {
       s.checkDirs();
     }
   }
+
+  @Override
+  public VolumeCheckResult check(VolumeCheckContext ignored)
+      throws DiskErrorException {
+    // TODO:FEDERATION valid synchronization
+    for(BlockPoolSlice s : bpSlices.values()) {
+      s.checkDirs();
+    }
+    return VolumeCheckResult.HEALTHY;
+  }
     
   void getVolumeMap(ReplicaMap volumeMap,
                     final RamDiskReplicaTracker ramDiskReplicaMap)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index c9d74bb..671c98c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4376,11 +4376,11 @@
     <name>dfs.datanode.disk.check.timeout</name>
     <value>10m</value>
     <description>
-      Maximum allowed time for a disk check to complete. If the check does
-      not complete within this time interval then the disk is declared as
-      failed. This setting supports multiple time unit suffixes as described
-      in dfs.heartbeat.interval. If no suffix is specified then milliseconds
-      is assumed.
+      Maximum allowed time for a disk check to complete during DataNode
+      startup. If the check does not complete within this time interval
+      then the disk is declared as failed. This setting supports
+      multiple time unit suffixes as described in dfs.heartbeat.interval.
+      If no suffix is specified then milliseconds is assumed.
     </description>
   </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/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 8e6191a..5d63d07 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
@@ -42,6 +42,7 @@ import org.apache.commons.lang.ArrayUtils;
 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.checker.VolumeCheckResult;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -557,6 +558,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
         throws InterruptedException, IOException {
       return null;
     }
+
+    @Override
+    public VolumeCheckResult check(VolumeCheckContext context)
+        throws Exception {
+      return VolumeCheckResult.HEALTHY;
+    }
   }
 
   private final Map<String, Map<Block, BInfo>> blockMap

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/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 d05e2a7..f08b579 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
@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -897,6 +898,12 @@ public class TestDirectoryScanner {
       return null;
     }
 
+
+    @Override
+    public VolumeCheckResult check(VolumeCheckContext context)
+        throws Exception {
+      return VolumeCheckResult.HEALTHY;
+    }
   }
 
   private final static TestFsVolumeSpi TEST_VOLUME = new TestFsVolumeSpi();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestDatasetVolumeChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestDatasetVolumeChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestDatasetVolumeChecker.java
new file mode 100644
index 0000000..fa809d1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestDatasetVolumeChecker.java
@@ -0,0 +1,261 @@
+/**
+ * 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.checker;
+
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.*;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.VolumeCheckContext;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.FakeTimer;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult.*;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Tests for {@link DatasetVolumeChecker} when the {@link FsVolumeSpi#check}
+ * method returns different values of {@link VolumeCheckResult}.
+ */
+@RunWith(Parameterized.class)
+public class TestDatasetVolumeChecker {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestDatasetVolumeChecker.class);
+
+  @Rule
+  public TestName testName = new TestName();
+
+  /**
+   * Run each test case for each possible value of {@link VolumeCheckResult}.
+   * Including "null" for 'throw exception'.
+   * @return
+   */
+  @Parameters(name="{0}")
+  public static Collection<Object[]> data() {
+    List<Object[]> values = new ArrayList<>();
+    for (VolumeCheckResult result : VolumeCheckResult.values()) {
+      values.add(new Object[] {result});
+    }
+    values.add(new Object[] {null});
+    return values;
+  }
+
+  /**
+   * When null, the check call should throw an exception.
+   */
+  private final VolumeCheckResult expectedVolumeHealth;
+  private static final int NUM_VOLUMES = 2;
+
+
+  public TestDatasetVolumeChecker(VolumeCheckResult expectedVolumeHealth) {
+    this.expectedVolumeHealth = expectedVolumeHealth;
+  }
+
+  /**
+   * Test {@link DatasetVolumeChecker#checkVolume} propagates the
+   * check to the delegate checker.
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 10000)
+  public void testCheckOneVolume() throws Exception {
+    LOG.info("Executing {}", testName.getMethodName());
+    final FsVolumeSpi volume = makeVolumes(1, expectedVolumeHealth).get(0);
+    final DatasetVolumeChecker checker =
+        new DatasetVolumeChecker(new HdfsConfiguration(), new FakeTimer());
+    checker.setDelegateChecker(new DummyChecker());
+    final AtomicLong numCallbackInvocations = new AtomicLong(0);
+
+    /**
+     * Request a check and ensure it triggered {@link FsVolumeSpi#check}.
+     */
+    checker.checkVolume(volume, new DatasetVolumeChecker.Callback() {
+      @Override
+      public void call(Set<StorageLocation> healthyVolumes,
+                       Set<StorageLocation> failedVolumes) {
+        numCallbackInvocations.incrementAndGet();
+        if (expectedVolumeHealth != null && expectedVolumeHealth != FAILED) {
+          assertThat(healthyVolumes.size(), is(1));
+          assertThat(failedVolumes.size(), is(0));
+        } else {
+          assertThat(healthyVolumes.size(), is(0));
+          assertThat(failedVolumes.size(), is(1));
+        }
+      }
+    });
+
+    // Ensure that the check was invoked at least once.
+    verify(volume, times(1)).check(anyObject());
+    assertThat(numCallbackInvocations.get(), is(1L));
+  }
+
+  /**
+   * Test {@link DatasetVolumeChecker#checkAllVolumes} propagates
+   * checks for all volumes to the delegate checker.
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 10000)
+  public void testCheckAllVolumes() throws Exception {
+    LOG.info("Executing {}", testName.getMethodName());
+
+    final List<FsVolumeSpi> volumes = makeVolumes(
+        NUM_VOLUMES, expectedVolumeHealth);
+    final FsDatasetSpi<FsVolumeSpi> dataset = makeDataset(volumes);
+    final DatasetVolumeChecker checker =
+        new DatasetVolumeChecker(new HdfsConfiguration(), new FakeTimer());
+    checker.setDelegateChecker(new DummyChecker());
+
+    Set<StorageLocation> failedVolumes = checker.checkAllVolumes(dataset);
+    LOG.info("Got back {} failed volumes", failedVolumes.size());
+
+    if (expectedVolumeHealth == null || expectedVolumeHealth == FAILED) {
+      assertThat(failedVolumes.size(), is(NUM_VOLUMES));
+    } else {
+      assertTrue(failedVolumes.isEmpty());
+    }
+
+    // Ensure each volume's check() method was called exactly once.
+    for (FsVolumeSpi volume : volumes) {
+      verify(volume, times(1)).check(anyObject());
+    }
+  }
+
+  /**
+   * Unit test for {@link DatasetVolumeChecker#checkAllVolumesAsync}.
+   *
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testCheckAllVolumesAsync() throws Exception {
+    LOG.info("Executing {}", testName.getMethodName());
+
+    final List<FsVolumeSpi> volumes = makeVolumes(
+        NUM_VOLUMES, expectedVolumeHealth);
+    final FsDatasetSpi<FsVolumeSpi> dataset = makeDataset(volumes);
+    final DatasetVolumeChecker checker =
+        new DatasetVolumeChecker(new HdfsConfiguration(), new FakeTimer());
+    checker.setDelegateChecker(new DummyChecker());
+    final AtomicLong numCallbackInvocations = new AtomicLong(0);
+
+    checker.checkAllVolumesAsync(
+        dataset, new DatasetVolumeChecker.Callback() {
+          @Override
+          public void call(
+              Set<StorageLocation> healthyVolumes,
+              Set<StorageLocation> failedVolumes) {
+            LOG.info("Got back {} failed volumes", failedVolumes.size());
+            if (expectedVolumeHealth == null ||
+                expectedVolumeHealth == FAILED) {
+              assertThat(healthyVolumes.size(), is(0));
+              assertThat(failedVolumes.size(), is(NUM_VOLUMES));
+            } else {
+              assertThat(healthyVolumes.size(), is(NUM_VOLUMES));
+              assertThat(failedVolumes.size(), is(0));
+            }
+            numCallbackInvocations.incrementAndGet();
+          }
+        });
+
+    // The callback should be invoked exactly once.
+    assertThat(numCallbackInvocations.get(), is(1L));
+
+    // Ensure each volume's check() method was called exactly once.
+    for (FsVolumeSpi volume : volumes) {
+      verify(volume, times(1)).check(anyObject());
+    }
+  }
+
+  /**
+   * A checker to wraps the result of {@link FsVolumeSpi#check} in
+   * an ImmediateFuture.
+   */
+  static class DummyChecker
+      implements AsyncChecker<VolumeCheckContext, VolumeCheckResult> {
+    @Override
+    public ListenableFuture<VolumeCheckResult> schedule(
+        Checkable<VolumeCheckContext, VolumeCheckResult> target,
+        VolumeCheckContext context) {
+      try {
+        return Futures.immediateFuture(target.check(context));
+      } catch (Exception e) {
+        LOG.info("check routine threw exception " + e);
+        return Futures.immediateFailedFuture(e);
+      }
+    }
+
+    @Override
+    public void shutdownAndWait(long timeout, TimeUnit timeUnit)
+        throws InterruptedException {
+      // Nothing to cancel.
+    }
+  }
+
+  /**
+   * Create a dataset with the given volumes.
+   */
+  static FsDatasetSpi<FsVolumeSpi> makeDataset(List<FsVolumeSpi> volumes)
+      throws Exception {
+    // Create dataset and init volume health.
+    final FsDatasetSpi<FsVolumeSpi> dataset = mock(FsDatasetSpi.class);
+    final FsDatasetSpi.FsVolumeReferences references = new
+        FsDatasetSpi.FsVolumeReferences(volumes);
+    when(dataset.getFsVolumeReferences()).thenReturn(references);
+    return dataset;
+  }
+
+  private static List<FsVolumeSpi> makeVolumes(
+      int numVolumes, VolumeCheckResult health) throws Exception {
+    final List<FsVolumeSpi> volumes = new ArrayList<>(numVolumes);
+    for (int i = 0; i < numVolumes; ++i) {
+      final FsVolumeSpi volume = mock(FsVolumeSpi.class);
+      final FsVolumeReference reference = mock(FsVolumeReference.class);
+      final StorageLocation location = mock(StorageLocation.class);
+
+      when(reference.getVolume()).thenReturn(volume);
+      when(volume.obtainReference()).thenReturn(reference);
+      when(volume.getStorageLocation()).thenReturn(location);
+
+      if (health != null) {
+        when(volume.check(anyObject())).thenReturn(health);
+      } else {
+        final DiskErrorException de = new DiskErrorException("Fake Exception");
+        when(volume.check(anyObject())).thenThrow(de);
+      }
+      volumes.add(volume);
+    }
+    return volumes;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestDatasetVolumeCheckerFailures.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestDatasetVolumeCheckerFailures.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestDatasetVolumeCheckerFailures.java
new file mode 100644
index 0000000..b57d84f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestDatasetVolumeCheckerFailures.java
@@ -0,0 +1,193 @@
+/**
+ * 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.checker;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.*;
+import org.apache.hadoop.util.FakeTimer;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.channels.ClosedChannelException;
+import java.util.concurrent.TimeUnit;
+import java.util.*;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Test a few more conditions not covered by TestDatasetVolumeChecker.
+ */
+public class TestDatasetVolumeCheckerFailures {
+  public static final Logger LOG =LoggerFactory.getLogger(
+      TestDatasetVolumeCheckerFailures.class);
+
+  /**
+   * Test timeout in {@link DatasetVolumeChecker#checkAllVolumes}.
+   * @throws Exception
+   */
+  @Test(timeout=60000)
+  public void testTimeout() throws Exception {
+    // Add a volume whose check routine hangs forever.
+    final List<FsVolumeSpi> volumes =
+        Collections.singletonList(makeHungVolume());
+
+    final FsDatasetSpi<FsVolumeSpi> dataset =
+        TestDatasetVolumeChecker.makeDataset(volumes);
+
+    // Create a disk checker with a very low timeout.
+    final HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setTimeDuration(DFSConfigKeys.DFS_DATANODE_DISK_CHECK_TIMEOUT_KEY,
+        1, TimeUnit.SECONDS);
+    final DatasetVolumeChecker checker =
+        new DatasetVolumeChecker(conf, new FakeTimer());
+
+    // Ensure that the hung volume is detected as failed.
+    Set<StorageLocation> failedVolumes = checker.checkAllVolumes(dataset);
+    assertThat(failedVolumes.size(), is(1));
+  }
+
+  /**
+   * Test checking a closed volume i.e. one which cannot be referenced.
+   *
+   * @throws Exception
+   */
+  @Test(timeout=60000)
+  public void testCheckingClosedVolume() throws Exception {
+    // Add a volume that cannot be referenced.
+    final List<FsVolumeSpi> volumes =
+        Collections.singletonList(makeClosedVolume());
+
+    final FsDatasetSpi<FsVolumeSpi> dataset =
+        TestDatasetVolumeChecker.makeDataset(volumes);
+
+    DatasetVolumeChecker checker =
+        new DatasetVolumeChecker(new HdfsConfiguration(), new FakeTimer());
+    Set<StorageLocation> failedVolumes = checker.checkAllVolumes(dataset);
+    assertThat(failedVolumes.size(), is(0));
+
+    // The closed volume should not have been checked as it cannot
+    // be referenced.
+    verify(volumes.get(0), times(0)).check(anyObject());
+  }
+
+  @Test(timeout=60000)
+  public void testMinGapIsEnforcedForSyncChecks() throws Exception {
+    final FsDatasetSpi<FsVolumeSpi> dataset =
+        TestDatasetVolumeChecker.makeDataset(Collections.emptyList());
+    final FakeTimer timer = new FakeTimer();
+    final Configuration conf = new HdfsConfiguration();
+    final long minGapMs = 100;
+    conf.setTimeDuration(DFSConfigKeys.DFS_DATANODE_DISK_CHECK_MIN_GAP_KEY,
+        minGapMs, TimeUnit.MILLISECONDS);
+    final DatasetVolumeChecker checker = new DatasetVolumeChecker(conf, timer);
+
+    checker.checkAllVolumes(dataset);
+    assertThat(checker.getNumSyncDatasetChecks(), is(1L));
+
+    // Re-check without advancing the timer. Ensure the check is skipped.
+    checker.checkAllVolumes(dataset);
+    assertThat(checker.getNumSyncDatasetChecks(), is(1L));
+    assertThat(checker.getNumSkippedChecks(), is(1L));
+
+    // Re-check after advancing the timer. Ensure the check is performed.
+    timer.advance(minGapMs);
+    checker.checkAllVolumes(dataset);
+    assertThat(checker.getNumSyncDatasetChecks(), is(2L));
+    assertThat(checker.getNumSkippedChecks(), is(1L));
+  }
+
+  @Test(timeout=60000)
+  public void testMinGapIsEnforcedForASyncChecks() throws Exception {
+    final FsDatasetSpi<FsVolumeSpi> dataset =
+        TestDatasetVolumeChecker.makeDataset(Collections.emptyList());
+    final FakeTimer timer = new FakeTimer();
+    final Configuration conf = new HdfsConfiguration();
+    final long minGapMs = 100;
+    conf.setTimeDuration(DFSConfigKeys.DFS_DATANODE_DISK_CHECK_MIN_GAP_KEY,
+        minGapMs, TimeUnit.MILLISECONDS);
+    final DatasetVolumeChecker checker = new DatasetVolumeChecker(conf, timer);
+
+    checker.checkAllVolumesAsync(dataset, null);
+    assertThat(checker.getNumAsyncDatasetChecks(), is(1L));
+
+    // Re-check without advancing the timer. Ensure the check is skipped.
+    checker.checkAllVolumesAsync(dataset, null);
+    assertThat(checker.getNumAsyncDatasetChecks(), is(1L));
+    assertThat(checker.getNumSkippedChecks(), is(1L));
+
+    // Re-check after advancing the timer. Ensure the check is performed.
+    timer.advance(minGapMs);
+    checker.checkAllVolumesAsync(dataset, null);
+    assertThat(checker.getNumAsyncDatasetChecks(), is(2L));
+    assertThat(checker.getNumSkippedChecks(), is(1L));
+  }
+
+  /**
+   * Create a mock FsVolumeSpi whose {@link FsVolumeSpi#check} routine
+   * hangs forever.
+   *
+   * @return volume
+   * @throws Exception
+   */
+  private static FsVolumeSpi makeHungVolume() throws Exception {
+    final FsVolumeSpi volume = mock(FsVolumeSpi.class);
+    final FsVolumeReference reference = mock(FsVolumeReference.class);
+    final StorageLocation location = mock(StorageLocation.class);
+
+    when(reference.getVolume()).thenReturn(volume);
+    when(volume.obtainReference()).thenReturn(reference);
+    when(volume.getStorageLocation()).thenReturn(location);
+    when(volume.check(anyObject())).thenAnswer(
+        new Answer<VolumeCheckResult>() {
+        @Override
+        public VolumeCheckResult answer(InvocationOnMock invocation)
+            throws Throwable {
+          Thread.sleep(Long.MAX_VALUE);     // Sleep forever.
+          return VolumeCheckResult.HEALTHY; // unreachable.
+        }
+      });
+    return volume;
+  }
+
+  /**
+   * Create a mock FsVolumeSpi which is closed and hence cannot
+   * be referenced.
+   *
+   * @return volume
+   * @throws Exception
+   */
+  private static FsVolumeSpi makeClosedVolume() throws Exception {
+    final FsVolumeSpi volume = mock(FsVolumeSpi.class);
+    final StorageLocation location = mock(StorageLocation.class);
+
+    when(volume.obtainReference()).thenThrow(new ClosedChannelException());
+    when(volume.getStorageLocation()).thenReturn(location);
+    return volume;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaaa3295/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 83d6c4c..2753a61 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
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.StorageType;
 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;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -112,4 +113,10 @@ public class ExternalVolumeImpl implements FsVolumeSpi {
       throws InterruptedException, IOException {
     return null;
   }
+
+  @Override
+  public VolumeCheckResult check(VolumeCheckContext context)
+      throws Exception {
+    return VolumeCheckResult.HEALTHY;
+  }
 }


---------------------------------------------------------------------
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: HADOOP-10930. Refactor: Wrap Datanode IO related operations. Contributed by Xiaoyu Yao.

Posted by xg...@apache.org.
HADOOP-10930. Refactor: Wrap Datanode IO related operations. 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/aeecfa24
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/aeecfa24
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/aeecfa24

Branch: refs/heads/YARN-5734
Commit: aeecfa24f4fb6af289920cbf8830c394e66bd78e
Parents: eaaa329
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Nov 29 20:52:36 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Tue Nov 29 20:52:36 2016 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/BlockReceiver.java     |  66 +++----
 .../hdfs/server/datanode/BlockSender.java       | 105 ++++-------
 .../hadoop/hdfs/server/datanode/DNConf.java     |   4 +
 .../hdfs/server/datanode/DataStorage.java       |   5 +
 .../hdfs/server/datanode/LocalReplica.java      | 179 +++++++++++++------
 .../server/datanode/LocalReplicaInPipeline.java |  30 ++--
 .../hdfs/server/datanode/ReplicaInPipeline.java |   4 +-
 .../server/datanode/fsdataset/FsDatasetSpi.java |   3 +-
 .../datanode/fsdataset/ReplicaInputStreams.java | 102 ++++++++++-
 .../fsdataset/ReplicaOutputStreams.java         | 107 ++++++++++-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |  32 ++--
 .../impl/FsDatasetAsyncDiskService.java         |   7 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   5 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   5 +-
 .../org/apache/hadoop/hdfs/TestFileAppend.java  |   2 +-
 .../server/datanode/SimulatedFSDataset.java     |  13 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   2 +-
 .../server/datanode/TestSimulatedFSDataset.java |   2 +-
 .../extdataset/ExternalDatasetImpl.java         |   4 +-
 .../extdataset/ExternalReplicaInPipeline.java   |   6 +-
 20 files changed, 445 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 39419c1..f372072 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
@@ -24,10 +24,7 @@ import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
-import java.io.FileDescriptor;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.nio.ByteBuffer;
@@ -53,7 +50,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
@@ -88,8 +84,6 @@ class BlockReceiver implements Closeable {
    * the DataNode needs to recalculate checksums before writing.
    */
   private final boolean needsChecksumTranslation;
-  private OutputStream out = null; // to block file at local disk
-  private FileDescriptor outFd;
   private DataOutputStream checksumOut = null; // to crc file at local disk
   private final int bytesPerChecksum;
   private final int checksumSize;
@@ -250,7 +244,8 @@ class BlockReceiver implements Closeable {
       
       final boolean isCreate = isDatanode || isTransfer 
           || stage == BlockConstructionStage.PIPELINE_SETUP_CREATE;
-      streams = replicaInfo.createStreams(isCreate, requestedChecksum);
+      streams = replicaInfo.createStreams(isCreate, requestedChecksum,
+          datanodeSlowLogThresholdMs);
       assert streams != null : "null streams!";
 
       // read checksum meta information
@@ -260,13 +255,6 @@ class BlockReceiver implements Closeable {
       this.bytesPerChecksum = diskChecksum.getBytesPerChecksum();
       this.checksumSize = diskChecksum.getChecksumSize();
 
-      this.out = streams.getDataOut();
-      if (out instanceof FileOutputStream) {
-        this.outFd = ((FileOutputStream)out).getFD();
-      } else {
-        LOG.warn("Could not get file descriptor for outputstream of class " +
-            out.getClass());
-      }
       this.checksumOut = new DataOutputStream(new BufferedOutputStream(
           streams.getChecksumOut(), DFSUtilClient.getSmallBufferSize(
           datanode.getConf())));
@@ -319,7 +307,7 @@ class BlockReceiver implements Closeable {
     packetReceiver.close();
 
     IOException ioe = null;
-    if (syncOnClose && (out != null || checksumOut != null)) {
+    if (syncOnClose && (streams.getDataOut() != null || checksumOut != null)) {
       datanode.metrics.incrFsyncCount();      
     }
     long flushTotalNanos = 0;
@@ -348,9 +336,9 @@ class BlockReceiver implements Closeable {
     }
     // close block file
     try {
-      if (out != null) {
+      if (streams.getDataOut() != null) {
         long flushStartNanos = System.nanoTime();
-        out.flush();
+        streams.flushDataOut();
         long flushEndNanos = System.nanoTime();
         if (syncOnClose) {
           long fsyncStartNanos = flushEndNanos;
@@ -359,14 +347,13 @@ class BlockReceiver implements Closeable {
         }
         flushTotalNanos += flushEndNanos - flushStartNanos;
         measuredFlushTime = true;
-        out.close();
-        out = null;
+        streams.closeDataStream();
       }
     } catch (IOException e) {
       ioe = e;
     }
     finally{
-      IOUtils.closeStream(out);
+      streams.close();
     }
     if (replicaHandler != null) {
       IOUtils.cleanup(null, replicaHandler);
@@ -419,9 +406,9 @@ class BlockReceiver implements Closeable {
       }
       flushTotalNanos += flushEndNanos - flushStartNanos;
     }
-    if (out != null) {
+    if (streams.getDataOut() != null) {
       long flushStartNanos = System.nanoTime();
-      out.flush();
+      streams.flushDataOut();
       long flushEndNanos = System.nanoTime();
       if (isSync) {
         long fsyncStartNanos = flushEndNanos;
@@ -430,10 +417,10 @@ class BlockReceiver implements Closeable {
       }
       flushTotalNanos += flushEndNanos - flushStartNanos;
     }
-    if (checksumOut != null || out != null) {
+    if (checksumOut != null || streams.getDataOut() != null) {
       datanode.metrics.addFlushNanos(flushTotalNanos);
       if (isSync) {
-    	  datanode.metrics.incrFsyncCount();      
+        datanode.metrics.incrFsyncCount();
       }
     }
     long duration = Time.monotonicNow() - begin;
@@ -716,16 +703,12 @@ class BlockReceiver implements Closeable {
           int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
           
           // Write data to disk.
-          long begin = Time.monotonicNow();
-          out.write(dataBuf.array(), startByteToDisk, numBytesToDisk);
-          long duration = Time.monotonicNow() - begin;
+          long duration = streams.writeToDisk(dataBuf.array(),
+              startByteToDisk, numBytesToDisk);
+
           if (duration > maxWriteToDiskMs) {
             maxWriteToDiskMs = duration;
           }
-          if (duration > datanodeSlowLogThresholdMs) {
-            LOG.warn("Slow BlockReceiver write data to disk cost:" + duration
-                + "ms (threshold=" + datanodeSlowLogThresholdMs + "ms)");
-          }
 
           final byte[] lastCrc;
           if (shouldNotWriteChecksum) {
@@ -842,7 +825,7 @@ class BlockReceiver implements Closeable {
 
   private void manageWriterOsCache(long offsetInBlock) {
     try {
-      if (outFd != null &&
+      if (streams.getOutFd() != null &&
           offsetInBlock > lastCacheManagementOffset + CACHE_DROP_LAG_BYTES) {
         long begin = Time.monotonicNow();
         //
@@ -857,12 +840,11 @@ class BlockReceiver implements Closeable {
         if (syncBehindWrites) {
           if (syncBehindWritesInBackground) {
             this.datanode.getFSDataset().submitBackgroundSyncFileRangeRequest(
-                block, outFd, lastCacheManagementOffset,
+                block, streams, lastCacheManagementOffset,
                 offsetInBlock - lastCacheManagementOffset,
                 SYNC_FILE_RANGE_WRITE);
           } else {
-            NativeIO.POSIX.syncFileRangeIfPossible(outFd,
-                lastCacheManagementOffset,
+            streams.syncFileRangeIfPossible(lastCacheManagementOffset,
                 offsetInBlock - lastCacheManagementOffset,
                 SYNC_FILE_RANGE_WRITE);
           }
@@ -879,8 +861,8 @@ class BlockReceiver implements Closeable {
         //                     
         long dropPos = lastCacheManagementOffset - CACHE_DROP_LAG_BYTES;
         if (dropPos > 0 && dropCacheBehindWrites) {
-          NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-              block.getBlockName(), outFd, 0, dropPos, POSIX_FADV_DONTNEED);
+          streams.dropCacheBehindWrites(block.getBlockName(), 0, dropPos,
+              POSIX_FADV_DONTNEED);
         }
         lastCacheManagementOffset = offsetInBlock;
         long duration = Time.monotonicNow() - begin;
@@ -989,7 +971,7 @@ class BlockReceiver implements Closeable {
               // The worst case is not recovering this RBW replica. 
               // Client will fall back to regular pipeline recovery.
             } finally {
-              IOUtils.closeStream(out);
+              IOUtils.closeStream(streams.getDataOut());
             }
             try {              
               // Even if the connection is closed after the ack packet is
@@ -1047,8 +1029,8 @@ class BlockReceiver implements Closeable {
    * will be overwritten.
    */
   private void adjustCrcFilePosition() throws IOException {
-    if (out != null) {
-     out.flush();
+    if (streams.getDataOut() != null) {
+      streams.flushDataOut();
     }
     if (checksumOut != null) {
       checksumOut.flush();
@@ -1094,10 +1076,10 @@ class BlockReceiver implements Closeable {
     byte[] crcbuf = new byte[checksumSize];
     try (ReplicaInputStreams instr =
         datanode.data.getTmpInputStreams(block, blkoff, ckoff)) {
-      IOUtils.readFully(instr.getDataIn(), buf, 0, sizePartialChunk);
+      instr.readDataFully(buf, 0, sizePartialChunk);
 
       // open meta file and read in crc value computer earlier
-      IOUtils.readFully(instr.getChecksumIn(), crcbuf, 0, crcbuf.length);
+      instr.readChecksumFully(crcbuf, 0, crcbuf.length);
     }
 
     // compute crc of partial chunk from data read in the block file.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 a1b1f86..9182c88 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
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
-import java.io.FileDescriptor;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -42,11 +41,11 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.SocketOutputStream;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.DataChecksum;
@@ -120,12 +119,11 @@ class BlockSender implements java.io.Closeable {
   
   /** the block to read from */
   private final ExtendedBlock block;
-  /** Stream to read block data from */
-  private InputStream blockIn;
+
+  /** InputStreams and file descriptors to read block/checksum. */
+  private ReplicaInputStreams ris;
   /** updated while using transferTo() */
   private long blockInPosition = -1;
-  /** Stream to read checksum */
-  private DataInputStream checksumIn;
   /** Checksum utility */
   private final DataChecksum checksum;
   /** Initial position to read */
@@ -152,11 +150,6 @@ class BlockSender implements java.io.Closeable {
   private final String clientTraceFmt;
   private volatile ChunkChecksum lastChunkChecksum = null;
   private DataNode datanode;
-  
-  /** The file descriptor of the block being sent */
-  private FileDescriptor blockInFd;
-  /** The reference to the volume where the block is located */
-  private FsVolumeReference volumeRef;
 
   /** The replica of the block that is being read. */
   private final Replica replica;
@@ -201,6 +194,9 @@ class BlockSender implements java.io.Closeable {
               boolean sendChecksum, DataNode datanode, String clientTraceFmt,
               CachingStrategy cachingStrategy)
       throws IOException {
+    InputStream blockIn = null;
+    DataInputStream checksumIn = null;
+    FsVolumeReference volumeRef = null;
     try {
       this.block = block;
       this.corruptChecksumOk = corruptChecksumOk;
@@ -281,7 +277,7 @@ class BlockSender implements java.io.Closeable {
         (!is32Bit || length <= Integer.MAX_VALUE);
 
       // Obtain a reference before reading data
-      this.volumeRef = datanode.data.getVolume(block).obtainReference();
+      volumeRef = datanode.data.getVolume(block).obtainReference();
 
       /* 
        * (corruptChecksumOK, meta_file_exist): operation
@@ -405,14 +401,9 @@ class BlockSender implements java.io.Closeable {
         DataNode.LOG.debug("replica=" + replica);
       }
       blockIn = datanode.data.getBlockInputStream(block, offset); // seek to offset
-      if (blockIn instanceof FileInputStream) {
-        blockInFd = ((FileInputStream)blockIn).getFD();
-      } else {
-        blockInFd = null;
-      }
+      ris = new ReplicaInputStreams(blockIn, checksumIn, volumeRef);
     } catch (IOException ioe) {
       IOUtils.closeStream(this);
-      IOUtils.closeStream(blockIn);
       throw ioe;
     }
   }
@@ -422,12 +413,11 @@ class BlockSender implements java.io.Closeable {
    */
   @Override
   public void close() throws IOException {
-    if (blockInFd != null &&
+    if (ris.getDataInFd() != null &&
         ((dropCacheBehindAllReads) ||
          (dropCacheBehindLargeReads && isLongRead()))) {
       try {
-        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-            block.getBlockName(), blockInFd, lastCacheDropOffset,
+        ris.dropCacheBehindReads(block.getBlockName(), lastCacheDropOffset,
             offset - lastCacheDropOffset, POSIX_FADV_DONTNEED);
       } catch (Exception e) {
         LOG.warn("Unable to drop cache on file close", e);
@@ -436,32 +426,12 @@ class BlockSender implements java.io.Closeable {
     if (curReadahead != null) {
       curReadahead.cancel();
     }
-    
-    IOException ioe = null;
-    if(checksumIn!=null) {
-      try {
-        checksumIn.close(); // close checksum file
-      } catch (IOException e) {
-        ioe = e;
-      }
-      checksumIn = null;
-    }   
-    if(blockIn!=null) {
-      try {
-        blockIn.close(); // close data file
-      } catch (IOException e) {
-        ioe = e;
-      }
-      blockIn = null;
-      blockInFd = null;
-    }
-    if (volumeRef != null) {
-      IOUtils.cleanup(null, volumeRef);
-      volumeRef = null;
-    }
-    // throw IOException if there is any
-    if(ioe!= null) {
-      throw ioe;
+
+    try {
+      ris.closeStreams();
+    } finally {
+      IOUtils.closeStream(ris);
+      ris = null;
     }
   }
   
@@ -565,7 +535,7 @@ class BlockSender implements java.io.Closeable {
     int checksumOff = pkt.position();
     byte[] buf = pkt.array();
     
-    if (checksumSize > 0 && checksumIn != null) {
+    if (checksumSize > 0 && ris.getChecksumIn() != null) {
       readChecksum(buf, checksumOff, checksumDataLen);
 
       // write in progress that we need to use to get last checksum
@@ -581,7 +551,7 @@ class BlockSender implements java.io.Closeable {
     
     int dataOff = checksumOff + checksumDataLen;
     if (!transferTo) { // normal transfer
-      IOUtils.readFully(blockIn, buf, dataOff, dataLen);
+      ris.readDataFully(buf, dataOff, dataLen);
 
       if (verifyChecksum) {
         verifyChecksum(buf, dataOff, dataLen, numChunks, checksumOff);
@@ -593,12 +563,12 @@ class BlockSender implements java.io.Closeable {
         SocketOutputStream sockOut = (SocketOutputStream)out;
         // First write header and checksums
         sockOut.write(buf, headerOff, dataOff - headerOff);
-        
+
         // no need to flush since we know out is not a buffered stream
-        FileChannel fileCh = ((FileInputStream)blockIn).getChannel();
+        FileChannel fileCh = ((FileInputStream)ris.getDataIn()).getChannel();
         LongWritable waitTime = new LongWritable();
         LongWritable transferTime = new LongWritable();
-        sockOut.transferToFully(fileCh, blockInPosition, dataLen, 
+        sockOut.transferToFully(fileCh, blockInPosition, dataLen,
             waitTime, transferTime);
         datanode.metrics.addSendDataPacketBlockedOnNetworkNanos(waitTime.get());
         datanode.metrics.addSendDataPacketTransferNanos(transferTime.get());
@@ -630,7 +600,7 @@ class BlockSender implements java.io.Closeable {
         if (!ioem.startsWith("Broken pipe") && !ioem.startsWith("Connection reset")) {
           LOG.error("BlockSender.sendChunks() exception: ", e);
           datanode.getBlockScanner().markSuspectBlock(
-              volumeRef.getVolume().getStorageID(),
+              ris.getVolumeRef().getVolume().getStorageID(),
               block);
         }
       }
@@ -653,16 +623,15 @@ class BlockSender implements java.io.Closeable {
    */
   private void readChecksum(byte[] buf, final int checksumOffset,
       final int checksumLen) throws IOException {
-    if (checksumSize <= 0 && checksumIn == null) {
+    if (checksumSize <= 0 && ris.getChecksumIn() == null) {
       return;
     }
     try {
-      checksumIn.readFully(buf, checksumOffset, checksumLen);
+      ris.readChecksumFully(buf, checksumOffset, checksumLen);
     } catch (IOException e) {
       LOG.warn(" Could not read or failed to verify checksum for data"
           + " at offset " + offset + " for block " + block, e);
-      IOUtils.closeStream(checksumIn);
-      checksumIn = null;
+      ris.closeChecksumStream();
       if (corruptChecksumOk) {
         if (checksumOffset < checksumLen) {
           // Just fill the array with zeros.
@@ -746,10 +715,10 @@ class BlockSender implements java.io.Closeable {
     
     lastCacheDropOffset = initialOffset;
 
-    if (isLongRead() && blockInFd != null) {
+    if (isLongRead() && ris.getDataInFd() != null) {
       // Advise that this file descriptor will be accessed sequentially.
-      NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-          block.getBlockName(), blockInFd, 0, 0, POSIX_FADV_SEQUENTIAL);
+      ris.dropCacheBehindReads(block.getBlockName(), 0, 0,
+          POSIX_FADV_SEQUENTIAL);
     }
     
     // Trigger readahead of beginning of file if configured.
@@ -761,9 +730,10 @@ class BlockSender implements java.io.Closeable {
       int pktBufSize = PacketHeader.PKT_MAX_HEADER_LEN;
       boolean transferTo = transferToAllowed && !verifyChecksum
           && baseStream instanceof SocketOutputStream
-          && blockIn instanceof FileInputStream;
+          && ris.getDataIn() instanceof FileInputStream;
       if (transferTo) {
-        FileChannel fileChannel = ((FileInputStream)blockIn).getChannel();
+        FileChannel fileChannel =
+            ((FileInputStream)ris.getDataIn()).getChannel();
         blockInPosition = fileChannel.position();
         streamForSendChunks = baseStream;
         maxChunksPerPacket = numberOfChunks(TRANSFERTO_BUFFER_SIZE);
@@ -818,14 +788,16 @@ class BlockSender implements java.io.Closeable {
   private void manageOsCache() throws IOException {
     // We can't manage the cache for this block if we don't have a file
     // descriptor to work with.
-    if (blockInFd == null) return;
+    if (ris.getDataInFd() == null) {
+      return;
+    }
 
     // Perform readahead if necessary
     if ((readaheadLength > 0) && (datanode.readaheadPool != null) &&
           (alwaysReadahead || isLongRead())) {
       curReadahead = datanode.readaheadPool.readaheadStream(
-          clientTraceFmt, blockInFd, offset, readaheadLength, Long.MAX_VALUE,
-          curReadahead);
+          clientTraceFmt, ris.getDataInFd(), offset, readaheadLength,
+          Long.MAX_VALUE, curReadahead);
     }
 
     // Drop what we've just read from cache, since we aren't
@@ -835,8 +807,7 @@ class BlockSender implements java.io.Closeable {
       long nextCacheDropOffset = lastCacheDropOffset + CACHE_DROP_INTERVAL_BYTES;
       if (offset >= nextCacheDropOffset) {
         long dropLength = offset - lastCacheDropOffset;
-        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-            block.getBlockName(), blockInFd, lastCacheDropOffset,
+        ris.dropCacheBehindReads(block.getBlockName(), lastCacheDropOffset,
             dropLength, POSIX_FADV_DONTNEED);
         lastCacheDropOffset = offset;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
index 823d05c..c1487b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
@@ -402,6 +402,10 @@ public class DNConf {
     return volsConfigured;
   }
 
+  public long getSlowIoWarningThresholdMs() {
+    return datanodeSlowIoWarningThresholdMs;
+  }
+
   int getMaxDataLength() {
     return maxDataLength;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 29b14e7..f4deb6d 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
@@ -1355,4 +1355,9 @@ public class DataStorage extends Storage {
   synchronized void removeBlockPoolStorage(String bpId) {
     bpStorageMap.remove(bpId);
   }
+
+  public static boolean fullyDelete(final File dir) {
+    boolean result = FileUtil.fullyDelete(dir);
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 f829111..3615cd1 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,9 +29,6 @@ import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -46,6 +43,8 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.DataChecksum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -69,15 +68,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   private static final Map<String, File> internedBaseDirs = new HashMap<String, File>();
 
-  static final Log LOG = LogFactory.getLog(LocalReplica.class);
-  private final static boolean IS_NATIVE_IO_AVAIL;
-  static {
-    IS_NATIVE_IO_AVAIL = NativeIO.isAvailable();
-    if (Path.WINDOWS && !IS_NATIVE_IO_AVAIL) {
-      LOG.warn("Data node cannot fully support concurrent reading"
-          + " and writing without native code extensions on Windows.");
-    }
-  }
+  static final Logger LOG = LoggerFactory.getLogger(LocalReplica.class);
 
   /**
    * Constructor
@@ -199,14 +190,14 @@ abstract public class LocalReplica extends ReplicaInfo {
     File tmpFile = DatanodeUtil.createTmpFile(b, DatanodeUtil.getUnlinkTmpFile(file));
     try (FileInputStream in = new FileInputStream(file)) {
       try (FileOutputStream out = new FileOutputStream(tmpFile)){
-        IOUtils.copyBytes(in, out, 16 * 1024);
+        copyBytes(in, out, 16 * 1024);
       }
       if (file.length() != tmpFile.length()) {
         throw new IOException("Copy of file " + file + " size " + file.length()+
                               " into file " + tmpFile +
                               " resulted in a size of " + tmpFile.length());
       }
-      FileUtil.replaceFile(tmpFile, file);
+      replaceFile(tmpFile, file);
     } catch (IOException e) {
       boolean done = tmpFile.delete();
       if (!done) {
@@ -241,13 +232,13 @@ abstract public class LocalReplica extends ReplicaInfo {
     }
     File meta = getMetaFile();
 
-    int linkCount = HardLink.getLinkCount(file);
+    int linkCount = getHardLinkCount(file);
     if (linkCount > 1) {
       DataNode.LOG.info("Breaking hardlink for " + linkCount + "x-linked " +
           "block " + this);
       breakHardlinks(file, this);
     }
-    if (HardLink.getLinkCount(meta) > 1) {
+    if (getHardLinkCount(meta) > 1) {
       breakHardlinks(meta, this);
     }
     return true;
@@ -260,18 +251,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public InputStream getDataInputStream(long seekOffset) throws IOException {
-
-    File blockFile = getBlockFile();
-    if (IS_NATIVE_IO_AVAIL) {
-      return NativeIO.getShareDeleteFileInputStream(blockFile, seekOffset);
-    } else {
-      try {
-        return FsDatasetUtil.openAndSeek(blockFile, seekOffset);
-      } catch (FileNotFoundException fnfe) {
-        throw new IOException("Block " + this + " is not valid. " +
-            "Expected block file at " + blockFile + " does not exist.");
-      }
-    }
+    return getDataInputStream(getBlockFile(), seekOffset);
   }
 
   @Override
@@ -286,7 +266,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean deleteBlockData() {
-    return getBlockFile().delete();
+    return fullyDelete(getBlockFile());
   }
 
   @Override
@@ -320,7 +300,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean deleteMetadata() {
-    return getMetaFile().delete();
+    return fullyDelete(getMetaFile());
   }
 
   @Override
@@ -340,7 +320,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   private boolean renameFile(File srcfile, File destfile) throws IOException {
     try {
-      NativeIO.renameTo(srcfile, destfile);
+      rename(srcfile, destfile);
       return true;
     } catch (IOException e) {
       throw new IOException("Failed to move block file for " + this
@@ -367,22 +347,14 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean getPinning(LocalFileSystem localFS) throws IOException {
-    FileStatus fss =
-        localFS.getFileStatus(new Path(getBlockFile().getAbsolutePath()));
-    return fss.getPermission().getStickyBit();
+    return getPinning(localFS, new Path(getBlockFile().getAbsolutePath()));
   }
 
   @Override
   public void setPinning(LocalFileSystem localFS) throws IOException {
     File f = getBlockFile();
     Path p = new Path(f.getAbsolutePath());
-
-    FsPermission oldPermission = localFS.getFileStatus(
-        new Path(f.getAbsolutePath())).getPermission();
-    //sticky bit is used for pinning purpose
-    FsPermission permission = new FsPermission(oldPermission.getUserAction(),
-        oldPermission.getGroupAction(), oldPermission.getOtherAction(), true);
-    localFS.setPermission(p, permission);
+    setPinning(localFS, p);
   }
 
   @Override
@@ -398,7 +370,7 @@ abstract public class LocalReplica extends ReplicaInfo {
     }
     try {
       // calling renameMeta on the ReplicaInfo doesn't work here
-      NativeIO.renameTo(oldmeta, newmeta);
+      rename(oldmeta, newmeta);
     } catch (IOException e) {
       setGenerationStamp(oldGS); // restore old GS
       throw new IOException("Block " + this + " reopen failed. " +
@@ -417,7 +389,113 @@ abstract public class LocalReplica extends ReplicaInfo {
     return info.getBlockFile().compareTo(getBlockFile());
   }
 
-  static public void truncateBlock(File blockFile, File metaFile,
+  @Override
+  public void copyMetadata(URI destination) throws IOException {
+    //for local replicas, we assume the destination URI is file
+    nativeCopyFileUnbuffered(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);
+  }
+
+  /**
+   * Get input stream for a local file and optionally seek to the offset.
+   * @param f path to the file
+   * @param seekOffset offset to seek
+   * @return
+   * @throws IOException
+   */
+  private FileInputStream getDataInputStream(File f, long seekOffset)
+      throws IOException {
+    FileInputStream fis;
+    if (NativeIO.isAvailable()) {
+      fis = NativeIO.getShareDeleteFileInputStream(f, seekOffset);
+    } else {
+      try {
+        fis = FsDatasetUtil.openAndSeek(f, seekOffset);
+      } catch (FileNotFoundException fnfe) {
+        throw new IOException("Expected block file at " + f +
+            " does not exist.");
+      }
+    }
+    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
+   * @param path path to be checked
+   * @return
+   * @throws IOException
+   */
+  public boolean getPinning(LocalFileSystem localFS, Path path) throws
+      IOException {
+    boolean stickyBit =
+        localFS.getFileStatus(path).getPermission().getStickyBit();
+    return stickyBit;
+  }
+
+  /**
+   * Set sticky bit on path to pin file.
+   * @param localFS local file system
+   * @param path path to be pinned with sticky bit
+   * @throws IOException
+   */
+  public void setPinning(LocalFileSystem localFS, Path path) throws
+      IOException {
+    FsPermission oldPermission = localFS.getFileStatus(path).getPermission();
+    FsPermission permission = new FsPermission(oldPermission.getUserAction(),
+        oldPermission.getGroupAction(), oldPermission.getOtherAction(), true);
+    localFS.setPermission(path, permission);
+  }
+
+  public static void truncateBlock(File blockFile, File metaFile,
       long oldlen, long newlen) throws IOException {
     LOG.info("truncateBlock: blockFile=" + blockFile
         + ", metaFile=" + metaFile
@@ -467,19 +545,4 @@ abstract public class LocalReplica extends ReplicaInfo {
       metaRAF.close();
     }
   }
-
-  @Override
-  public void copyMetadata(URI destination) throws IOException {
-    //for local replicas, we assume the destination URI is file
-    Storage.nativeCopyFileUnbuffered(getMetaFile(),
-        new File(destination), true);
-  }
-
-  @Override
-  public void copyBlockdata(URI destination) throws IOException {
-    //for local replicas, we assume the destination URI is file
-    Storage.nativeCopyFileUnbuffered(getBlockFile(),
-        new File(destination), true);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 bc7bc6d..1387155 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
@@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 
@@ -246,7 +245,8 @@ public class LocalReplicaInPipeline extends LocalReplica
 
   @Override // ReplicaInPipeline
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum) throws IOException {
+      DataChecksum requestedChecksum, long slowLogThresholdMs)
+      throws IOException {
     File blockFile = getBlockFile();
     File metaFile = getMetaFile();
     if (DataNode.LOG.isDebugEnabled()) {
@@ -313,7 +313,7 @@ public class LocalReplicaInPipeline extends LocalReplica
         crcOut.getChannel().position(crcDiskSize);
       }
       return new ReplicaOutputStreams(blockOut, crcOut, checksum,
-          getVolume().isTransientStorage());
+          getVolume().isTransientStorage(), slowLogThresholdMs);
     } catch (IOException e) {
       IOUtils.closeStream(blockOut);
       IOUtils.closeStream(metaRAF);
@@ -373,40 +373,30 @@ public class LocalReplicaInPipeline extends LocalReplica
           + " should be derived from LocalReplica");
     }
 
-    LocalReplica localReplica = (LocalReplica) oldReplicaInfo;
-
-    File oldmeta = localReplica.getMetaFile();
+    LocalReplica oldReplica = (LocalReplica) oldReplicaInfo;
+    File oldmeta = oldReplica.getMetaFile();
     File newmeta = getMetaFile();
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming " + oldmeta + " to " + newmeta);
-    }
     try {
-      NativeIO.renameTo(oldmeta, newmeta);
+      oldReplica.renameMeta(newmeta);
     } catch (IOException e) {
       throw new IOException("Block " + oldReplicaInfo + " reopen failed. " +
                             " Unable to move meta file  " + oldmeta +
                             " to rbw dir " + newmeta, e);
     }
 
-    File blkfile = localReplica.getBlockFile();
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming " + blkfile + " to " + newBlkFile
-          + ", file length=" + blkfile.length());
-    }
     try {
-      NativeIO.renameTo(blkfile, newBlkFile);
+      oldReplica.renameBlock(newBlkFile);
     } catch (IOException e) {
       try {
-        NativeIO.renameTo(newmeta, oldmeta);
+        renameMeta(oldmeta);
       } catch (IOException ex) {
         LOG.warn("Cannot move meta file " + newmeta +
             "back to the finalized directory " + oldmeta, ex);
       }
       throw new IOException("Block " + oldReplicaInfo + " reopen failed. " +
-                              " Unable to move block file " + blkfile +
-                              " to rbw dir " + newBlkFile, e);
+          " Unable to move block file " + oldReplica.getBlockFile() +
+          " to rbw dir " + newBlkFile, e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 efa6ea6..5fdbec0 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,11 +69,13 @@ 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) throws IOException;
+      DataChecksum requestedChecksum, long slowLogThresholdMs)
+      throws IOException;
 
   /**
    * Create an output stream to write restart metadata in case of datanode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 57ec2b4..30f045f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 import java.io.Closeable;
 import java.io.EOFException;
 import java.io.File;
-import java.io.FileDescriptor;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -605,7 +604,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * submit a sync_file_range request to AsyncDiskService.
    */
   void submitBackgroundSyncFileRangeRequest(final ExtendedBlock block,
-      final FileDescriptor fd, final long offset, final long nbytes,
+      final ReplicaOutputStreams outs, final long offset, final long nbytes,
       final int flags);
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 227179d..54d0e96 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
@@ -18,24 +18,45 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
 import java.io.Closeable;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
 import java.io.InputStream;
+import java.io.IOException;
 
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIOException;
+import org.slf4j.Logger;
 
 /**
  * Contains the input streams for the data and checksum of a replica.
  */
 public class ReplicaInputStreams implements Closeable {
-  private final InputStream dataIn;
-  private final InputStream checksumIn;
-  private final FsVolumeReference volumeRef;
+  public static final Logger LOG = DataNode.LOG;
+
+  private InputStream dataIn;
+  private InputStream checksumIn;
+  private FsVolumeReference volumeRef;
+  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) {
     this.volumeRef = volumeRef;
     this.dataIn = dataStream;
     this.checksumIn = checksumStream;
+    if (dataIn instanceof FileInputStream) {
+      try {
+        dataInFd = ((FileInputStream) dataIn).getFD();
+      } catch (Exception e) {
+        LOG.warn("Could not get file descriptor for inputstream of class " +
+            this.dataIn.getClass());
+      }
+    } else {
+      LOG.debug("Could not get file descriptor for inputstream of class " +
+          this.dataIn.getClass());
+    }
   }
 
   /** @return the data input stream. */
@@ -48,10 +69,81 @@ public class ReplicaInputStreams implements Closeable {
     return checksumIn;
   }
 
+  public FileDescriptor getDataInFd() {
+    return dataInFd;
+  }
+
+  public FsVolumeReference getVolumeRef() {
+    return volumeRef;
+  }
+
+  public void readDataFully(byte[] buf, int off, int len)
+      throws IOException {
+    IOUtils.readFully(dataIn, buf, off, len);
+  }
+
+  public void readChecksumFully(byte[] buf, int off, int len)
+      throws IOException {
+    IOUtils.readFully(checksumIn, buf, off, len);
+  }
+
+  public void skipDataFully(long len) throws IOException {
+    IOUtils.skipFully(dataIn, len);
+  }
+
+  public void skipChecksumFully(long len) throws IOException {
+    IOUtils.skipFully(checksumIn, len);
+  }
+
+  public void closeChecksumStream() throws IOException {
+    IOUtils.closeStream(checksumIn);
+    checksumIn = null;
+  }
+
+  public void dropCacheBehindReads(String identifier, long offset, long len,
+      int flags) throws NativeIOException {
+    assert this.dataInFd != null : "null dataInFd!";
+    NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+        identifier, dataInFd, offset, len, flags);
+  }
+
+  public void closeStreams() throws IOException {
+    IOException ioe = null;
+    if(checksumIn!=null) {
+      try {
+        checksumIn.close(); // close checksum file
+      } catch (IOException e) {
+        ioe = e;
+      }
+      checksumIn = null;
+    }
+    if(dataIn!=null) {
+      try {
+        dataIn.close(); // close data file
+      } catch (IOException e) {
+        ioe = e;
+      }
+      dataIn = null;
+      dataInFd = null;
+    }
+    if (volumeRef != null) {
+      IOUtils.cleanup(null, volumeRef);
+      volumeRef = null;
+    }
+    // throw IOException if there is any
+    if(ioe!= null) {
+      throw ioe;
+    }
+  }
+
   @Override
   public void close() {
     IOUtils.closeStream(dataIn);
+    dataIn = null;
+    dataInFd = null;
     IOUtils.closeStream(checksumIn);
+    checksumIn = null;
     IOUtils.cleanup(null, volumeRef);
+    volumeRef = null;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 bd1461a..a66847a 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
@@ -18,32 +18,62 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
 import java.io.Closeable;
+import java.io.FileDescriptor;
 import java.io.FileOutputStream;
 import java.io.OutputStream;
 import java.io.IOException;
 
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 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;
 
 /**
  * Contains the output streams for the data and checksum of a replica.
  */
 public class ReplicaOutputStreams implements Closeable {
-  private final OutputStream dataOut;
+  public static final Logger LOG = DataNode.LOG;
+
+  private FileDescriptor outFd = null;
+  /** Stream to block. */
+  private OutputStream dataOut;
+  /** Stream to checksum. */
   private final OutputStream checksumOut;
   private final DataChecksum checksum;
   private final boolean isTransientStorage;
+  private final long slowLogThresholdMs;
 
   /**
    * 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) {
+  public ReplicaOutputStreams(OutputStream dataOut,
+      OutputStream checksumOut, DataChecksum checksum,
+      boolean isTransientStorage, long slowLogThresholdMs) {
     this.dataOut = dataOut;
-    this.checksumOut = checksumOut;
     this.checksum = checksum;
+    this.slowLogThresholdMs = slowLogThresholdMs;
     this.isTransientStorage = isTransientStorage;
+    this.checksumOut = checksumOut;
+
+    try {
+      if (this.dataOut instanceof FileOutputStream) {
+        this.outFd = ((FileOutputStream)this.dataOut).getFD();
+      } else {
+        LOG.debug("Could not get file descriptor for outputstream of class " +
+            this.dataOut.getClass());
+      }
+    } catch (IOException e) {
+      LOG.warn("Could not get file descriptor for outputstream of class " +
+          this.dataOut.getClass());
+    }
+  }
+
+  public FileDescriptor getOutFd() {
+    return outFd;
   }
 
   /** @return the data output stream. */
@@ -72,12 +102,17 @@ public class ReplicaOutputStreams implements Closeable {
     IOUtils.closeStream(checksumOut);
   }
 
+  public void closeDataStream() throws IOException {
+    dataOut.close();
+    dataOut = null;
+  }
+
   /**
    * Sync the data stream if it supports it.
    */
   public void syncDataOut() throws IOException {
     if (dataOut instanceof FileOutputStream) {
-      ((FileOutputStream)dataOut).getChannel().force(true);
+      sync((FileOutputStream)dataOut);
     }
   }
   
@@ -86,8 +121,68 @@ public class ReplicaOutputStreams implements Closeable {
    */
   public void syncChecksumOut() throws IOException {
     if (checksumOut instanceof FileOutputStream) {
-      ((FileOutputStream)checksumOut).getChannel().force(true);
+      sync((FileOutputStream)checksumOut);
     }
   }
 
+  /**
+   * Flush the data stream if it supports it.
+   */
+  public void flushDataOut() throws IOException {
+    flush(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);
+    }
+  }
+
+  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();
+    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);
+  }
+
+  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);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 29dbb29..8323140 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
@@ -49,11 +49,13 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.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;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
+
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker;
@@ -145,7 +147,7 @@ class BlockPoolSlice {
     //
     this.tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP);
     if (tmpDir.exists()) {
-      FileUtil.fullyDelete(tmpDir);
+      DataStorage.fullyDelete(tmpDir);
     }
     this.rbwDir = new File(currentDir, DataStorage.STORAGE_DIR_RBW);
     if (!rbwDir.mkdirs()) {  // create rbw directory if not exist
@@ -436,7 +438,7 @@ class BlockPoolSlice {
 
           final File targetMetaFile = new File(targetDir, metaFile.getName());
           try {
-            NativeIO.renameTo(metaFile, targetMetaFile);
+            LocalReplica.rename(metaFile, targetMetaFile);
           } catch (IOException e) {
             LOG.warn("Failed to move meta file from "
                 + metaFile + " to " + targetMetaFile, e);
@@ -446,7 +448,7 @@ class BlockPoolSlice {
 
           final File targetBlockFile = new File(targetDir, blockFile.getName());
           try {
-            NativeIO.renameTo(blockFile, targetBlockFile);
+            LocalReplica.rename(blockFile, targetBlockFile);
           } catch (IOException e) {
             LOG.warn("Failed to move block file from "
                 + blockFile + " to " + targetBlockFile, e);
@@ -688,6 +690,7 @@ class BlockPoolSlice {
    * @return the number of valid bytes
    */
   private long validateIntegrityAndSetLength(File blockFile, long genStamp) {
+    ReplicaInputStreams ris = null;
     DataInputStream checksumIn = null;
     InputStream blockIn = null;
     try {
@@ -714,21 +717,22 @@ class BlockPoolSlice {
       if (numChunks == 0) {
         return 0;
       }
-      IOUtils.skipFully(checksumIn, (numChunks-1)*checksumSize);
       blockIn = new FileInputStream(blockFile);
+      ris = new ReplicaInputStreams(blockIn, checksumIn,
+          volume.obtainReference());
+      ris.skipChecksumFully((numChunks-1)*checksumSize);
       long lastChunkStartPos = (numChunks-1)*bytesPerChecksum;
-      IOUtils.skipFully(blockIn, lastChunkStartPos);
+      ris.skipDataFully(lastChunkStartPos);
       int lastChunkSize = (int)Math.min(
           bytesPerChecksum, blockFileLen-lastChunkStartPos);
       byte[] buf = new byte[lastChunkSize+checksumSize];
-      checksumIn.readFully(buf, lastChunkSize, checksumSize);
-      IOUtils.readFully(blockIn, buf, 0, lastChunkSize);
-
+      ris.readChecksumFully(buf, lastChunkSize, checksumSize);
+      ris.readDataFully(buf, 0, lastChunkSize);
       checksum.update(buf, 0, lastChunkSize);
       long validFileLength;
       if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
         validFileLength = lastChunkStartPos + lastChunkSize;
-      } else { // last chunck is corrupt
+      } else { // last chunk is corrupt
         validFileLength = lastChunkStartPos;
       }
 
@@ -748,8 +752,12 @@ class BlockPoolSlice {
       FsDatasetImpl.LOG.warn(e);
       return 0;
     } finally {
-      IOUtils.closeStream(checksumIn);
-      IOUtils.closeStream(blockIn);
+      if (ris != null) {
+        ris.close();
+      } else {
+        IOUtils.closeStream(checksumIn);
+        IOUtils.closeStream(blockIn);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 b9c731b..97dcf8d 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
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
-import java.io.FileDescriptor;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -38,9 +37,9 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIOException;
 
 /**
@@ -202,13 +201,13 @@ class FsDatasetAsyncDiskService {
   }
 
   public void submitSyncFileRangeRequest(FsVolumeImpl volume,
-      final FileDescriptor fd, final long offset, final long nbytes,
+      final ReplicaOutputStreams streams, final long offset, final long nbytes,
       final int flags) {
     execute(volume, new Runnable() {
       @Override
       public void run() {
         try {
-          NativeIO.POSIX.syncFileRangeIfPossible(fd, offset, nbytes, flags);
+          streams.syncFileRangeIfPossible(offset, nbytes, flags);
         } catch (NativeIOException e) {
           LOG.warn("sync_file_range error", e);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 954d6ef..6065df2 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,7 +21,6 @@ import java.io.BufferedOutputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.File;
-import java.io.FileDescriptor;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -2755,9 +2754,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
   @Override
   public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
-      FileDescriptor fd, long offset, long nbytes, int flags) {
+      ReplicaOutputStreams outs, long offset, long nbytes, int flags) {
     FsVolumeImpl fsVolumeImpl = this.getVolume(block);
-    asyncDiskService.submitSyncFileRangeRequest(fsVolumeImpl, fd, offset,
+    asyncDiskService.submitSyncFileRangeRequest(fsVolumeImpl, outs, offset,
         nbytes, flags);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 a231e03..08564de 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
@@ -1067,7 +1067,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
         DataStorage.STORAGE_DIR_LAZY_PERSIST);
     File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
     if (force) {
-      FileUtil.fullyDelete(bpDir);
+      DataStorage.fullyDelete(bpDir);
     } else {
       if (!rbwDir.delete()) {
         throw new IOException("Failed to delete " + rbwDir);
@@ -1081,7 +1081,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
           !FileUtil.fullyDelete(lazypersistDir)))) {
         throw new IOException("Failed to delete " + lazypersistDir);
       }
-      FileUtil.fullyDelete(tmpDir);
+      DataStorage.fullyDelete(tmpDir);
       for (File f : FileUtil.listFiles(bpCurrentDir)) {
         if (!f.delete()) {
           throw new IOException("Failed to delete " + f);
@@ -1437,4 +1437,3 @@ public class FsVolumeImpl implements FsVolumeSpi {
         replicaState);
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 20cec6a..e963d41 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);
+          outputStreams = rbw.createStreams(false, DEFAULT_CHECKSUM, 300);
       OutputStream dataOutput = outputStreams.getDataOut();
 
       byte[] appendBytes = new byte[1];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 5d63d07..f0266ef 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
@@ -17,11 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.File;
-import java.io.FileDescriptor;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
+import java.io.*;
 import java.net.URI;
 import java.nio.channels.ClosedChannelException;
 import java.util.Collection;
@@ -261,14 +257,15 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
     @Override
     synchronized public ReplicaOutputStreams createStreams(boolean isCreate, 
-        DataChecksum requestedChecksum) throws IOException {
+        DataChecksum requestedChecksum, long slowLogThresholdMs)
+        throws IOException {
       if (finalized) {
         throw new IOException("Trying to write to a finalized replica "
             + theBlock);
       } else {
         SimulatedOutputStream crcStream = new SimulatedOutputStream();
         return new ReplicaOutputStreams(oStream, crcStream, requestedChecksum,
-            volume.isTransientStorage());
+            volume.isTransientStorage(), slowLogThresholdMs);
       }
     }
 
@@ -1364,7 +1361,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override
   public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
-      FileDescriptor fd, long offset, long nbytes, int flags) {
+      ReplicaOutputStreams outs, long offset, long nbytes, int flags) {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 619eda0..8439991 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));
+          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512), 300);
       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/aeecfa24/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 4e724bc7..fa980c2 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));
+          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512), 300);
       try {
         OutputStream dataOut  = out.getDataOut();
         assertEquals(0, fsdataset.getLength(b));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 4166346..2417c9d 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
@@ -318,8 +318,8 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block, FileDescriptor fd, long offset, long nbytes, int flags) {
-
+  public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
+      ReplicaOutputStreams outs, long offset, long nbytes, int flags) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecfa24/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 90c3b8a..6fa2830 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,8 +58,10 @@ public class ExternalReplicaInPipeline implements ReplicaInPipeline {
 
   @Override
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum) throws IOException {
-    return new ReplicaOutputStreams(null, null, requestedChecksum, false);
+      DataChecksum requestedChecksum, long slowLogThresholdMs)
+      throws IOException {
+    return new ReplicaOutputStreams(null, null, requestedChecksum, false,
+        slowLogThresholdMs);
   }
 
   @Override


---------------------------------------------------------------------
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: HDFS-11132. Allow AccessControlException in contract tests when getFileStatus on subdirectory of existing files. Contributed by Vishwajeet Dusane

Posted by xg...@apache.org.
HDFS-11132. Allow AccessControlException in contract tests when getFileStatus on subdirectory of existing files. 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/19f373a4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/19f373a4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/19f373a4

Branch: refs/heads/YARN-5734
Commit: 19f373a46b2abb7a575f7884a9c7443b8ed67cd3
Parents: 96c5749
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Dec 1 12:54:03 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Dec 1 12:54:28 2016 -0800

----------------------------------------------------------------------
 .../fs/FileContextMainOperationsBaseTest.java   | 21 ++++++++++++++++----
 .../hadoop/fs/FileSystemContractBaseTest.java   | 17 ++++++++++++++--
 2 files changed, 32 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f373a4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
index 5f9151a..2b3ab2a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.fs.Options.CreateOpts;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -251,8 +252,14 @@ public abstract class FileContextMainOperationsBaseTest  {
     } catch (IOException e) {
       // expected
     }
-    Assert.assertFalse(exists(fc, testSubDir));
-    
+
+    try {
+      Assert.assertFalse(exists(fc, testSubDir));
+    } catch (AccessControlException e) {
+      // Expected : HDFS-11132 Checks on paths under file may be rejected by
+      // file missing execute permission.
+    }
+
     Path testDeepSubDir = getTestRootPath(fc, "test/hadoop/file/deep/sub/dir");
     try {
       fc.mkdir(testDeepSubDir, FsPermission.getDefault(), true);
@@ -260,8 +267,14 @@ public abstract class FileContextMainOperationsBaseTest  {
     } catch (IOException e) {
       // expected
     }
-    Assert.assertFalse(exists(fc, testDeepSubDir));
-    
+
+    try {
+      Assert.assertFalse(exists(fc, testDeepSubDir));
+    } catch (AccessControlException e) {
+      // Expected : HDFS-11132 Checks on paths under file may be rejected by
+      // file missing execute permission.
+    }
+
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f373a4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
index bbd7336..6247959 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
@@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -158,7 +159,13 @@ public abstract class FileSystemContractBaseTest extends TestCase {
     } catch (IOException e) {
       // expected
     }
-    assertFalse(fs.exists(testSubDir));
+
+    try {
+      assertFalse(fs.exists(testSubDir));
+    } catch (AccessControlException e) {
+      // Expected : HDFS-11132 Checks on paths under file may be rejected by
+      // file missing execute permission.
+    }
 
     Path testDeepSubDir = path("/test/hadoop/file/deep/sub/dir");
     try {
@@ -167,7 +174,13 @@ public abstract class FileSystemContractBaseTest extends TestCase {
     } catch (IOException e) {
       // expected
     }
-    assertFalse(fs.exists(testDeepSubDir));
+
+    try {
+      assertFalse(fs.exists(testDeepSubDir));
+    } catch (AccessControlException e) {
+      // Expected : HDFS-11132 Checks on paths under file may be rejected by
+      // file missing execute permission.
+    }
 
   }
 


---------------------------------------------------------------------
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: HADOOP-13857. S3AUtils.translateException to map (wrapped) InterruptedExceptions to InterruptedIOEs. Contributed by Steve Loughran

Posted by xg...@apache.org.
HADOOP-13857. S3AUtils.translateException to map (wrapped) InterruptedExceptions to InterruptedIOEs. 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/2ff84a00
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2ff84a00
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2ff84a00

Branch: refs/heads/YARN-5734
Commit: 2ff84a00405e977b1fd791cfb974244580dd5ae8
Parents: c7ff34f
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Dec 2 13:36:04 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Fri Dec 2 13:36:04 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java | 23 ++++++++++++
 .../fs/s3a/TestS3AExceptionTranslation.java     | 38 ++++++++++++++++++++
 2 files changed, 61 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ff84a00/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 49f8862..dedbfd4 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
@@ -40,6 +40,7 @@ import org.slf4j.Logger;
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -113,6 +114,10 @@ public final class S3AUtils {
         path != null ? (" on " + path) : "",
         exception);
     if (!(exception instanceof AmazonServiceException)) {
+      if (containsInterruptedException(exception)) {
+        return (IOException)new InterruptedIOException(message)
+            .initCause(exception);
+      }
       return new AWSClientIOException(message, exception);
     } else {
 
@@ -195,6 +200,24 @@ public final class S3AUtils {
   }
 
   /**
+   * Recurse down the exception loop looking for any inner details about
+   * an interrupted exception.
+   * @param thrown exception thrown
+   * @return true if down the execution chain the operation was an interrupt
+   */
+  static boolean containsInterruptedException(Throwable thrown) {
+    if (thrown == null) {
+      return false;
+    }
+    if (thrown instanceof InterruptedException ||
+        thrown instanceof InterruptedIOException) {
+      return true;
+    }
+    // tail recurse
+    return containsInterruptedException(thrown.getCause());
+  }
+
+  /**
    * Get low level details of an amazon exception for logging; multi-line.
    * @param e exception
    * @return string details

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ff84a00/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java
index a7dafa0..e548ac2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java
@@ -25,9 +25,12 @@ import static org.junit.Assert.*;
 
 import java.io.EOFException;
 import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.nio.file.AccessDeniedException;
 import java.util.Collections;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
@@ -124,4 +127,39 @@ public class TestS3AExceptionTranslation {
     return verifyExceptionClass(clazz,
         translateException("test", "/", exception));
   }
+
+  private void assertContainsInterrupted(boolean expected, Throwable thrown)
+      throws Throwable {
+    if (containsInterruptedException(thrown) != expected) {
+      throw thrown;
+    }
+  }
+
+  @Test
+  public void testInterruptExceptionDetecting() throws Throwable {
+    InterruptedException interrupted = new InterruptedException("irq");
+    assertContainsInterrupted(true, interrupted);
+    IOException ioe = new IOException("ioe");
+    assertContainsInterrupted(false, ioe);
+    assertContainsInterrupted(true, ioe.initCause(interrupted));
+    assertContainsInterrupted(true,
+        new InterruptedIOException("ioirq"));
+  }
+
+  @Test(expected = InterruptedIOException.class)
+  public void testExtractInterrupted() throws Throwable {
+    throw extractException("", "",
+        new ExecutionException(
+            new AmazonClientException(
+                new InterruptedException(""))));
+  }
+
+  @Test(expected = InterruptedIOException.class)
+  public void testExtractInterruptedIO() throws Throwable {
+    throw extractException("", "",
+        new ExecutionException(
+            new AmazonClientException(
+              new InterruptedIOException(""))));
+  }
+
 }


---------------------------------------------------------------------
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: HADOOP-13830. Intermittent failure of ITestS3NContractRootDir#testRecursiveRootListing: "Can not create a Path from an empty string". Contributed by Steve Loughran

Posted by xg...@apache.org.
 HADOOP-13830. Intermittent failure of ITestS3NContractRootDir#testRecursiveRootListing: "Can not create a Path from an empty string". 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/3fd844b9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3fd844b9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3fd844b9

Branch: refs/heads/YARN-5734
Commit: 3fd844b99fdfae6be6e5e261f371d175aad14229
Parents: 4fca94f
Author: Mingliang Liu <li...@apache.org>
Authored: Wed Nov 30 13:01:02 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Wed Nov 30 13:01:19 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/s3native/NativeS3FileSystem.java     | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fd844b9/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
index f741298..1a45db3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
@@ -587,7 +587,12 @@ public class NativeS3FileSystem extends FileSystem {
       for (String commonPrefix : listing.getCommonPrefixes()) {
         Path subpath = keyToPath(commonPrefix);
         String relativePath = pathUri.relativize(subpath.toUri()).getPath();
-        status.add(newDirectory(new Path(absolutePath, relativePath)));
+        // sometimes the common prefix includes the base dir (HADOOP-13830).
+        // avoid that problem by detecting it and keeping it out
+        // of the list
+        if (!relativePath.isEmpty()) {
+          status.add(newDirectory(new Path(absolutePath, relativePath)));
+        }
       }
       priorLastKey = listing.getPriorLastKey();
     } while (priorLastKey != null);


---------------------------------------------------------------------
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: MAPREDUCE-6815. Fix flaky TestKill.testKillTask(). Contributed by Haibo Chen

Posted by xg...@apache.org.
MAPREDUCE-6815. Fix flaky TestKill.testKillTask(). Contributed by Haibo Chen


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

Branch: refs/heads/YARN-5734
Commit: 0cfd7ad21f4457513ed3416e5d77f3123bfe9da0
Parents: f304cca
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Dec 2 17:22:11 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Dec 2 17:22:11 2016 +0000

----------------------------------------------------------------------
 .../java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java | 1 +
 .../src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cfd7ad2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
index 34d9f0e..8a6fa30 100755
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
@@ -259,6 +259,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
     // d. TA processes TA_KILL event and sends T_ATTEMPT_KILLED to the task.
     .addTransition(TaskStateInternal.KILLED, TaskStateInternal.KILLED,
         EnumSet.of(TaskEventType.T_KILL,
+                   TaskEventType.T_SCHEDULE,
                    TaskEventType.T_ATTEMPT_KILLED,
                    TaskEventType.T_ADD_SPEC_ATTEMPT))
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cfd7ad2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java
index 0714647..f681cf8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java
@@ -105,7 +105,7 @@ public class TestKill {
     Job job = app.submit(new Configuration());
     
     //wait and vailidate for Job to become RUNNING
-    app.waitForState(job, JobState.RUNNING);
+    app.waitForInternalState((JobImpl) job, JobStateInternal.RUNNING);
     Map<TaskId,Task> tasks = job.getTasks();
     Assert.assertEquals("No of tasks is not correct", 2, 
         tasks.size());


---------------------------------------------------------------------
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: YARN-4997. Update fair scheduler to use pluggable auth provider (Contributed by Tao Jie via Daniel Templeton)

Posted by xg...@apache.org.
YARN-4997. Update fair scheduler to use pluggable auth provider (Contributed by Tao Jie via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: b3befc021b0e2d63d1a3710ea450797d1129f1f5
Parents: 625df87
Author: Daniel Templeton <te...@apache.org>
Authored: Wed Nov 30 09:50:33 2016 -0800
Committer: Daniel Templeton <te...@apache.org>
Committed: Wed Nov 30 09:50:33 2016 -0800

----------------------------------------------------------------------
 .../security/YarnAuthorizationProvider.java     | 15 +++++
 .../scheduler/fair/AllocationConfiguration.java | 38 +++++------
 .../fair/AllocationFileLoaderService.java       | 68 +++++++++++++++++---
 .../resourcemanager/scheduler/fair/FSQueue.java | 22 +++++--
 .../scheduler/fair/FairScheduler.java           | 45 +++++++++++--
 .../scheduler/fair/TestFairScheduler.java       |  4 +-
 6 files changed, 149 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3befc02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java
index 4b43ea1..9ae4bd7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.util.List;
 
 /**
@@ -61,6 +62,20 @@ public abstract class YarnAuthorizationProvider {
   }
 
   /**
+   * Destroy the {@link YarnAuthorizationProvider} instance.
+   * This method is called only in Tests.
+   */
+  @VisibleForTesting
+  public static void destroy() {
+    synchronized (YarnAuthorizationProvider.class) {
+      if (authorizer != null) {
+        LOG.debug(authorizer.getClass().getName() + " is destroyed.");
+        authorizer = null;
+      }
+    }
+  }
+
+  /**
    * Initialize the provider. Invoked on daemon startup. DefaultYarnAuthorizer is
    * initialized based on configurations.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3befc02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.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/scheduler/fair/AllocationConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
index c771887..7bd2616 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
@@ -17,6 +17,7 @@
 */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -25,13 +26,14 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.ReservationACL;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -69,7 +71,7 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
   private final float queueMaxAMShareDefault;
 
   // ACL's for each queue. Only specifies non-default ACL's from configuration.
-  private final Map<String, Map<QueueACL, AccessControlList>> queueAcls;
+  private final Map<String, Map<AccessType, AccessControlList>> queueAcls;
 
   // Reservation ACL's for each queue. Only specifies non-default ACL's from
   // configuration.
@@ -123,7 +125,7 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
       Map<String, Long> minSharePreemptionTimeouts,
       Map<String, Long> fairSharePreemptionTimeouts,
       Map<String, Float> fairSharePreemptionThresholds,
-      Map<String, Map<QueueACL, AccessControlList>> queueAcls,
+      Map<String, Map<AccessType, AccessControlList>> queueAcls,
       Map<String, Map<ReservationACL, AccessControlList>> resAcls,
       QueuePlacementPolicy placementPolicy,
       Map<FSQueueType, Set<String>> configuredQueues,
@@ -191,9 +193,10 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
    * nobody ("")
    */
   public AccessControlList getQueueAcl(String queue, QueueACL operation) {
-    Map<QueueACL, AccessControlList> queueAcls = this.queueAcls.get(queue);
-    if (queueAcls != null) {
-      AccessControlList operationAcl = queueAcls.get(operation);
+    Map<AccessType, AccessControlList> acls = this.queueAcls.get(queue);
+    if (acls != null) {
+      AccessControlList operationAcl =
+          acls.get(SchedulerUtils.toAccessType(operation));
       if (operationAcl != null) {
         return operationAcl;
       }
@@ -201,6 +204,14 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
     return (queue.equals("root")) ? EVERYBODY_ACL : NOBODY_ACL;
   }
 
+  /**
+   * Get the map of ACLs of all queues.
+   * @return the map of ACLs of all queues
+   */
+  public Map<String, Map<AccessType, AccessControlList>> getQueueAcls() {
+    return Collections.unmodifiableMap(this.queueAcls);
+  }
+
   @Override
   /**
    * Get the map of reservation ACLs to {@link AccessControlList} for the
@@ -315,21 +326,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
     return maxChildQueueResources.get(queue);
   }
 
-  public boolean hasAccess(String queueName, QueueACL acl,
-      UserGroupInformation user) {
-    int lastPeriodIndex = queueName.length();
-    while (lastPeriodIndex != -1) {
-      String queue = queueName.substring(0, lastPeriodIndex);
-      if (getQueueAcl(queue, acl).isUserAllowed(user)) {
-        return true;
-      }
-
-      lastPeriodIndex = queueName.lastIndexOf('.', lastPeriodIndex - 1);
-    }
-    
-    return false;
-  }
-
   @VisibleForTesting
   SchedulingPolicy getSchedulingPolicy(String queueName) {
     SchedulingPolicy policy = schedulingPolicies.get(queueName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3befc02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.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/scheduler/fair/AllocationFileLoaderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
index ee71981..3aecbfd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
@@ -41,8 +41,13 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.ReservationACL;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.security.AccessType;
+import org.apache.hadoop.yarn.security.Permission;
+import org.apache.hadoop.yarn.security.PrivilegedEntity;
+import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -74,6 +79,12 @@ public class AllocationFileLoaderService extends AbstractService {
 
   public static final long THREAD_JOIN_TIMEOUT_MS = 1000;
 
+  private static final String ROOT = "root";
+  private static final AccessControlList EVERYBODY_ACL =
+      new AccessControlList("*");
+  private static final AccessControlList NOBODY_ACL =
+      new AccessControlList(" ");
+
   private final Clock clock;
 
   private long lastSuccessfulReload; // Last time we successfully reloaded queues
@@ -93,7 +104,9 @@ public class AllocationFileLoaderService extends AbstractService {
   public AllocationFileLoaderService() {
     this(SystemClock.getInstance());
   }
-  
+
+  private List<Permission> defaultPermissions;
+
   public AllocationFileLoaderService(Clock clock) {
     super(AllocationFileLoaderService.class.getName());
     this.clock = clock;
@@ -208,6 +221,7 @@ public class AllocationFileLoaderService extends AbstractService {
       ParserConfigurationException, SAXException,
       AllocationConfigurationException {
     if (allocFile == null) {
+      reloadListener.onReload(null);
       return;
     }
     LOG.info("Loading allocation file " + allocFile);
@@ -224,9 +238,10 @@ public class AllocationFileLoaderService extends AbstractService {
     Map<String, Long> minSharePreemptionTimeouts = new HashMap<>();
     Map<String, Long> fairSharePreemptionTimeouts = new HashMap<>();
     Map<String, Float> fairSharePreemptionThresholds = new HashMap<>();
-    Map<String, Map<QueueACL, AccessControlList>> queueAcls = new HashMap<>();
+    Map<String, Map<AccessType, AccessControlList>> queueAcls =
+        new HashMap<>();
     Map<String, Map<ReservationACL, AccessControlList>> reservationAcls =
-            new HashMap<>();
+        new HashMap<>();
     Set<String> reservableQueues = new HashSet<>();
     Set<String> nonPreemptableQueues = new HashSet<>();
     int userMaxAppsDefault = Integer.MAX_VALUE;
@@ -444,7 +459,7 @@ public class AllocationFileLoaderService extends AbstractService {
       Map<String, Long> minSharePreemptionTimeouts,
       Map<String, Long> fairSharePreemptionTimeouts,
       Map<String, Float> fairSharePreemptionThresholds,
-      Map<String, Map<QueueACL, AccessControlList>> queueAcls,
+      Map<String, Map<AccessType, AccessControlList>> queueAcls,
       Map<String, Map<ReservationACL, AccessControlList>> resAcls,
       Map<FSQueueType, Set<String>> configuredQueues,
       Set<String> reservableQueues,
@@ -468,7 +483,7 @@ public class AllocationFileLoaderService extends AbstractService {
       queueName = parentName + "." + queueName;
     }
 
-    Map<QueueACL, AccessControlList> acls = new HashMap<>();
+    Map<AccessType, AccessControlList> acls = new HashMap<>();
     Map<ReservationACL, AccessControlList> racls = new HashMap<>();
     NodeList fields = element.getChildNodes();
     boolean isLeaf = true;
@@ -526,10 +541,10 @@ public class AllocationFileLoaderService extends AbstractService {
         queuePolicies.put(queueName, policy);
       } else if ("aclSubmitApps".equals(field.getTagName())) {
         String text = ((Text)field.getFirstChild()).getData();
-        acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
+        acls.put(AccessType.SUBMIT_APP, new AccessControlList(text));
       } else if ("aclAdministerApps".equals(field.getTagName())) {
         String text = ((Text)field.getFirstChild()).getData();
-        acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
+        acls.put(AccessType.ADMINISTER_QUEUE, new AccessControlList(text));
       } else if ("aclAdministerReservations".equals(field.getTagName())) {
         String text = ((Text)field.getFirstChild()).getData();
         racls.put(ReservationACL.ADMINISTER_RESERVATIONS,
@@ -578,6 +593,17 @@ public class AllocationFileLoaderService extends AbstractService {
       }
       configuredQueues.get(FSQueueType.PARENT).add(queueName);
     }
+    // Set default acls if not defined
+    // The root queue defaults to all access
+    for (QueueACL acl : QueueACL.values()) {
+      AccessType accessType = SchedulerUtils.toAccessType(acl);
+      if (acls.get(accessType) == null){
+        AccessControlList defaultAcl = queueName.equals(ROOT) ?
+            EVERYBODY_ACL : NOBODY_ACL;
+        acls.put(accessType, defaultAcl);
+      }
+    }
+
     queueAcls.put(queueName, acls);
     resAcls.put(queueName, racls);
     if (maxQueueResources.containsKey(queueName) &&
@@ -590,8 +616,30 @@ public class AllocationFileLoaderService extends AbstractService {
               minQueueResources.get(queueName)));
     }
   }
-  
-  public interface Listener {
-    public void onReload(AllocationConfiguration info);
+
+  /**
+   * Returns the list of default permissions.
+   * The default permission for the root queue is everybody ("*")
+   * and the default permission for all other queues is nobody ("").
+   * The default permission list would be loaded before the permissions
+   * from allocation file.
+   * @return default permission list
+   */
+  protected List<Permission> getDefaultPermissions() {
+    if (defaultPermissions == null) {
+      defaultPermissions = new ArrayList<>();
+      Map<AccessType, AccessControlList> acls =
+          new HashMap<>();
+      for (QueueACL acl : QueueACL.values()) {
+        acls.put(SchedulerUtils.toAccessType(acl), EVERYBODY_ACL);
+      }
+      defaultPermissions.add(new Permission(
+          new PrivilegedEntity(EntityType.QUEUE, ROOT), acls));
+    }
+    return defaultPermissions;
+  }
+
+  interface Listener {
+    void onReload(AllocationConfiguration info) throws IOException;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3befc02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.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/scheduler/fair/FSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
index 38c0340..d87668d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
@@ -27,6 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -37,8 +38,13 @@ import org.apache.hadoop.yarn.api.records.QueueStatistics;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.security.AccessRequest;
+import org.apache.hadoop.yarn.security.PrivilegedEntity;
+import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
+import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -53,6 +59,8 @@ public abstract class FSQueue implements Queue, Schedulable {
   private Resource steadyFairShare = Resources.createResource(0, 0);
   private final String name;
   protected final FairScheduler scheduler;
+  private final YarnAuthorizationProvider authorizer;
+  private final PrivilegedEntity queueEntity;
   private final FSQueueMetrics metrics;
   
   protected final FSParentQueue parent;
@@ -78,6 +86,9 @@ public abstract class FSQueue implements Queue, Schedulable {
   public FSQueue(String name, FairScheduler scheduler, FSParentQueue parent) {
     this.name = name;
     this.scheduler = scheduler;
+    this.authorizer =
+        YarnAuthorizationProvider.getInstance(scheduler.getConf());
+    this.queueEntity = new PrivilegedEntity(EntityType.QUEUE, name);
     this.metrics = FSQueueMetrics.forQueue(getName(), parent, true, scheduler.getConf());
     this.parent = parent;
   }
@@ -96,16 +107,16 @@ public abstract class FSQueue implements Queue, Schedulable {
   public String getName() {
     return name;
   }
-  
+
   @Override
   public String getQueueName() {
     return name;
   }
-  
+
   public SchedulingPolicy getPolicy() {
     return policy;
   }
-  
+
   public FSParentQueue getParent() {
     return parent;
   }
@@ -266,7 +277,10 @@ public abstract class FSQueue implements Queue, Schedulable {
   }
 
   public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
-    return scheduler.getAllocationConfiguration().hasAccess(name, acl, user);
+    return authorizer.checkPermission(
+        new AccessRequest(queueEntity, user,
+            SchedulerUtils.toAccessType(acl), null, null,
+            Server.getRemoteAddress(), null));
   }
 
   long getFairSharePreemptionTimeout() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3befc02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.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/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 4285e29..fbcac76 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -25,6 +25,8 @@ import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -34,6 +36,7 @@ import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -53,6 +56,11 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+import org.apache.hadoop.yarn.security.AccessType;
+import org.apache.hadoop.yarn.security.Permission;
+import org.apache.hadoop.yarn.security.PrivilegedEntity;
+import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
+import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
@@ -124,6 +132,7 @@ public class FairScheduler extends
   private FairSchedulerConfiguration conf;
 
   private FSContext context;
+  private YarnAuthorizationProvider authorizer;
   private Resource incrAllocation;
   private QueueManager queueMgr;
   private boolean usePortForNodeName;
@@ -1209,6 +1218,7 @@ public class FairScheduler extends
       writeLock.lock();
       this.conf = new FairSchedulerConfiguration(conf);
       validateConf(this.conf);
+      authorizer = YarnAuthorizationProvider.getInstance(conf);
       minimumAllocation = this.conf.getMinimumAllocation();
       initMaximumResourceCapability(this.conf.getMaximumAllocation());
       incrAllocation = this.conf.getIncrementAllocation();
@@ -1417,23 +1427,46 @@ public class FairScheduler extends
       AllocationFileLoaderService.Listener {
 
     @Override
-    public void onReload(AllocationConfiguration queueInfo) {
+    public void onReload(AllocationConfiguration queueInfo)
+        throws IOException {
       // Commit the reload; also create any queue defined in the alloc file
       // if it does not already exist, so it can be displayed on the web UI.
 
       writeLock.lock();
       try {
-        allocConf = queueInfo;
-        allocConf.getDefaultSchedulingPolicy().initialize(getClusterResource());
-        queueMgr.updateAllocationConfiguration(allocConf);
-        applyChildDefaults();
-        maxRunningEnforcer.updateRunnabilityOnReload();
+        if (queueInfo == null) {
+          authorizer.setPermission(allocsLoader.getDefaultPermissions(),
+              UserGroupInformation.getCurrentUser());
+        } else {
+          allocConf = queueInfo;
+          setQueueAcls(allocConf.getQueueAcls());
+          allocConf.getDefaultSchedulingPolicy().initialize(
+              getClusterResource());
+          queueMgr.updateAllocationConfiguration(allocConf);
+          applyChildDefaults();
+          maxRunningEnforcer.updateRunnabilityOnReload();
+        }
       } finally {
         writeLock.unlock();
       }
     }
   }
 
+  private void setQueueAcls(
+      Map<String, Map<AccessType, AccessControlList>> queueAcls)
+      throws IOException {
+    authorizer.setPermission(allocsLoader.getDefaultPermissions(),
+        UserGroupInformation.getCurrentUser());
+    List<Permission> permissions = new ArrayList<>();
+    for (Entry<String, Map<AccessType, AccessControlList>> queueAcl : queueAcls
+        .entrySet()) {
+      permissions.add(new Permission(new PrivilegedEntity(EntityType.QUEUE,
+          queueAcl.getKey()), queueAcl.getValue()));
+    }
+    authorizer.setPermission(permissions,
+        UserGroupInformation.getCurrentUser());
+  }
+
   /**
    * After reloading the allocation config, the max resource settings for any
    * ad hoc queues will be missing. This method goes through the queue manager's

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3befc02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.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/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index c5ff5e7..55f8849 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@@ -94,10 +95,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdate
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
-
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
@@ -153,6 +152,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     }
     QueueMetrics.clearQueueMetrics();
     DefaultMetricsSystem.shutdown();
+    YarnAuthorizationProvider.destroy();
   }
 
 


---------------------------------------------------------------------
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-8674. Improve performance of postponed block scans. Contributed by Daryn Sharp.

Posted by xg...@apache.org.
HDFS-8674. Improve performance of postponed block scans. Contributed by Daryn Sharp.


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

Branch: refs/heads/YARN-5734
Commit: 96c574927a600d15fab919df1fdc9e07887af6c5
Parents: e0fa492
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Dec 1 12:11:27 2016 -0600
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Dec 1 12:11:27 2016 -0600

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    | 79 ++++++--------------
 1 file changed, 24 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c57492/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 1b744e7..e60703b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -30,6 +30,7 @@ import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -43,8 +44,6 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
 import javax.management.ObjectName;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -101,7 +100,6 @@ import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.util.FoldedTreeSet;
-import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.namenode.CacheManager;
 
@@ -184,7 +182,6 @@ public class BlockManager implements BlockStatsMXBean {
   /** flag indicating whether replication queues have been initialized */
   private boolean initializedReplQueues;
 
-  private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
   private final long startupDelayBlockDeletionInMs;
   private final BlockReportLeaseManager blockReportLeaseManager;
   private ObjectName mxBeanName;
@@ -219,7 +216,7 @@ public class BlockManager implements BlockStatsMXBean {
   }
   /** Used by metrics */
   public long getPostponedMisreplicatedBlocksCount() {
-    return postponedMisreplicatedBlocksCount.get();
+    return postponedMisreplicatedBlocks.size();
   }
   /** Used by metrics */
   public int getPendingDataNodeMessageCount() {
@@ -275,8 +272,10 @@ public class BlockManager implements BlockStatsMXBean {
    * notified of all block deletions that might have been pending
    * when the failover happened.
    */
-  private final LightWeightHashSet<Block> postponedMisreplicatedBlocks =
-      new LightWeightHashSet<>();
+  private final Set<Block> postponedMisreplicatedBlocks =
+      new LinkedHashSet<Block>();
+  private final int blocksPerPostpondedRescan;
+  private final ArrayList<Block> rescannedMisreplicatedBlocks;
 
   /**
    * Maps a StorageID to the set of blocks that are "extra" for this
@@ -378,7 +377,10 @@ public class BlockManager implements BlockStatsMXBean {
     datanodeManager = new DatanodeManager(this, namesystem, conf);
     heartbeatManager = datanodeManager.getHeartbeatManager();
     this.blockIdManager = new BlockIdManager(this);
-
+    blocksPerPostpondedRescan = (int)Math.min(Integer.MAX_VALUE,
+        datanodeManager.getBlocksPerPostponedMisreplicatedBlocksRescan());
+    rescannedMisreplicatedBlocks =
+        new ArrayList<Block>(blocksPerPostpondedRescan);
     startupDelayBlockDeletionInMs = conf.getLong(
         DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_KEY,
         DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_DEFAULT) * 1000L;
@@ -1613,9 +1615,7 @@ public class BlockManager implements BlockStatsMXBean {
 
 
   private void postponeBlock(Block blk) {
-    if (postponedMisreplicatedBlocks.add(blk)) {
-      postponedMisreplicatedBlocksCount.incrementAndGet();
-    }
+    postponedMisreplicatedBlocks.add(blk);
   }
   
   
@@ -2375,39 +2375,14 @@ public class BlockManager implements BlockStatsMXBean {
     if (getPostponedMisreplicatedBlocksCount() == 0) {
       return;
     }
-    long startTimeRescanPostponedMisReplicatedBlocks = Time.monotonicNow();
     namesystem.writeLock();
-    long startPostponedMisReplicatedBlocksCount =
-        getPostponedMisreplicatedBlocksCount();
+    long startTime = Time.monotonicNow();
+    long startSize = postponedMisreplicatedBlocks.size();
     try {
-      // blocksPerRescan is the configured number of blocks per rescan.
-      // Randomly select blocksPerRescan consecutive blocks from the HashSet
-      // when the number of blocks remaining is larger than blocksPerRescan.
-      // The reason we don't always pick the first blocksPerRescan blocks is to
-      // handle the case if for some reason some datanodes remain in
-      // content stale state for a long time and only impact the first
-      // blocksPerRescan blocks.
-      int i = 0;
-      long startIndex = 0;
-      long blocksPerRescan =
-          datanodeManager.getBlocksPerPostponedMisreplicatedBlocksRescan();
-      long base = getPostponedMisreplicatedBlocksCount() - blocksPerRescan;
-      if (base > 0) {
-        startIndex = ThreadLocalRandom.current().nextLong() % (base+1);
-        if (startIndex < 0) {
-          startIndex += (base+1);
-        }
-      }
       Iterator<Block> it = postponedMisreplicatedBlocks.iterator();
-      for (int tmp = 0; tmp < startIndex; tmp++) {
-        it.next();
-      }
-
-      for (;it.hasNext(); i++) {
+      for (int i=0; i < blocksPerPostpondedRescan && it.hasNext(); i++) {
         Block b = it.next();
-        if (i >= blocksPerRescan) {
-          break;
-        }
+        it.remove();
 
         BlockInfo bi = getStoredBlock(b);
         if (bi == null) {
@@ -2416,8 +2391,6 @@ public class BlockManager implements BlockStatsMXBean {
                 "Postponed mis-replicated block " + b + " no longer found " +
                 "in block map.");
           }
-          it.remove();
-          postponedMisreplicatedBlocksCount.decrementAndGet();
           continue;
         }
         MisReplicationResult res = processMisReplicatedBlock(bi);
@@ -2425,20 +2398,19 @@ public class BlockManager implements BlockStatsMXBean {
           LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
               "Re-scanned block " + b + ", result is " + res);
         }
-        if (res != MisReplicationResult.POSTPONE) {
-          it.remove();
-          postponedMisreplicatedBlocksCount.decrementAndGet();
+        if (res == MisReplicationResult.POSTPONE) {
+          rescannedMisreplicatedBlocks.add(b);
         }
       }
     } finally {
-      long endPostponedMisReplicatedBlocksCount =
-          getPostponedMisreplicatedBlocksCount();
+      postponedMisreplicatedBlocks.addAll(rescannedMisreplicatedBlocks);
+      rescannedMisreplicatedBlocks.clear();
+      long endSize = postponedMisreplicatedBlocks.size();
       namesystem.writeUnlock();
       LOG.info("Rescan of postponedMisreplicatedBlocks completed in " +
-          (Time.monotonicNow() - startTimeRescanPostponedMisReplicatedBlocks) +
-          " msecs. " + endPostponedMisReplicatedBlocksCount +
-          " blocks are left. " + (startPostponedMisReplicatedBlocksCount -
-          endPostponedMisReplicatedBlocksCount) + " blocks are removed.");
+          (Time.monotonicNow() - startTime) + " msecs. " +
+          endSize + " blocks are left. " +
+          (startSize - endSize) + " blocks were removed.");
     }
   }
   
@@ -4048,9 +4020,7 @@ public class BlockManager implements BlockStatsMXBean {
     // Remove the block from pendingReconstruction and neededReconstruction
     pendingReconstruction.remove(block);
     neededReconstruction.remove(block, LowRedundancyBlocks.LEVEL);
-    if (postponedMisreplicatedBlocks.remove(block)) {
-      postponedMisreplicatedBlocksCount.decrementAndGet();
-    }
+    postponedMisreplicatedBlocks.remove(block);
   }
 
   public BlockInfo getStoredBlock(Block block) {
@@ -4464,7 +4434,6 @@ public class BlockManager implements BlockStatsMXBean {
     invalidateBlocks.clear();
     datanodeManager.clearPendingQueues();
     postponedMisreplicatedBlocks.clear();
-    postponedMisreplicatedBlocksCount.set(0);
   };
 
   public static LocatedBlock newLocatedBlock(


---------------------------------------------------------------------
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: HADOOP-13790. Make qbt script executable. Contributed by Andrew Wang.

Posted by xg...@apache.org.
HADOOP-13790. Make qbt script executable. 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/be5a7570
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/be5a7570
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/be5a7570

Branch: refs/heads/YARN-5734
Commit: be5a757096246d5c4ef73da9d233adda67bd3d69
Parents: 7c84871
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 1 03:52:04 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 1 03:52:44 2016 +0900

----------------------------------------------------------------------
 dev-support/bin/qbt | 0
 1 file changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/be5a7570/dev-support/bin/qbt
----------------------------------------------------------------------
diff --git a/dev-support/bin/qbt b/dev-support/bin/qbt
old mode 100644
new mode 100755


---------------------------------------------------------------------
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: YARN-5915. ATS 1.5 FileSystemTimelineWriter causes flush() to be called after every event write. Contributed by Atul Sikaria

Posted by xg...@apache.org.
YARN-5915. ATS 1.5 FileSystemTimelineWriter causes flush() to be called after every event write. Contributed by Atul Sikaria


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

Branch: refs/heads/YARN-5734
Commit: f304ccae3c2e0849b0b0b24c4bfe7a3a1ec2bb94
Parents: c87b3a4
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Dec 2 16:54:15 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Dec 2 16:54:15 2016 +0000

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f304ccae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
index 54b4912..fc3385b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
@@ -63,6 +63,7 @@ import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.core.util.MinimalPrettyPrinter;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
 import com.fasterxml.jackson.databind.type.TypeFactory;
 import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
 import com.sun.jersey.api.client.Client;
@@ -276,6 +277,7 @@ public class FileSystemTimelineWriter extends TimelineWriter{
     mapper.setAnnotationIntrospector(
         new JaxbAnnotationIntrospector(TypeFactory.defaultInstance()));
     mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+    mapper.configure(SerializationFeature.FLUSH_AFTER_WRITE_VALUE, false);
     return mapper;
   }
 
@@ -356,6 +358,7 @@ public class FileSystemTimelineWriter extends TimelineWriter{
 
     public void flush() throws IOException {
       if (stream != null) {
+        jsonGenerator.flush();
         stream.hflush();
       }
     }
@@ -368,8 +371,6 @@ public class FileSystemTimelineWriter extends TimelineWriter{
       this.stream = createLogFileStream(fs, logPath);
       this.jsonGenerator = new JsonFactory().createGenerator(stream);
       this.jsonGenerator.setPrettyPrinter(new MinimalPrettyPrinter("\n"));
-      this.jsonGenerator.configure(
-          JsonGenerator.Feature.FLUSH_PASSED_TO_STREAM, false);
       this.lastModifiedTime = Time.monotonicNow();
     }
 


---------------------------------------------------------------------
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: YARN-5761. Separate QueueManager from Scheduler. (Xuan Gong via gtcarrera9)

Posted by xg...@apache.org.
YARN-5761. Separate QueueManager from Scheduler. (Xuan Gong via gtcarrera9)


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

Branch: refs/heads/YARN-5734
Commit: 69fb70c31aa277f7fb14b05c0185ddc5cd90793d
Parents: 3fd844b
Author: Li Lu <gt...@apache.org>
Authored: Wed Nov 30 13:38:42 2016 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Wed Nov 30 13:38:42 2016 -0800

----------------------------------------------------------------------
 .../scheduler/SchedulerQueueManager.java        |  75 ++++
 .../scheduler/capacity/CapacityScheduler.java   | 294 +++------------
 .../capacity/CapacitySchedulerQueueManager.java | 361 +++++++++++++++++++
 .../capacity/TestApplicationLimits.java         |  35 +-
 .../TestApplicationLimitsByPartition.java       |   7 +-
 .../scheduler/capacity/TestChildQueueOrder.java |   9 +-
 .../scheduler/capacity/TestLeafQueue.java       |   9 +-
 .../scheduler/capacity/TestParentQueue.java     |  39 +-
 .../scheduler/capacity/TestReservations.java    |   8 +-
 .../scheduler/capacity/TestUtils.java           |   2 +-
 10 files changed, 536 insertions(+), 303 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerQueueManager.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/scheduler/SchedulerQueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerQueueManager.java
new file mode 100644
index 0000000..92b989a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerQueueManager.java
@@ -0,0 +1,75 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     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.yarn.server.resourcemanager.scheduler;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
+
+/**
+ *
+ * Context of the Queues in Scheduler.
+ *
+ */
+@Private
+@Unstable
+public interface SchedulerQueueManager<T extends Queue,
+    E extends ReservationSchedulerConfiguration> {
+
+  /**
+   * Get the root queue.
+   * @return root queue
+   */
+  T getRootQueue();
+
+  /**
+   * Get all the queues.
+   * @return a map contains all the queues as well as related queue names
+   */
+  Map<String, T> getQueues();
+
+  /**
+   * Remove the queue from the existing queue.
+   * @param queueName the queue name
+   */
+  void removeQueue(String queueName);
+
+  /**
+   * Add a new queue to the existing queues.
+   * @param queueName the queue name
+   * @param queue the queue object
+   */
+  void addQueue(String queueName, T queue);
+
+  /**
+   * Get a queue matching the specified queue name.
+   * @param queueName the queue name
+   * @return a queue object
+   */
+  T getQueue(String queueName);
+
+  /**
+   * Reinitialize the queues.
+   * @param newConf the configuration
+   * @throws IOException if fails to re-initialize queues
+   */
+  void reinitializeQueues(E newConf) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index cc8b3b0..e42b20c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -25,7 +25,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -68,8 +67,6 @@ import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
-import org.apache.hadoop.yarn.security.Permission;
-import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
@@ -156,9 +153,9 @@ public class CapacityScheduler extends
     ResourceAllocationCommitter {
 
   private static final Log LOG = LogFactory.getLog(CapacityScheduler.class);
-  private YarnAuthorizationProvider authorizer;
 
-  private CSQueue root;
+  private CapacitySchedulerQueueManager queueManager;
+
   // timeout to join when we stop this service
   protected final long THREAD_JOIN_TIMEOUT_MS = 1000;
 
@@ -168,22 +165,6 @@ public class CapacityScheduler extends
 
   private int offswitchPerHeartbeatLimit;
 
-  static final Comparator<CSQueue> nonPartitionedQueueComparator =
-      new Comparator<CSQueue>() {
-    @Override
-    public int compare(CSQueue q1, CSQueue q2) {
-      if (q1.getUsedCapacity() < q2.getUsedCapacity()) {
-        return -1;
-      } else if (q1.getUsedCapacity() > q2.getUsedCapacity()) {
-        return 1;
-      }
-
-      return q1.getQueuePath().compareTo(q2.getQueuePath());
-    }
-  };
-  
-  static final PartitionedQueueComparator partitionedQueueComparator =
-      new PartitionedQueueComparator();
 
   @Override
   public void setConf(Configuration conf) {
@@ -236,8 +217,6 @@ public class CapacityScheduler extends
   private CapacitySchedulerConfiguration conf;
   private Configuration yarnConf;
 
-  private Map<String, CSQueue> queues = new ConcurrentHashMap<String, CSQueue>();
-
   private ResourceCalculator calculator;
   private boolean usePortForNodeName;
 
@@ -261,11 +240,11 @@ public class CapacityScheduler extends
 
   @Override
   public QueueMetrics getRootQueueMetrics() {
-    return root.getMetrics();
+    return getRootQueue().getMetrics();
   }
 
   public CSQueue getRootQueue() {
-    return root;
+    return queueManager.getRootQueue();
   }
 
   @Override
@@ -290,12 +269,12 @@ public class CapacityScheduler extends
 
   @Override
   public Comparator<CSQueue> getNonPartitionedQueueComparator() {
-    return nonPartitionedQueueComparator;
+    return CapacitySchedulerQueueManager.NON_PARTITIONED_QUEUE_COMPARATOR;
   }
 
   @Override
   public PartitionedQueueComparator getPartitionedQueueComparator() {
-    return partitionedQueueComparator;
+    return CapacitySchedulerQueueManager.PARTITIONED_QUEUE_COMPARATOR;
   }
 
   @Override
@@ -326,7 +305,10 @@ public class CapacityScheduler extends
       this.usePortForNodeName = this.conf.getUsePortForNodeName();
       this.applications = new ConcurrentHashMap<>();
       this.labelManager = rmContext.getNodeLabelManager();
-      authorizer = YarnAuthorizationProvider.getInstance(yarnConf);
+      this.queueManager = new CapacitySchedulerQueueManager(yarnConf,
+          this.labelManager);
+      this.queueManager.setCapacitySchedulerContext(this);
+
       this.activitiesManager = new ActivitiesManager(rmContext);
       activitiesManager.init(conf);
       initializeQueues(this.conf);
@@ -554,13 +536,6 @@ public class CapacityScheduler extends
     }
   }
 
-  static class QueueHook {
-    public CSQueue hook(CSQueue queue) {
-      return queue;
-    }
-  }
-  private static final QueueHook noop = new QueueHook();
-
   @VisibleForTesting
   public UserGroupMappingPlacementRule
       getUserGroupMappingPlacementRule() throws IOException {
@@ -578,7 +553,7 @@ public class CapacityScheduler extends
         if (!mappingQueue.equals(
             UserGroupMappingPlacementRule.CURRENT_USER_MAPPING) && !mappingQueue
             .equals(UserGroupMappingPlacementRule.PRIMARY_GROUP_MAPPING)) {
-          CSQueue queue = queues.get(mappingQueue);
+          CSQueue queue = getQueue(mappingQueue);
           if (queue == null || !(queue instanceof LeafQueue)) {
             throw new IOException(
                 "mapping contains invalid or non-leaf queue " + mappingQueue);
@@ -616,184 +591,29 @@ public class CapacityScheduler extends
   private void initializeQueues(CapacitySchedulerConfiguration conf)
     throws IOException {
 
-    root =
-        parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT,
-            queues, queues, noop);
-    labelManager.reinitializeQueueLabels(getQueueToLabels());
-    LOG.info("Initialized root queue " + root);
+    this.queueManager.initializeQueues(conf);
+
     updatePlacementRules();
-    setQueueAcls(authorizer, queues);
 
     // Notify Preemption Manager
-    preemptionManager.refreshQueues(null, root);
+    preemptionManager.refreshQueues(null, this.getRootQueue());
   }
 
   @Lock(CapacityScheduler.class)
   private void reinitializeQueues(CapacitySchedulerConfiguration newConf)
   throws IOException {
-    // Parse new queues
-    Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
-    CSQueue newRoot =
-        parseQueue(this, newConf, null, CapacitySchedulerConfiguration.ROOT,
-            newQueues, queues, noop);
-
-    // Ensure all existing queues are still present
-    validateExistingQueues(queues, newQueues);
-
-    // Add new queues
-    addNewQueues(queues, newQueues);
-
-    // Re-configure queues
-    root.reinitialize(newRoot, getClusterResource());
+    this.queueManager.reinitializeQueues(newConf);
     updatePlacementRules();
 
-    // Re-calculate headroom for active applications
-    Resource clusterResource = getClusterResource();
-    root.updateClusterResource(clusterResource, new ResourceLimits(
-        clusterResource));
-
-    labelManager.reinitializeQueueLabels(getQueueToLabels());
-    setQueueAcls(authorizer, queues);
-
     // Notify Preemption Manager
-    preemptionManager.refreshQueues(null, root);
-  }
-
-  @VisibleForTesting
-  public static void setQueueAcls(YarnAuthorizationProvider authorizer,
-      Map<String, CSQueue> queues) throws IOException {
-    List<Permission> permissions = new ArrayList<>();
-    for (CSQueue queue : queues.values()) {
-      AbstractCSQueue csQueue = (AbstractCSQueue) queue;
-      permissions.add(
-          new Permission(csQueue.getPrivilegedEntity(), csQueue.getACLs()));
-    }
-    authorizer.setPermission(permissions, UserGroupInformation.getCurrentUser());
-  }
-
-  private Map<String, Set<String>> getQueueToLabels() {
-    Map<String, Set<String>> queueToLabels = new HashMap<String, Set<String>>();
-    for (CSQueue queue : queues.values()) {
-      queueToLabels.put(queue.getQueueName(), queue.getAccessibleNodeLabels());
-    }
-    return queueToLabels;
-  }
-
-  /**
-   * Ensure all existing queues are present. Queues cannot be deleted
-   * @param queues existing queues
-   * @param newQueues new queues
-   */
-  @Lock(CapacityScheduler.class)
-  private void validateExistingQueues(
-      Map<String, CSQueue> queues, Map<String, CSQueue> newQueues)
-  throws IOException {
-    // check that all static queues are included in the newQueues list
-    for (Map.Entry<String, CSQueue> e : queues.entrySet()) {
-      if (!(e.getValue() instanceof ReservationQueue)) {
-        String queueName = e.getKey();
-        CSQueue oldQueue = e.getValue();
-        CSQueue newQueue = newQueues.get(queueName);
-        if (null == newQueue) {
-          throw new IOException(queueName + " cannot be found during refresh!");
-        } else if (!oldQueue.getQueuePath().equals(newQueue.getQueuePath())) {
-          throw new IOException(queueName + " is moved from:"
-              + oldQueue.getQueuePath() + " to:" + newQueue.getQueuePath()
-              + " after refresh, which is not allowed.");
-        }
-      }
-    }
-  }
-
-  /**
-   * Add the new queues (only) to our list of queues...
-   * ... be careful, do not overwrite existing queues.
-   * @param queues
-   * @param newQueues
-   */
-  @Lock(CapacityScheduler.class)
-  private void addNewQueues(
-      Map<String, CSQueue> queues, Map<String, CSQueue> newQueues)
-  {
-    for (Map.Entry<String, CSQueue> e : newQueues.entrySet()) {
-      String queueName = e.getKey();
-      CSQueue queue = e.getValue();
-      if (!queues.containsKey(queueName)) {
-        queues.put(queueName, queue);
-      }
-    }
-  }
-
-  @Lock(CapacityScheduler.class)
-  static CSQueue parseQueue(
-      CapacitySchedulerContext csContext,
-      CapacitySchedulerConfiguration conf,
-      CSQueue parent, String queueName, Map<String, CSQueue> queues,
-      Map<String, CSQueue> oldQueues,
-      QueueHook hook) throws IOException {
-    CSQueue queue;
-    String fullQueueName =
-        (parent == null) ? queueName
-            : (parent.getQueuePath() + "." + queueName);
-    String[] childQueueNames =
-      conf.getQueues(fullQueueName);
-    boolean isReservableQueue = conf.isReservable(fullQueueName);
-    if (childQueueNames == null || childQueueNames.length == 0) {
-      if (null == parent) {
-        throw new IllegalStateException(
-            "Queue configuration missing child queue names for " + queueName);
-      }
-      // Check if the queue will be dynamically managed by the Reservation
-      // system
-      if (isReservableQueue) {
-        queue =
-            new PlanQueue(csContext, queueName, parent,
-                oldQueues.get(queueName));
-      } else {
-        queue =
-            new LeafQueue(csContext, queueName, parent,
-                oldQueues.get(queueName));
-
-        // Used only for unit tests
-        queue = hook.hook(queue);
-      }
-    } else {
-      if (isReservableQueue) {
-        throw new IllegalStateException(
-            "Only Leaf Queues can be reservable for " + queueName);
-      }
-      ParentQueue parentQueue =
-        new ParentQueue(csContext, queueName, parent, oldQueues.get(queueName));
-
-      // Used only for unit tests
-      queue = hook.hook(parentQueue);
-
-      List<CSQueue> childQueues = new ArrayList<CSQueue>();
-      for (String childQueueName : childQueueNames) {
-        CSQueue childQueue =
-          parseQueue(csContext, conf, queue, childQueueName,
-              queues, oldQueues, hook);
-        childQueues.add(childQueue);
-      }
-      parentQueue.setChildQueues(childQueues);
-    }
-
-    if (queue instanceof LeafQueue && queues.containsKey(queueName)
-        && queues.get(queueName) instanceof LeafQueue) {
-      throw new IOException("Two leaf queues were named " + queueName
-          + ". Leaf queue names must be distinct");
-    }
-    queues.put(queueName, queue);
-
-    LOG.info("Initialized queue: " + queue);
-    return queue;
+    preemptionManager.refreshQueues(null, this.getRootQueue());
   }
 
   public CSQueue getQueue(String queueName) {
     if (queueName == null) {
       return null;
     }
-    return queues.get(queueName);
+    return this.queueManager.getQueue(queueName);
   }
 
   private void addApplicationOnRecovery(
@@ -1047,7 +867,7 @@ public class CapacityScheduler extends
 
       // Inform the queue
       String queueName = attempt.getQueue().getQueueName();
-      CSQueue queue = queues.get(queueName);
+      CSQueue queue = this.getQueue(queueName);
       if (!(queue instanceof LeafQueue)) {
         LOG.error(
             "Cannot finish application " + "from non-leaf queue: " + queueName);
@@ -1174,7 +994,7 @@ public class CapacityScheduler extends
       boolean includeChildQueues, boolean recursive)
   throws IOException {
     CSQueue queue = null;
-    queue = this.queues.get(queueName);
+    queue = this.getQueue(queueName);
     if (queue == null) {
       throw new IOException("Unknown queue: " + queueName);
     }
@@ -1192,7 +1012,7 @@ public class CapacityScheduler extends
       return new ArrayList<QueueUserACLInfo>();
     }
 
-    return root.getQueueUserAclInfo(user);
+    return getRootQueue().getQueueUserAclInfo(user);
   }
 
   @Override
@@ -1235,7 +1055,7 @@ public class CapacityScheduler extends
       writeLock.lock();
       updateNodeResource(nm, resourceOption);
       Resource clusterResource = getClusterResource();
-      root.updateClusterResource(clusterResource,
+      getRootQueue().updateClusterResource(clusterResource,
           new ResourceLimits(clusterResource));
     } finally {
       writeLock.unlock();
@@ -1471,8 +1291,8 @@ public class CapacityScheduler extends
 
   private CSAssignment allocateOrReserveNewContainers(
       PlacementSet<FiCaSchedulerNode> ps, boolean withNodeHeartbeat) {
-    CSAssignment assignment = root.assignContainers(getClusterResource(), ps,
-        new ResourceLimits(labelManager
+    CSAssignment assignment = getRootQueue().assignContainers(
+        getClusterResource(), ps, new ResourceLimits(labelManager
             .getResourceByLabel(ps.getPartition(), getClusterResource())),
         SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
 
@@ -1506,7 +1326,7 @@ public class CapacityScheduler extends
     }
 
     // Try to use NON_EXCLUSIVE
-    assignment = root.assignContainers(getClusterResource(), ps,
+    assignment = getRootQueue().assignContainers(getClusterResource(), ps,
         // TODO, now we only consider limits for parent for non-labeled
         // resources, should consider labeled resources as well.
         new ResourceLimits(labelManager
@@ -1526,8 +1346,8 @@ public class CapacityScheduler extends
       PlacementSet<FiCaSchedulerNode> ps) {
     // When this time look at multiple nodes, try schedule if the
     // partition has any available resource or killable resource
-    if (root.getQueueCapacities().getUsedCapacity(ps.getPartition()) >= 1.0f
-        && preemptionManager.getKillableResource(
+    if (getRootQueue().getQueueCapacities().getUsedCapacity(
+        ps.getPartition()) >= 1.0f && preemptionManager.getKillableResource(
         CapacitySchedulerConfiguration.ROOT, ps.getPartition()) == Resources
         .none()) {
       if (LOG.isDebugEnabled()) {
@@ -1710,7 +1530,7 @@ public class CapacityScheduler extends
         updateLabelsOnNode(id, labels);
       }
       Resource clusterResource = getClusterResource();
-      root.updateClusterResource(clusterResource,
+      getRootQueue().updateClusterResource(clusterResource,
           new ResourceLimits(clusterResource));
     } finally {
       writeLock.unlock();
@@ -1731,7 +1551,7 @@ public class CapacityScheduler extends
       }
 
       Resource clusterResource = getClusterResource();
-      root.updateClusterResource(clusterResource,
+      getRootQueue().updateClusterResource(clusterResource,
           new ResourceLimits(clusterResource));
 
       LOG.info(
@@ -1782,7 +1602,7 @@ public class CapacityScheduler extends
 
       nodeTracker.removeNode(nodeId);
       Resource clusterResource = getClusterResource();
-      root.updateClusterResource(clusterResource,
+      getRootQueue().updateClusterResource(clusterResource,
           new ResourceLimits(clusterResource));
       int numNodes = nodeTracker.nodeCount();
 
@@ -2020,7 +1840,7 @@ public class CapacityScheduler extends
 
   @Override
   public List<ApplicationAttemptId> getAppsInQueue(String queueName) {
-    CSQueue queue = queues.get(queueName);
+    CSQueue queue = getQueue(queueName);
     if (queue == null) {
       return null;
     }
@@ -2030,7 +1850,8 @@ public class CapacityScheduler extends
   }
 
   public boolean isSystemAppsLimitReached() {
-    if (root.getNumApplications() < conf.getMaximumSystemApplications()) {
+    if (getRootQueue().getNumApplications() < conf
+        .getMaximumSystemApplications()) {
       return false;
     }
     return true;
@@ -2131,7 +1952,7 @@ public class CapacityScheduler extends
       }
 
       ((PlanQueue) disposableLeafQueue.getParent()).removeChildQueue(q);
-      this.queues.remove(queueName);
+      this.queueManager.removeQueue(queueName);
       LOG.info("Removal of ReservationQueue " + queueName + " has succeeded");
     } finally {
       writeLock.unlock();
@@ -2160,7 +1981,7 @@ public class CapacityScheduler extends
       PlanQueue parentPlan = (PlanQueue) newQueue.getParent();
       String queuename = newQueue.getQueueName();
       parentPlan.addChildQueue(newQueue);
-      this.queues.put(queuename, newQueue);
+      this.queueManager.addQueue(queuename, newQueue);
       LOG.info("Creation of ReservationQueue " + newQueue + " succeeded");
     } finally {
       writeLock.unlock();
@@ -2172,7 +1993,7 @@ public class CapacityScheduler extends
       throws YarnException {
     try {
       writeLock.lock();
-      LeafQueue queue = getAndCheckLeafQueue(inQueue);
+      LeafQueue queue = this.queueManager.getAndCheckLeafQueue(inQueue);
       ParentQueue parent = (ParentQueue) queue.getParent();
 
       if (!(queue instanceof ReservationQueue)) {
@@ -2224,9 +2045,10 @@ public class CapacityScheduler extends
       FiCaSchedulerApp app = getApplicationAttempt(
           ApplicationAttemptId.newInstance(appId, 0));
       String sourceQueueName = app.getQueue().getQueueName();
-      LeafQueue source = getAndCheckLeafQueue(sourceQueueName);
+      LeafQueue source = this.queueManager.getAndCheckLeafQueue(
+          sourceQueueName);
       String destQueueName = handleMoveToPlanQueue(targetQueueName);
-      LeafQueue dest = getAndCheckLeafQueue(destQueueName);
+      LeafQueue dest = this.queueManager.getAndCheckLeafQueue(destQueueName);
       // Validation check - ACLs, submission limits for user & queue
       String user = app.getUser();
       checkQueuePartition(app, dest);
@@ -2290,27 +2112,6 @@ public class CapacityScheduler extends
     }
   }
 
-  /**
-   * Check that the String provided in input is the name of an existing,
-   * LeafQueue, if successful returns the queue.
-   *
-   * @param queue
-   * @return the LeafQueue
-   * @throws YarnException
-   */
-  private LeafQueue getAndCheckLeafQueue(String queue) throws YarnException {
-    CSQueue ret = this.getQueue(queue);
-    if (ret == null) {
-      throw new YarnException("The specified Queue: " + queue
-          + " doesn't exist");
-    }
-    if (!(ret instanceof LeafQueue)) {
-      throw new YarnException("The specified Queue: " + queue
-          + " is not a Leaf Queue. Move is supported only for Leaf Queues.");
-    }
-    return (LeafQueue) ret;
-  }
-
   /** {@inheritDoc} */
   @Override
   public EnumSet<SchedulerResourceTypes> getSchedulingResourceTypes() {
@@ -2347,7 +2148,7 @@ public class CapacityScheduler extends
   @Override
   public Set<String> getPlanQueues() {
     Set<String> ret = new HashSet<String>();
-    for (Map.Entry<String, CSQueue> l : queues.entrySet()) {
+    for (Map.Entry<String, CSQueue> l : queueManager.getQueues().entrySet()) {
       if (l.getValue() instanceof PlanQueue) {
         ret.add(l.getKey());
       }
@@ -2367,7 +2168,8 @@ public class CapacityScheduler extends
     if (null == priorityFromContext) {
       // Get the default priority for the Queue. If Queue is non-existent, then
       // use default priority
-      priorityFromContext = getDefaultPriorityForQueue(queueName);
+      priorityFromContext = this.queueManager.getDefaultPriorityForQueue(
+          queueName);
 
       LOG.info("Application '" + applicationId
           + "' is submitted without priority "
@@ -2391,18 +2193,6 @@ public class CapacityScheduler extends
     return appPriority;
   }
 
-  private Priority getDefaultPriorityForQueue(String queueName) {
-    Queue queue = getQueue(queueName);
-    if (null == queue || null == queue.getDefaultApplicationPriority()) {
-      // Return with default application priority
-      return Priority.newInstance(CapacitySchedulerConfiguration
-          .DEFAULT_CONFIGURATION_APPLICATION_PRIORITY);
-    }
-
-    return Priority.newInstance(queue.getDefaultApplicationPriority()
-        .getPriority());
-  }
-
   @Override
   public Priority updateApplicationPriority(Priority newPriority,
       ApplicationId applicationId, SettableFuture<Object> future)
@@ -2456,7 +2246,7 @@ public class CapacityScheduler extends
 
   @Override
   public ResourceUsage getClusterResourceUsage() {
-    return root.getQueueResourceUsage();
+    return getRootQueue().getQueueResourceUsage();
   }
 
   private SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> getSchedulerContainer(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.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/scheduler/capacity/CapacitySchedulerQueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java
new file mode 100644
index 0000000..7a6ce56
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java
@@ -0,0 +1,361 @@
+/**
+* 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.yarn.server.resourcemanager.scheduler.capacity;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.Permission;
+import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerQueueManager;
+
+/**
+ *
+ * Context of the Queues in Capacity Scheduler.
+ *
+ */
+@Private
+@Unstable
+public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
+    CSQueue, CapacitySchedulerConfiguration>{
+
+  private static final Log LOG = LogFactory.getLog(
+      CapacitySchedulerQueueManager.class);
+
+  static final Comparator<CSQueue> NON_PARTITIONED_QUEUE_COMPARATOR =
+      new Comparator<CSQueue>() {
+    @Override
+    public int compare(CSQueue q1, CSQueue q2) {
+      if (q1.getUsedCapacity() < q2.getUsedCapacity()) {
+        return -1;
+      } else if (q1.getUsedCapacity() > q2.getUsedCapacity()) {
+        return 1;
+      }
+
+      return q1.getQueuePath().compareTo(q2.getQueuePath());
+    }
+  };
+
+  static final PartitionedQueueComparator PARTITIONED_QUEUE_COMPARATOR =
+      new PartitionedQueueComparator();
+
+  static class QueueHook {
+    public CSQueue hook(CSQueue queue) {
+      return queue;
+    }
+  }
+
+  private static final QueueHook NOOP = new QueueHook();
+  private CapacitySchedulerContext csContext;
+  private final YarnAuthorizationProvider authorizer;
+  private final Map<String, CSQueue> queues = new ConcurrentHashMap<>();
+  private CSQueue root;
+  private final RMNodeLabelsManager labelManager;
+
+  /**
+   * Construct the service.
+   * @param conf the configuration
+   * @param labelManager the labelManager
+   */
+  public CapacitySchedulerQueueManager(Configuration conf,
+      RMNodeLabelsManager labelManager) {
+    this.authorizer = YarnAuthorizationProvider.getInstance(conf);
+    this.labelManager = labelManager;
+  }
+
+  @Override
+  public CSQueue getRootQueue() {
+    return this.root;
+  }
+
+  @Override
+  public Map<String, CSQueue> getQueues() {
+    return queues;
+  }
+
+  @Override
+  public void removeQueue(String queueName) {
+    this.queues.remove(queueName);
+  }
+
+  @Override
+  public void addQueue(String queueName, CSQueue queue) {
+    this.queues.put(queueName, queue);
+  }
+
+  @Override
+  public CSQueue getQueue(String queueName) {
+    return queues.get(queueName);
+  }
+
+  /**
+   * Set the CapacitySchedulerContext.
+   * @param capacitySchedulerContext the CapacitySchedulerContext
+   */
+  public void setCapacitySchedulerContext(
+      CapacitySchedulerContext capacitySchedulerContext) {
+    this.csContext = capacitySchedulerContext;
+  }
+
+  /**
+   * Initialized the queues.
+   * @param conf the CapacitySchedulerConfiguration
+   * @throws IOException if fails to initialize queues
+   */
+  public void initializeQueues(CapacitySchedulerConfiguration conf)
+      throws IOException {
+    root = parseQueue(this.csContext, conf, null,
+        CapacitySchedulerConfiguration.ROOT, queues, queues, NOOP);
+    setQueueAcls(authorizer, queues);
+    labelManager.reinitializeQueueLabels(getQueueToLabels());
+    LOG.info("Initialized root queue " + root);
+  }
+
+  @Override
+  public void reinitializeQueues(CapacitySchedulerConfiguration newConf)
+      throws IOException {
+    // Parse new queues
+    Map<String, CSQueue> newQueues = new HashMap<>();
+    CSQueue newRoot =  parseQueue(this.csContext, newConf, null,
+        CapacitySchedulerConfiguration.ROOT, newQueues, queues, NOOP);
+
+    // Ensure all existing queues are still present
+    validateExistingQueues(queues, newQueues);
+
+    // Add new queues
+    addNewQueues(queues, newQueues);
+
+    // Re-configure queues
+    root.reinitialize(newRoot, this.csContext.getClusterResource());
+
+    setQueueAcls(authorizer, queues);
+
+    // Re-calculate headroom for active applications
+    Resource clusterResource = this.csContext.getClusterResource();
+    root.updateClusterResource(clusterResource, new ResourceLimits(
+        clusterResource));
+
+    labelManager.reinitializeQueueLabels(getQueueToLabels());
+  }
+
+  /**
+   * Parse the queue from the configuration.
+   * @param csContext the CapacitySchedulerContext
+   * @param conf the CapacitySchedulerConfiguration
+   * @param parent the parent queue
+   * @param queueName the queue name
+   * @param queues all the queues
+   * @param oldQueues the old queues
+   * @param hook the queue hook
+   * @return the CSQueue
+   * @throws IOException
+   */
+  static CSQueue parseQueue(
+      CapacitySchedulerContext csContext,
+      CapacitySchedulerConfiguration conf,
+      CSQueue parent, String queueName, Map<String, CSQueue> queues,
+      Map<String, CSQueue> oldQueues,
+      QueueHook hook) throws IOException {
+    CSQueue queue;
+    String fullQueueName =
+        (parent == null) ? queueName
+            : (parent.getQueuePath() + "." + queueName);
+    String[] childQueueNames = conf.getQueues(fullQueueName);
+    boolean isReservableQueue = conf.isReservable(fullQueueName);
+    if (childQueueNames == null || childQueueNames.length == 0) {
+      if (null == parent) {
+        throw new IllegalStateException(
+            "Queue configuration missing child queue names for " + queueName);
+      }
+      // Check if the queue will be dynamically managed by the Reservation
+      // system
+      if (isReservableQueue) {
+        queue =
+            new PlanQueue(csContext, queueName, parent,
+                oldQueues.get(queueName));
+      } else {
+        queue =
+            new LeafQueue(csContext, queueName, parent,
+                oldQueues.get(queueName));
+
+        // Used only for unit tests
+        queue = hook.hook(queue);
+      }
+    } else {
+      if (isReservableQueue) {
+        throw new IllegalStateException(
+            "Only Leaf Queues can be reservable for " + queueName);
+      }
+      ParentQueue parentQueue =
+          new ParentQueue(csContext, queueName, parent,
+              oldQueues.get(queueName));
+
+      // Used only for unit tests
+      queue = hook.hook(parentQueue);
+
+      List<CSQueue> childQueues = new ArrayList<>();
+      for (String childQueueName : childQueueNames) {
+        CSQueue childQueue =
+            parseQueue(csContext, conf, queue, childQueueName,
+              queues, oldQueues, hook);
+        childQueues.add(childQueue);
+      }
+      parentQueue.setChildQueues(childQueues);
+    }
+
+    if (queue instanceof LeafQueue && queues.containsKey(queueName)
+        && queues.get(queueName) instanceof LeafQueue) {
+      throw new IOException("Two leaf queues were named " + queueName
+          + ". Leaf queue names must be distinct");
+    }
+    queues.put(queueName, queue);
+
+    LOG.info("Initialized queue: " + queue);
+    return queue;
+  }
+
+  /**
+   * Ensure all existing queues are present. Queues cannot be deleted
+   * @param queues existing queues
+   * @param newQueues new queues
+   */
+  private void validateExistingQueues(
+      Map<String, CSQueue> queues, Map<String, CSQueue> newQueues)
+      throws IOException {
+    // check that all static queues are included in the newQueues list
+    for (Map.Entry<String, CSQueue> e : queues.entrySet()) {
+      if (!(e.getValue() instanceof ReservationQueue)) {
+        String queueName = e.getKey();
+        CSQueue oldQueue = e.getValue();
+        CSQueue newQueue = newQueues.get(queueName);
+        if (null == newQueue) {
+          throw new IOException(queueName + " cannot be found during refresh!");
+        } else if (!oldQueue.getQueuePath().equals(newQueue.getQueuePath())) {
+          throw new IOException(queueName + " is moved from:"
+              + oldQueue.getQueuePath() + " to:" + newQueue.getQueuePath()
+              + " after refresh, which is not allowed.");
+        }
+      }
+    }
+  }
+
+  /**
+   * Add the new queues (only) to our list of queues...
+   * ... be careful, do not overwrite existing queues.
+   * @param queues the existing queues
+   * @param newQueues the new queues
+   */
+  private void addNewQueues(
+      Map<String, CSQueue> queues, Map<String, CSQueue> newQueues) {
+    for (Map.Entry<String, CSQueue> e : newQueues.entrySet()) {
+      String queueName = e.getKey();
+      CSQueue queue = e.getValue();
+      if (!queues.containsKey(queueName)) {
+        queues.put(queueName, queue);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  /**
+   * Set the acls for the queues.
+   * @param authorizer the yarnAuthorizationProvider
+   * @param queues the queues
+   * @throws IOException if fails to set queue acls
+   */
+  public static void setQueueAcls(YarnAuthorizationProvider authorizer,
+      Map<String, CSQueue> queues) throws IOException {
+    List<Permission> permissions = new ArrayList<>();
+    for (CSQueue queue : queues.values()) {
+      AbstractCSQueue csQueue = (AbstractCSQueue) queue;
+      permissions.add(
+          new Permission(csQueue.getPrivilegedEntity(), csQueue.getACLs()));
+    }
+    authorizer.setPermission(permissions,
+        UserGroupInformation.getCurrentUser());
+  }
+
+  /**
+   * Check that the String provided in input is the name of an existing,
+   * LeafQueue, if successful returns the queue.
+   *
+   * @param queue the queue name
+   * @return the LeafQueue
+   * @throws YarnException if the queue does not exist or the queue
+   *           is not the type of LeafQueue.
+   */
+  public LeafQueue getAndCheckLeafQueue(String queue) throws YarnException {
+    CSQueue ret = this.getQueue(queue);
+    if (ret == null) {
+      throw new YarnException("The specified Queue: " + queue
+          + " doesn't exist");
+    }
+    if (!(ret instanceof LeafQueue)) {
+      throw new YarnException("The specified Queue: " + queue
+          + " is not a Leaf Queue.");
+    }
+    return (LeafQueue) ret;
+  }
+
+  /**
+   * Get the default priority of the queue.
+   * @param queueName the queue name
+   * @return the default priority of the queue
+   */
+  public Priority getDefaultPriorityForQueue(String queueName) {
+    Queue queue = getQueue(queueName);
+    if (null == queue || null == queue.getDefaultApplicationPriority()) {
+      // Return with default application priority
+      return Priority.newInstance(CapacitySchedulerConfiguration
+          .DEFAULT_CONFIGURATION_APPLICATION_PRIORITY);
+    }
+    return Priority.newInstance(queue.getDefaultApplicationPriority()
+        .getPriority());
+  }
+
+  /**
+   * Get a map of queueToLabels.
+   * @return the map of queueToLabels
+   */
+  private Map<String, Set<String>> getQueueToLabels() {
+    Map<String, Set<String>> queueToLabels = new HashMap<>();
+    for (CSQueue queue :  getQueues().values()) {
+      queueToLabels.put(queue.getQueueName(), queue.getAccessibleNodeLabels());
+    }
+    return queueToLabels;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.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/scheduler/capacity/TestApplicationLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
index 11e94b9..7382f3d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
@@ -111,7 +111,8 @@ public class TestApplicationLimits {
     when(csContext.getClusterResource()).
         thenReturn(Resources.createResource(10 * 16 * GB, 10 * 32));
     when(csContext.getNonPartitionedQueueComparator()).
-        thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+        thenReturn(
+            CapacitySchedulerQueueManager.NON_PARTITIONED_QUEUE_COMPARATOR);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
@@ -123,9 +124,9 @@ public class TestApplicationLimits {
         containerTokenSecretManager);
 
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
-    CSQueue root = 
-        CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
-            queues, queues, 
+    CSQueue root = CapacitySchedulerQueueManager
+        .parseQueue(csContext, csConf, null, "root",
+            queues, queues,
             TestUtils.spyHook);
 
     
@@ -276,7 +277,8 @@ public class TestApplicationLimits {
     when(csContext.getMaximumResourceCapability()).
         thenReturn(Resources.createResource(16*GB, 16));
     when(csContext.getNonPartitionedQueueComparator()).
-        thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+        thenReturn(
+            CapacitySchedulerQueueManager.NON_PARTITIONED_QUEUE_COMPARATOR);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
     when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
@@ -288,8 +290,8 @@ public class TestApplicationLimits {
     
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root = 
-        CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
-            queues, queues, TestUtils.spyHook);
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
+            "root", queues, queues, TestUtils.spyHook);
 
     LeafQueue queue = (LeafQueue)queues.get(A);
     
@@ -356,9 +358,9 @@ public class TestApplicationLimits {
         + ".maximum-am-resource-percent", 0.5f);
     // Re-create queues to get new configs.
     queues = new HashMap<String, CSQueue>();
-    root = 
-        CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
-            queues, queues, TestUtils.spyHook);
+    root = CapacitySchedulerQueueManager.parseQueue(
+        csContext, csConf, null, "root",
+        queues, queues, TestUtils.spyHook);
     clusterResource = Resources.createResource(100 * 16 * GB);
 
     queue = (LeafQueue)queues.get(A);
@@ -378,9 +380,9 @@ public class TestApplicationLimits {
         9999);
     // Re-create queues to get new configs.
     queues = new HashMap<String, CSQueue>();
-    root = 
-        CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
-            queues, queues, TestUtils.spyHook);
+    root = CapacitySchedulerQueueManager.parseQueue(
+        csContext, csConf, null, "root",
+        queues, queues, TestUtils.spyHook);
 
     queue = (LeafQueue)queues.get(A);
     assertEquals(9999, (int)csConf.getMaximumApplicationsPerQueue(queue.getQueuePath()));
@@ -580,7 +582,8 @@ public class TestApplicationLimits {
     when(csContext.getMaximumResourceCapability()).
         thenReturn(Resources.createResource(16*GB));
     when(csContext.getNonPartitionedQueueComparator()).
-        thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+        thenReturn(
+            CapacitySchedulerQueueManager.NON_PARTITIONED_QUEUE_COMPARATOR);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
     
@@ -589,8 +592,8 @@ public class TestApplicationLimits {
     when(csContext.getClusterResource()).thenReturn(clusterResource);
     
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
-    CSQueue rootQueue = CapacityScheduler.parseQueue(csContext, csConf, null,
-        "root", queues, queues, TestUtils.spyHook);
+    CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(csContext,
+        csConf, null, "root", queues, queues, TestUtils.spyHook);
 
     ResourceUsage queueCapacities = rootQueue.getQueueResourceUsage();
     when(csContext.getClusterResourceUsage())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimitsByPartition.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/scheduler/capacity/TestApplicationLimitsByPartition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimitsByPartition.java
index d335552..5c53fda 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimitsByPartition.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimitsByPartition.java
@@ -595,7 +595,8 @@ public class TestApplicationLimitsByPartition {
     when(csContext.getMaximumResourceCapability())
         .thenReturn(Resources.createResource(16 * GB));
     when(csContext.getNonPartitionedQueueComparator())
-        .thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+        .thenReturn(
+            CapacitySchedulerQueueManager.NON_PARTITIONED_QUEUE_COMPARATOR);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     RMContext rmContext = TestUtils.getMockRMContext();
     RMContext spyRMContext = spy(rmContext);
@@ -614,8 +615,8 @@ public class TestApplicationLimitsByPartition {
     when(csContext.getClusterResource()).thenReturn(clusterResource);
 
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
-    CSQueue rootQueue = CapacityScheduler.parseQueue(csContext, csConf, null,
-        "root", queues, queues, TestUtils.spyHook);
+    CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(csContext,
+        csConf, null, "root", queues, queues, TestUtils.spyHook);
 
     ResourceUsage queueResUsage = rootQueue.getQueueResourceUsage();
     when(csContext.getClusterResourceUsage())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.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/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
index 40e5d2a..a6ae0c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
@@ -95,11 +95,12 @@ public class TestChildQueueOrder {
     when(csContext.getMaximumResourceCapability()).thenReturn(
         Resources.createResource(16*GB, 32));
     when(csContext.getClusterResource()).
-    thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
+        thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getNonPartitionedQueueComparator()).
-    thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+        thenReturn(
+            CapacitySchedulerQueueManager.NON_PARTITIONED_QUEUE_COMPARATOR);
     when(csContext.getResourceCalculator()).
-    thenReturn(resourceComparator);
+        thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);
     when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
   }
@@ -222,7 +223,7 @@ public class TestChildQueueOrder {
     setupSortedQueues(csConf);
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root = 
-      CapacityScheduler.parseQueue(csContext, csConf, null, 
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
           CapacitySchedulerConfiguration.ROOT, queues, queues, 
           TestUtils.spyHook);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.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/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index 8694efb..2ce5fcb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -175,7 +175,8 @@ public class TestLeafQueue {
     when(csContext.getClusterResource()).
         thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getNonPartitionedQueueComparator()).
-        thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+        thenReturn(
+            CapacitySchedulerQueueManager.NON_PARTITIONED_QUEUE_COMPARATOR);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
     when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
@@ -188,7 +189,7 @@ public class TestLeafQueue {
         containerTokenSecretManager);
 
     root = 
-        CapacityScheduler.parseQueue(csContext, csConf, null, 
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
             CapacitySchedulerConfiguration.ROOT, 
             queues, queues, 
             TestUtils.spyHook);
@@ -2380,7 +2381,7 @@ public class TestLeafQueue {
         .DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT * 2);
     Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
     CSQueue newRoot =
-        CapacityScheduler.parseQueue(csContext, csConf, null,
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
             CapacitySchedulerConfiguration.ROOT,
             newQueues, queues,
             TestUtils.spyHook);
@@ -2405,7 +2406,7 @@ public class TestLeafQueue {
         .NODE_LOCALITY_DELAY, 60);
     Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
     CSQueue newRoot =
-        CapacityScheduler.parseQueue(csContext, csConf, null,
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
             CapacitySchedulerConfiguration.ROOT,
             newQueues, queues,
             TestUtils.spyHook);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.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/scheduler/capacity/TestParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
index d875969..a36db44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
@@ -97,10 +97,11 @@ public class TestParentQueue {
     when(csContext.getClusterResource()).
         thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getNonPartitionedQueueComparator()).
-    thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+        thenReturn(
+            CapacitySchedulerQueueManager.NON_PARTITIONED_QUEUE_COMPARATOR);
     when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getResourceCalculator()).
-    thenReturn(resourceComparator);
+        thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);
   }
   
@@ -231,7 +232,7 @@ public class TestParentQueue {
     
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root =
-        CapacityScheduler.parseQueue(csContext, csConf, null,
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
             TestUtils.spyHook);
 
@@ -346,7 +347,7 @@ public class TestParentQueue {
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     boolean exceptionOccured = false;
     try {
-      CapacityScheduler.parseQueue(csContext, csConf, null,
+      CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
           CapacitySchedulerConfiguration.ROOT, queues, queues,
           TestUtils.spyHook);
     } catch (IllegalArgumentException ie) {
@@ -360,7 +361,7 @@ public class TestParentQueue {
     exceptionOccured = false;
     queues.clear();
     try {
-      CapacityScheduler.parseQueue(csContext, csConf, null,
+      CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
           CapacitySchedulerConfiguration.ROOT, queues, queues,
           TestUtils.spyHook);
     } catch (IllegalArgumentException ie) {
@@ -374,7 +375,7 @@ public class TestParentQueue {
     exceptionOccured = false;
     queues.clear();
     try {
-      CapacityScheduler.parseQueue(csContext, csConf, null,
+      CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
           CapacitySchedulerConfiguration.ROOT, queues, queues,
           TestUtils.spyHook);
     } catch (IllegalArgumentException ie) {
@@ -467,7 +468,7 @@ public class TestParentQueue {
     
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root =
-        CapacityScheduler.parseQueue(csContext, csConf, null,
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
             CapacitySchedulerConfiguration.ROOT, queues, queues,
             TestUtils.spyHook);
     
@@ -623,8 +624,8 @@ public class TestParentQueue {
     csConf.setCapacity(Q_B + "." + B3, 0);
     
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>(); 
-    CapacityScheduler.parseQueue(csContext, csConf, null, 
-        CapacitySchedulerConfiguration.ROOT, queues, queues, 
+    CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
+        CapacitySchedulerConfiguration.ROOT, queues, queues,
         TestUtils.spyHook);
   }
   
@@ -640,8 +641,8 @@ public class TestParentQueue {
     csConf.setCapacity(Q_A, 60);
 
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>(); 
-    CapacityScheduler.parseQueue(csContext, csConf, null, 
-        CapacitySchedulerConfiguration.ROOT, queues, queues, 
+    CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
+        CapacitySchedulerConfiguration.ROOT, queues, queues,
         TestUtils.spyHook);
   }
   
@@ -662,8 +663,8 @@ public class TestParentQueue {
 
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>(); 
     try {
-      CapacityScheduler.parseQueue(csContext, csConf, null, 
-          CapacitySchedulerConfiguration.ROOT, queues, queues, 
+      CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
+          CapacitySchedulerConfiguration.ROOT, queues, queues,
           TestUtils.spyHook);
     } catch (IllegalArgumentException e) {
       fail("Failed to create queues with 0 capacity: " + e);
@@ -678,7 +679,7 @@ public class TestParentQueue {
 
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root =
-        CapacityScheduler.parseQueue(csContext, csConf, null,
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
             CapacitySchedulerConfiguration.ROOT, queues, queues,
             TestUtils.spyHook);
 
@@ -754,8 +755,8 @@ public class TestParentQueue {
     //B3
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root = 
-        CapacityScheduler.parseQueue(csContext, csConf, null, 
-            CapacitySchedulerConfiguration.ROOT, queues, queues, 
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
+            CapacitySchedulerConfiguration.ROOT, queues, queues,
             TestUtils.spyHook);
 
     // Setup some nodes
@@ -850,12 +851,12 @@ public class TestParentQueue {
 
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root = 
-        CapacityScheduler.parseQueue(csContext, csConf, null, 
-            CapacitySchedulerConfiguration.ROOT, queues, queues, 
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
+            CapacitySchedulerConfiguration.ROOT, queues, queues,
             TestUtils.spyHook);
     YarnAuthorizationProvider authorizer =
         YarnAuthorizationProvider.getInstance(conf);
-    CapacityScheduler.setQueueAcls(authorizer, queues);
+    CapacitySchedulerQueueManager.setQueueAcls(authorizer, queues);
 
     UserGroupInformation user = UserGroupInformation.getCurrentUser();
     // Setup queue configs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.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/scheduler/capacity/TestReservations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
index f6caa50..3e05456 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
@@ -134,7 +134,7 @@ public class TestReservations {
     when(csContext.getClusterResource()).thenReturn(
         Resources.createResource(100 * 16 * GB, 100 * 12));
     when(csContext.getNonPartitionedQueueComparator()).thenReturn(
-        CapacityScheduler.nonPartitionedQueueComparator);
+        CapacitySchedulerQueueManager.NON_PARTITIONED_QUEUE_COMPARATOR);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
@@ -144,7 +144,7 @@ public class TestReservations {
     when(csContext.getContainerTokenSecretManager()).thenReturn(
         containerTokenSecretManager);
 
-    root = CapacityScheduler.parseQueue(csContext, csConf, null,
+    root = CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
         CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook);
 
     ResourceUsage queueResUsage = root.getQueueResourceUsage();
@@ -1180,8 +1180,8 @@ public class TestReservations {
     csConf.setBoolean(
         CapacitySchedulerConfiguration.RESERVE_CONT_LOOK_ALL_NODES, false);
     Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
-    CSQueue newRoot = CapacityScheduler.parseQueue(csContext, csConf, null,
-        CapacitySchedulerConfiguration.ROOT, newQueues, queues,
+    CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext,
+        csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues,
         TestUtils.spyHook);
     queues = newQueues;
     root.reinitialize(newRoot, cs.getClusterResource());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69fb70c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.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/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
index e34ee34..b982fab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
@@ -141,7 +141,7 @@ public class TestUtils {
   /**
    * Hook to spy on queues.
    */
-  static class SpyHook extends CapacityScheduler.QueueHook {
+  static class SpyHook extends CapacitySchedulerQueueManager.QueueHook {
     @Override
     public CSQueue hook(CSQueue queue) {
       return spy(queue);


---------------------------------------------------------------------
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-13840. Implement getUsed() for ViewFileSystem. Contributed by Manoj Govindassamy.

Posted by xg...@apache.org.
HADOOP-13840. Implement getUsed() for ViewFileSystem. Contributed by Manoj Govindassamy.


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

Branch: refs/heads/YARN-5734
Commit: 1f7613be958bbdb735fd2b49e3f0b48e2c8b7c13
Parents: 7226a71
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Nov 30 17:55:12 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Nov 30 17:55:12 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java | 18 ++++++++++++
 .../fs/viewfs/ViewFileSystemBaseTest.java       | 29 ++++++++++++++++++++
 2 files changed, 47 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f7613be/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index ed1bda2..8be666c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -859,6 +859,24 @@ public class ViewFileSystem extends FileSystem {
   }
 
   /**
+   * Return the total size of all files under "/", if {@link
+   * Constants#CONFIG_VIEWFS_LINK_MERGE_SLASH} is supported and is a valid
+   * mount point. Else, throw NotInMountpointException.
+   *
+   * @throws IOException
+   */
+  @Override
+  public long getUsed() throws IOException {
+    InodeTree.ResolveResult<FileSystem> res = fsState.resolve(
+        getUriPath(InodeTree.SlashPath), true);
+    if (res.isInternalDir()) {
+      throw new NotInMountpointException(InodeTree.SlashPath, "getUsed");
+    } else {
+      return res.targetFileSystem.getUsed();
+    }
+  }
+
+  /**
    * An instance of this class represents an internal dir of the viewFs
    * that is internal dir of the mount table.
    * It is a read only mount tables and create, mkdir or delete operations

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f7613be/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
index 06f9868..9a0bf02 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
@@ -1108,4 +1108,33 @@ abstract public class ViewFileSystemBaseTest {
       }
     });
   }
+
+  @Test
+  public void testUsed() throws IOException {
+    try {
+      fsView.getUsed();
+      fail("ViewFileSystem getUsed() should fail for slash root path when the" +
+          " slash root mount point is not configured.");
+    } catch (NotInMountpointException e) {
+      // expected exception.
+    }
+    long usedSpaceByPathViaViewFs = fsView.getUsed(new Path("/user"));
+    long usedSpaceByPathViaTargetFs =
+        fsTarget.getUsed(new Path(targetTestRoot, "user"));
+    assertEquals("Space used not matching between ViewFileSystem and " +
+        "the mounted FileSystem!",
+        usedSpaceByPathViaTargetFs, usedSpaceByPathViaViewFs);
+
+    Path mountDataRootPath = new Path("/data");
+    String fsTargetFileName = "debug.log";
+    Path fsTargetFilePath = new Path(targetTestRoot, "data/debug.log");
+    Path mountDataFilePath = new Path(mountDataRootPath, fsTargetFileName);
+    fileSystemTestHelper.createFile(fsTarget, fsTargetFilePath);
+
+    usedSpaceByPathViaViewFs = fsView.getUsed(mountDataFilePath);
+    usedSpaceByPathViaTargetFs = fsTarget.getUsed(fsTargetFilePath);
+    assertEquals("Space used not matching between ViewFileSystem and " +
+        "the mounted FileSystem!",
+        usedSpaceByPathViaTargetFs, usedSpaceByPathViaViewFs);
+  }
 }


---------------------------------------------------------------------
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: YARN-5929. Missing scheduling policy in the FS queue metric. (Contributed by Yufei Gu via Daniel Templeton)

Posted by xg...@apache.org.
YARN-5929. Missing scheduling policy in the FS queue metric. (Contributed by Yufei Gu via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: 5bd18c49bd5075fa20d24363dceea7828e3fa266
Parents: 2ff84a0
Author: Daniel Templeton <te...@apache.org>
Authored: Fri Dec 2 13:35:09 2016 -0800
Committer: Daniel Templeton <te...@apache.org>
Committed: Fri Dec 2 13:55:42 2016 -0800

----------------------------------------------------------------------
 .../scheduler/fair/FSQueueMetrics.java          | 32 +++++++--
 .../scheduler/fair/TestFSQueueMetrics.java      | 69 ++++++++++++++++++++
 2 files changed, 97 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bd18c49/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueMetrics.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/scheduler/fair/FSQueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueMetrics.java
index a970815..ca375f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueMetrics.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.annotation.Metric;
@@ -169,6 +170,12 @@ public class FSQueueMetrics extends QueueMetrics {
     amResourceUsageVCores.set(resource.getVirtualCores());
   }
 
+  /**
+   * Get the scheduling policy.
+   *
+   * @return the scheduling policy
+   */
+  @Metric("Scheduling policy")
   public String getSchedulingPolicy() {
     return schedulingPolicy;
   }
@@ -181,21 +188,38 @@ public class FSQueueMetrics extends QueueMetrics {
   static FSQueueMetrics forQueue(String queueName, Queue parent,
       boolean enableUserMetrics, Configuration conf) {
     MetricsSystem ms = DefaultMetricsSystem.instance();
+    return forQueue(ms, queueName, parent, enableUserMetrics, conf);
+  }
+
+  /**
+   * Get the FS queue metric for the given queue. Create one and register it to
+   * metrics system if there isn't one for the queue.
+   *
+   * @param ms the metric system
+   * @param queueName queue name
+   * @param parent parent queue
+   * @param enableUserMetrics  if user metrics is needed
+   * @param conf configuration
+   * @return a FSQueueMetrics object
+   */
+  @VisibleForTesting
+  public synchronized
+  static FSQueueMetrics forQueue(MetricsSystem ms, String queueName,
+      Queue parent, boolean enableUserMetrics, Configuration conf) {
     QueueMetrics metrics = queueMetrics.get(queueName);
     if (metrics == null) {
       metrics = new FSQueueMetrics(ms, queueName, parent, enableUserMetrics, conf)
           .tag(QUEUE_INFO, queueName);
-      
+
       // Register with the MetricsSystems
       if (ms != null) {
         metrics = ms.register(
-                sourceName(queueName).toString(), 
-                "Metrics for queue: " + queueName, metrics);
+            sourceName(queueName).toString(),
+            "Metrics for queue: " + queueName, metrics);
       }
       queueMetrics.put(queueName, metrics);
     }
 
     return (FSQueueMetrics)metrics;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bd18c49/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.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/scheduler/fair/TestFSQueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.java
new file mode 100644
index 0000000..7ccfbc3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.java
@@ -0,0 +1,69 @@
+/**
+ * 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.yarn.server.resourcemanager.scheduler.fair;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
+import org.apache.hadoop.metrics2.impl.MetricsRecords;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestQueueMetrics;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * The test class for {@link FSQueueMetrics}.
+ */
+public class TestFSQueueMetrics {
+  private static final Configuration CONF = new Configuration();
+
+  private MetricsSystem ms;
+
+  @Before public void setUp() {
+    ms = new MetricsSystemImpl();
+    QueueMetrics.clearQueueMetrics();
+  }
+
+  /**
+   * Test if the metric scheduling policy is set correctly.
+   */
+  @Test
+  public void testSchedulingPolicy() {
+    String queueName = "single";
+
+    FSQueueMetrics metrics = FSQueueMetrics.forQueue(ms, queueName, null, false,
+        CONF);
+    metrics.setSchedulingPolicy("drf");
+    checkSchedulingPolicy(queueName, "drf");
+
+    // test resetting the scheduling policy
+    metrics.setSchedulingPolicy("fair");
+    checkSchedulingPolicy(queueName, "fair");
+  }
+
+  private void checkSchedulingPolicy(String queueName, String policy) {
+    MetricsSource queueSource = TestQueueMetrics.queueSource(ms, queueName);
+    MetricsCollectorImpl collector = new MetricsCollectorImpl();
+    queueSource.getMetrics(collector, true);
+    MetricsRecords.assertTag(collector.getRecords().get(0), "SchedulingPolicy",
+        policy);
+  }
+}


---------------------------------------------------------------------
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: HADOOP-13257. Improve Azure Data Lake contract tests. Contributed by Vishwajeet Dusane

Posted by xg...@apache.org.
HADOOP-13257. Improve Azure Data Lake contract tests. 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/4113ec5f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4113ec5f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4113ec5f

Branch: refs/heads/YARN-5734
Commit: 4113ec5fa5ca049ebaba039b1faf3911c6a34f7b
Parents: 51211a7
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Dec 2 15:54:57 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Fri Dec 2 15:54:57 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |  24 +-
 .../org/apache/hadoop/fs/adl/TestAdlRead.java   |   6 +-
 .../apache/hadoop/fs/adl/TestListStatus.java    |   6 +-
 .../fs/adl/live/TestAdlContractAppendLive.java  |  11 +-
 .../fs/adl/live/TestAdlContractConcatLive.java  |  23 +-
 .../fs/adl/live/TestAdlContractCreateLive.java  |  19 +-
 .../fs/adl/live/TestAdlContractDeleteLive.java  |  11 +-
 .../live/TestAdlContractGetFileStatusLive.java  |  36 ++
 .../fs/adl/live/TestAdlContractMkdirLive.java   |  25 +-
 .../fs/adl/live/TestAdlContractOpenLive.java    |  11 +-
 .../fs/adl/live/TestAdlContractRenameLive.java  |  30 +-
 .../fs/adl/live/TestAdlContractRootDirLive.java |  19 +-
 .../fs/adl/live/TestAdlContractSeekLive.java    |  11 +-
 .../live/TestAdlDifferentSizeWritesLive.java    |  69 ++--
 .../live/TestAdlFileContextCreateMkdirLive.java |  67 ++++
 .../TestAdlFileContextMainOperationsLive.java   |  99 ++++++
 .../adl/live/TestAdlFileSystemContractLive.java |  57 +---
 .../live/TestAdlInternalCreateNonRecursive.java | 134 ++++++++
 .../fs/adl/live/TestAdlPermissionLive.java      | 116 +++++++
 .../adl/live/TestAdlSupportedCharsetInPath.java | 334 +++++++++++++++++++
 .../apache/hadoop/fs/adl/live/TestMetadata.java | 111 ++++++
 21 files changed, 995 insertions(+), 224 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
index 9083afc..bd43c52 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -346,7 +346,6 @@ public class AdlFileSystem extends FileSystem {
    * @see #setPermission(Path, FsPermission)
    * @deprecated API only for 0.20-append
    */
-  @Deprecated
   @Override
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       EnumSet<CreateFlag> flags, int bufferSize, short replication,
@@ -471,6 +470,10 @@ public class AdlFileSystem extends FileSystem {
   @Override
   public boolean rename(final Path src, final Path dst) throws IOException {
     statistics.incrementWriteOps(1);
+    if (toRelativeFilePath(src).equals("/")) {
+      return false;
+    }
+
     return adlClient.rename(toRelativeFilePath(src), toRelativeFilePath(dst));
   }
 
@@ -522,9 +525,24 @@ public class AdlFileSystem extends FileSystem {
   public boolean delete(final Path path, final boolean recursive)
       throws IOException {
     statistics.incrementWriteOps(1);
+    String relativePath = toRelativeFilePath(path);
+    // Delete on root directory not supported.
+    if (relativePath.equals("/")) {
+      // This is important check after recent commit
+      // HADOOP-12977 and HADOOP-13716 validates on root for
+      // 1. if root is empty and non recursive delete then return false.
+      // 2. if root is non empty and non recursive delete then throw exception.
+      if (!recursive
+          && adlClient.enumerateDirectory(toRelativeFilePath(path), 1).size()
+          > 0) {
+        throw new IOException("Delete on root is not supported.");
+      }
+      return false;
+    }
+
     return recursive ?
-        adlClient.deleteRecursive(toRelativeFilePath(path)) :
-        adlClient.delete(toRelativeFilePath(path));
+        adlClient.deleteRecursive(relativePath) :
+        adlClient.delete(relativePath);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java
index 734256a..172663c 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java
@@ -102,7 +102,7 @@ public class TestAdlRead extends AdlMockWebServer {
       n += count;
     }
 
-    Assert.assertEquals(testData.getActualData().length, expectedData.length);
+    Assert.assertEquals(expectedData.length, testData.getActualData().length);
     Assert.assertArrayEquals(expectedData, testData.getActualData());
     in.close();
     if (testData.isCheckOfNoOfCalls()) {
@@ -119,8 +119,8 @@ public class TestAdlRead extends AdlMockWebServer {
     for (int i = 0; i < 1000; ++i) {
       int position = random.nextInt(testData.getActualData().length);
       in.seek(position);
-      Assert.assertEquals(in.getPos(), position);
-      Assert.assertEquals(in.read(), testData.getActualData()[position] & 0xFF);
+      Assert.assertEquals(position, in.getPos());
+      Assert.assertEquals(testData.getActualData()[position] & 0xFF, in.read());
     }
     in.close();
     if (testData.isCheckOfNoOfCalls()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
index dd27a10..c151e89 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
@@ -50,7 +50,7 @@ public class TestListStatus extends AdlMockWebServer {
         .listStatus(new Path("/test1/test2"));
     long endTime = Time.monotonicNow();
     LOG.debug("Time : " + (endTime - startTime));
-    Assert.assertEquals(ls.length, 10);
+    Assert.assertEquals(10, ls.length);
 
     getMockServer().enqueue(new MockResponse().setResponseCode(200)
         .setBody(TestADLResponseData.getListFileStatusJSONResponse(200)));
@@ -58,7 +58,7 @@ public class TestListStatus extends AdlMockWebServer {
     ls = getMockAdlFileSystem().listStatus(new Path("/test1/test2"));
     endTime = Time.monotonicNow();
     LOG.debug("Time : " + (endTime - startTime));
-    Assert.assertEquals(ls.length, 200);
+    Assert.assertEquals(200, ls.length);
 
     getMockServer().enqueue(new MockResponse().setResponseCode(200)
         .setBody(TestADLResponseData.getListFileStatusJSONResponse(2048)));
@@ -66,7 +66,7 @@ public class TestListStatus extends AdlMockWebServer {
     ls = getMockAdlFileSystem().listStatus(new Path("/test1/test2"));
     endTime = Time.monotonicNow();
     LOG.debug("Time : " + (endTime - startTime));
-    Assert.assertEquals(ls.length, 2048);
+    Assert.assertEquals(2048, ls.length);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
index 83390af..ffe6dd3 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
@@ -23,11 +23,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
 import org.junit.Test;
 
 /**
- * Verify Adls APPEND semantics compliance with Hadoop.
+ * Test Append on Adl file system.
  */
 public class TestAdlContractAppendLive extends AbstractContractAppendTest {
 
@@ -42,12 +41,4 @@ public class TestAdlContractAppendLive extends AbstractContractAppendTest {
     ContractTestUtils.unsupported("Skipping since renaming file in append "
         + "mode not supported in Adl");
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
index 8474e9c..60d30ac 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
@@ -20,14 +20,15 @@
 package org.apache.hadoop.fs.adl.live;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.AbstractContractConcatTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
 import org.junit.Test;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
 /**
- * Verify Adls CONCAT semantics compliance with Hadoop.
+ * Test concat on Adl file system.
  */
 public class TestAdlContractConcatLive extends AbstractContractConcatTest {
 
@@ -36,17 +37,13 @@ public class TestAdlContractConcatLive extends AbstractContractConcatTest {
     return new AdlStorageContract(configuration);
   }
 
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
   @Test
   public void testConcatMissingTarget() throws Throwable {
-    ContractTestUtils.unsupported("BUG : Adl to support expectation from "
-        + "concat on missing targets.");
+    Path testPath = path("test");
+    Path zeroByteFile = new Path(testPath, "zero.txt");
+    Path target = new Path(testPath, "target");
+    touch(getFileSystem(), zeroByteFile);
+    // Concat on missing target is allowed on Adls file system.
+    getFileSystem().concat(target, new Path[] {zeroByteFile});
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
index 907c50c..06347e9 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
@@ -22,12 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Verify Adls CREATE semantics compliance with Hadoop.
+ * Test creating files, overwrite options.
  */
 public class TestAdlContractCreateLive extends AbstractContractCreateTest {
 
@@ -35,18 +32,4 @@ public class TestAdlContractCreateLive extends AbstractContractCreateTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
-  @Test
-  public void testOverwriteEmptyDirectory() throws Throwable {
-    ContractTestUtils
-        .unsupported("BUG : Adl to support override empty " + "directory.");
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
index 30eaec7..6961f15 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
@@ -22,10 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.junit.Before;
 
 /**
- * Verify Adls DELETE semantics compliance with Hadoop.
+ * Test delete contract test.
  */
 public class TestAdlContractDeleteLive extends AbstractContractDeleteTest {
 
@@ -33,12 +32,4 @@ public class TestAdlContractDeleteLive extends AbstractContractDeleteTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractGetFileStatusLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractGetFileStatusLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractGetFileStatusLive.java
new file mode 100644
index 0000000..d50dd68
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractGetFileStatusLive.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test getFileStatus contract test.
+ */
+public class TestAdlContractGetFileStatusLive extends
+    AbstractContractGetFileStatusTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
index e498110..5e760c5 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
@@ -22,34 +22,13 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Verify Adls MKDIR semantics compliance with Hadoop.
+ * Test Mkdir contract on Adl storage file system.
  */
 public class TestAdlContractMkdirLive extends AbstractContractMkdirTest {
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new AdlStorageContract(conf);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
-  @Test
-  public void testMkdirOverParentFile() throws Throwable {
-    ContractTestUtils.unsupported("Not supported by Adl");
-  }
-
-  @Test
-  public void testNoMkdirOverFile() throws Throwable {
-    ContractTestUtils.unsupported("Not supported by Adl");
-  }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
index 2bb2095..7a35d2c 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
@@ -22,10 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.junit.Before;
 
 /**
- * Verify Adls OPEN/READ semantics compliance with Hadoop.
+ * Test OPEN - read API.
  */
 public class TestAdlContractOpenLive extends AbstractContractOpenTest {
 
@@ -33,12 +32,4 @@ public class TestAdlContractOpenLive extends AbstractContractOpenTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
index 06063c5..d72d35e 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
@@ -22,12 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Verify Adls RENAME semantics compliance with Hadoop.
+ * Test rename contract test cases on Adl file system.
  */
 public class TestAdlContractRenameLive extends AbstractContractRenameTest {
 
@@ -35,29 +32,4 @@ public class TestAdlContractRenameLive extends AbstractContractRenameTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
-  @Test
-  public void testRenameFileOverExistingFile() throws Throwable {
-    ContractTestUtils
-        .unsupported("BUG : Adl to support full complete POSIX" + "behaviour");
-  }
-
-  @Test
-  public void testRenameFileNonexistentDir() throws Throwable {
-    ContractTestUtils
-        .unsupported("BUG : Adl to support create dir is not " + "exist");
-  }
-
-  @Test
-  public void testRenameWithNonEmptySubDir() throws Throwable {
-    ContractTestUtils.unsupported("BUG : Adl to support non empty dir move.");
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
index bf4e549..8ebc632 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
@@ -22,12 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Verify Adls root level operation support.
+ * Test operation on root level.
  */
 public class TestAdlContractRootDirLive
     extends AbstractContractRootDirectoryTest {
@@ -35,18 +32,4 @@ public class TestAdlContractRootDirLive
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
-  @Test
-  public void testRmNonEmptyRootDirNonRecursive() throws Throwable {
-    ContractTestUtils.unsupported(
-        "BUG : Adl should throw exception instred " + "of returning false.");
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
index 0976464..62423b6 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
@@ -22,10 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.junit.Before;
 
 /**
- * Verify Adls OPEN/READ seek operation support.
+ * Test seek operation on Adl file system.
  */
 public class TestAdlContractSeekLive extends AbstractContractSeekTest {
 
@@ -33,12 +32,4 @@ public class TestAdlContractSeekLive extends AbstractContractSeekTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
index 8f53400..5421e0b 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
@@ -23,27 +23,63 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.common.Parallelized;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.io.IOException;
 import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Random;
+import java.util.UUID;
+
+import static org.apache.hadoop.fs.adl.AdlConfKeys.WRITE_BUFFER_SIZE_KEY;
 
 /**
- * Verify different data segment size writes ensure the integrity and
- * order of the data.
+ * Verify data integrity with different data sizes with buffer size.
  */
+@RunWith(Parallelized.class)
 public class TestAdlDifferentSizeWritesLive {
+  private static Random rand = new Random();
+  private int totalSize;
+  private int chunkSize;
+
+  public TestAdlDifferentSizeWritesLive(int totalSize, int chunkSize) {
+    this.totalSize = totalSize;
+    this.chunkSize = chunkSize;
+  }
 
   public static byte[] getRandomByteArrayData(int size) {
     byte[] b = new byte[size];
-    Random rand = new Random();
     rand.nextBytes(b);
     return b;
   }
 
+  @Parameterized.Parameters(name = "{index}: Data Size [{0}] ; Chunk Size "
+      + "[{1}]")
+  public static Collection testDataForIntegrityTest() {
+    return Arrays.asList(
+        new Object[][] {{4 * 1024, 1 * 1024}, {4 * 1024, 7 * 1024},
+            {4 * 1024, 10}, {2 * 1024, 10}, {1 * 1024, 10}, {100, 1},
+            {4 * 1024, 1 * 1024}, {7 * 1024, 2 * 1024}, {9 * 1024, 2 * 1024},
+            {10 * 1024, 3 * 1024}, {10 * 1024, 1 * 1024},
+            {10 * 1024, 8 * 1024}});
+  }
+
+  @BeforeClass
+  public static void cleanUpParent() throws IOException, URISyntaxException {
+    if (AdlStorageConfiguration.isContractTestEnabled()) {
+      Path path = new Path("/test/dataIntegrityCheck/");
+      FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+      fs.delete(path, true);
+    }
+  }
+
   @Before
   public void setup() throws Exception {
     org.junit.Assume
@@ -51,32 +87,17 @@ public class TestAdlDifferentSizeWritesLive {
   }
 
   @Test
-  public void testSmallDataWrites() throws IOException {
-    testDataIntegrity(4 * 1024 * 1024, 1 * 1024);
-    testDataIntegrity(4 * 1024 * 1024, 7 * 1024);
-    testDataIntegrity(4 * 1024 * 1024, 10);
-    testDataIntegrity(2 * 1024 * 1024, 10);
-    testDataIntegrity(1 * 1024 * 1024, 10);
-    testDataIntegrity(100, 1);
-  }
-
-  @Test
-  public void testMediumDataWrites() throws IOException {
-    testDataIntegrity(4 * 1024 * 1024, 1 * 1024 * 1024);
-    testDataIntegrity(7 * 1024 * 1024, 2 * 1024 * 1024);
-    testDataIntegrity(9 * 1024 * 1024, 2 * 1024 * 1024);
-    testDataIntegrity(10 * 1024 * 1024, 3 * 1024 * 1024);
-  }
-
-  private void testDataIntegrity(int totalSize, int chunkSize)
-      throws IOException {
-    Path path = new Path("/test/dataIntegrityCheck");
+  public void testDataIntegrity() throws IOException {
+    Path path = new Path(
+        "/test/dataIntegrityCheck/" + UUID.randomUUID().toString());
     FileSystem fs = null;
+    AdlStorageConfiguration.getConfiguration()
+        .setInt(WRITE_BUFFER_SIZE_KEY, 4 * 1024);
     try {
       fs = AdlStorageConfiguration.createStorageConnector();
     } catch (URISyntaxException e) {
       throw new IllegalStateException("Can not initialize ADL FileSystem. "
-          + "Please check fs.defaultFS property.", e);
+          + "Please check test.fs.adl.name property.", e);
     }
     byte[] expectedData = getRandomByteArrayData(totalSize);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextCreateMkdirLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextCreateMkdirLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextCreateMkdirLive.java
new file mode 100644
index 0000000..5166de1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextCreateMkdirLive.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+
+import java.net.URI;
+import java.util.UUID;
+
+/**
+ * Test file context Create/Mkdir operation.
+ */
+public class TestAdlFileContextCreateMkdirLive
+    extends FileContextCreateMkdirBaseTest {
+  private static final String KEY_FILE_SYSTEM = "test.fs.adl.name";
+
+  @BeforeClass
+  public static void skipTestCheck() {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    Configuration conf = AdlStorageConfiguration.getConfiguration();
+    String fileSystem = conf.get(KEY_FILE_SYSTEM);
+    if (fileSystem == null || fileSystem.trim().length() == 0) {
+      throw new Exception("Default file system not configured.");
+    }
+    URI uri = new URI(fileSystem);
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+    fc = FileContext.getFileContext(
+        new DelegateToFileSystem(uri, fs, conf, fs.getScheme(), false) {
+        }, conf);
+    super.setUp();
+  }
+
+  @Override
+  protected FileContextTestHelper createFileContextHelper() {
+    // On Windows, root directory path is created from local running directory.
+    // Adl does not support ':' as part of the path which results in failure.
+    return new FileContextTestHelper(UUID.randomUUID().toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextMainOperationsLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextMainOperationsLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextMainOperationsLive.java
new file mode 100644
index 0000000..ee10da7
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextMainOperationsLive.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.UUID;
+
+import static org.apache.hadoop.util.Shell.WINDOWS;
+
+/**
+ * Run collection of tests for the {@link FileContext}.
+ */
+public class TestAdlFileContextMainOperationsLive
+    extends FileContextMainOperationsBaseTest {
+
+  private static final String KEY_FILE_SYSTEM = "test.fs.adl.name";
+
+  @BeforeClass
+  public static void skipTestCheck() {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    Configuration conf = AdlStorageConfiguration.getConfiguration();
+    String fileSystem = conf.get(KEY_FILE_SYSTEM);
+    if (fileSystem == null || fileSystem.trim().length() == 0) {
+      throw new Exception("Default file system not configured.");
+    }
+    URI uri = new URI(fileSystem);
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+    fc = FileContext.getFileContext(
+        new DelegateToFileSystem(uri, fs, conf, fs.getScheme(), false) {
+        }, conf);
+    super.setUp();
+  }
+
+  @Override
+  protected FileContextTestHelper createFileContextHelper() {
+    // On Windows, root directory path is created from local running directory.
+    // Adl does not support ':' as part of the path which results in failure.
+    //    return new FileContextTestHelper(GenericTestUtils
+    // .getRandomizedTestDir()
+    //        .getAbsolutePath().replaceAll(":",""));
+    return new FileContextTestHelper(UUID.randomUUID().toString());
+  }
+
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return false;
+  }
+
+  @Override
+  public void testWorkingDirectory() throws Exception {
+    if (WINDOWS) {
+      // TODO :Fix is required in Hadoop shell to support windows permission
+      // set.
+      // The test is failing with NPE on windows platform only, with Linux
+      // platform test passes.
+      Assume.assumeTrue(false);
+    } else {
+      super.testWorkingDirectory();
+    }
+  }
+
+  @Override
+  public void testUnsupportedSymlink() throws IOException {
+    Assume.assumeTrue(false);
+  }
+
+  @Test
+  public void testSetVerifyChecksum() throws IOException {
+    Assume.assumeTrue(false);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
index 0df7d05..657947e 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
@@ -22,12 +22,13 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.Path;
-import org.junit.Test;
+import org.junit.Assume;
+import org.junit.Before;
 
 import java.io.IOException;
 
 /**
- * Verify Adls adhere to Hadoop file system semantics.
+ * Test Base contract tests on Adl file system.
  */
 public class TestAdlFileSystemContractLive extends FileSystemContractBaseTest {
   private FileSystem adlStore;
@@ -60,52 +61,8 @@ public class TestAdlFileSystemContractLive extends FileSystemContractBaseTest {
     }
   }
 
-  public void testGetFileStatus() throws IOException {
-    if (!AdlStorageConfiguration.isContractTestEnabled()) {
-      return;
-    }
-
-    Path testPath = new Path("/test/adltest");
-    if (adlStore.exists(testPath)) {
-      adlStore.delete(testPath, false);
-    }
-
-    adlStore.create(testPath).close();
-    assertTrue(adlStore.delete(testPath, false));
-  }
-
-  /**
-   * The following tests are failing on Azure Data Lake and the Azure Data Lake
-   * file system code needs to be modified to make them pass.
-   * A separate work item has been opened for this.
-   */
-  @Test
-  @Override
-  public void testMkdirsFailsForSubdirectoryOfExistingFile() throws Exception {
-    // BUG : Adl should return exception instead of false.
-  }
-
-  @Test
-  @Override
-  public void testMkdirsWithUmask() throws Exception {
-    // Support under implementation in Adl
-  }
-
-  @Test
-  @Override
-  public void testMoveFileUnderParent() throws Exception {
-    // BUG: Adl server should return expected status code.
-  }
-
-  @Test
-  @Override
-  public void testRenameFileToSelf() throws Exception {
-    // BUG: Adl server should return expected status code.
-  }
-
-  @Test
-  @Override
-  public void testRenameToDirWithSamePrefixAllowed() throws Exception {
-    // BUG: Adl server should return expected status code.
+  @Before
+  public void skipTestCheck() {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlInternalCreateNonRecursive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlInternalCreateNonRecursive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlInternalCreateNonRecursive.java
new file mode 100644
index 0000000..7e11a54
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlInternalCreateNonRecursive.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.common.Parallelized;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.UUID;
+
+/**
+ * Test createNonRecursive API.
+ */
+@RunWith(Parallelized.class)
+public class TestAdlInternalCreateNonRecursive {
+  private Path inputFileName;
+  private FsPermission inputPermission;
+  private boolean inputOverride;
+  private boolean inputFileAlreadyExist;
+  private boolean inputParentAlreadyExist;
+  private Class<IOException> expectedExceptionType;
+  private FileSystem adlStore;
+
+  public TestAdlInternalCreateNonRecursive(String testScenario, String fileName,
+      FsPermission permission, boolean override, boolean fileAlreadyExist,
+      boolean parentAlreadyExist, Class<IOException> exceptionType) {
+
+    // Random parent path for each test so that parallel execution does not fail
+    // other running test.
+    inputFileName = new Path(
+        "/test/createNonRecursive/" + UUID.randomUUID().toString(), fileName);
+    inputPermission = permission;
+    inputFileAlreadyExist = fileAlreadyExist;
+    inputOverride = override;
+    inputParentAlreadyExist = parentAlreadyExist;
+    expectedExceptionType = exceptionType;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection adlCreateNonRecursiveTestData()
+      throws UnsupportedEncodingException {
+    /*
+      Test Data
+      File name, Permission, Override flag, File already exist, Parent
+      already exist
+      shouldCreateSucceed, expectedExceptionIfFileCreateFails
+
+      File already exist and Parent already exist are mutually exclusive.
+    */
+    return Arrays.asList(new Object[][] {
+        {"CNR - When file do not exist.", UUID.randomUUID().toString(),
+            FsPermission.getFileDefault(), false, false, true, null},
+        {"CNR - When file exist. Override false", UUID.randomUUID().toString(),
+            FsPermission.getFileDefault(), false, true, true,
+            FileAlreadyExistsException.class},
+        {"CNR - When file exist. Override true", UUID.randomUUID().toString(),
+            FsPermission.getFileDefault(), true, true, true, null},
+
+        //TODO: This test is skipped till the fixes are not made it to prod.
+        /*{ "CNR - When parent do no exist.", UUID.randomUUID().toString(),
+            FsPermission.getFileDefault(), false, false, true, false,
+            IOException.class }*/});
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    adlStore = AdlStorageConfiguration.createStorageConnector();
+  }
+
+  @Test
+  public void testCreateNonRecursiveFunctionality() throws IOException {
+    if (inputFileAlreadyExist) {
+      FileSystem.create(adlStore, inputFileName, inputPermission);
+    }
+
+    // Mutually exclusive to inputFileAlreadyExist
+    if (inputParentAlreadyExist) {
+      adlStore.mkdirs(inputFileName.getParent());
+    } else {
+      adlStore.delete(inputFileName.getParent(), true);
+    }
+
+    try {
+      adlStore.createNonRecursive(inputFileName, inputPermission, inputOverride,
+          CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT,
+          adlStore.getDefaultReplication(inputFileName),
+          adlStore.getDefaultBlockSize(inputFileName), null);
+    } catch (IOException e) {
+
+      if (expectedExceptionType == null) {
+        throw e;
+      }
+
+      Assert.assertEquals(expectedExceptionType, e.getClass());
+      return;
+    }
+
+    if (expectedExceptionType != null) {
+      Assert.fail("CreateNonRecursive should have failed with exception "
+          + expectedExceptionType.getName());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlPermissionLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlPermissionLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlPermissionLive.java
new file mode 100644
index 0000000..dd7c10d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlPermissionLive.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.common.Parallelized;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.*;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+
+/**
+ * Test ACL permission on file/folder on Adl file system.
+ */
+@RunWith(Parallelized.class)
+public class TestAdlPermissionLive {
+
+  private static Path testRoot = new Path("/test");
+  private FsPermission permission;
+  private Path path;
+  private FileSystem adlStore;
+
+  public TestAdlPermissionLive(FsPermission testPermission) {
+    permission = testPermission;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection adlCreateNonRecursiveTestData()
+      throws UnsupportedEncodingException {
+    /*
+      Test Data
+      File/Folder name, User permission, Group permission, Other Permission,
+      Parent already exist
+      shouldCreateSucceed, expectedExceptionIfFileCreateFails
+    */
+    final Collection<Object[]> datas = new ArrayList<>();
+    for (FsAction g : FsAction.values()) {
+      for (FsAction o : FsAction.values()) {
+        datas.add(new Object[] {new FsPermission(FsAction.ALL, g, o)});
+      }
+    }
+    return datas;
+  }
+
+  @AfterClass
+  public static void cleanUp() throws IOException, URISyntaxException {
+    if (AdlStorageConfiguration.isContractTestEnabled()) {
+      Assert.assertTrue(AdlStorageConfiguration.createStorageConnector()
+          .delete(testRoot, true));
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    adlStore = AdlStorageConfiguration.createStorageConnector();
+  }
+
+  @Test
+  public void testFilePermission() throws IOException {
+    path = new Path(testRoot, UUID.randomUUID().toString());
+    adlStore.getConf()
+        .set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000");
+
+    adlStore.mkdirs(path.getParent(),
+        new FsPermission(FsAction.ALL, FsAction.WRITE, FsAction.NONE));
+    adlStore.removeDefaultAcl(path.getParent());
+
+    adlStore.create(path, permission, true, 1024, (short) 1, 1023, null);
+    FileStatus status = adlStore.getFileStatus(path);
+    Assert.assertEquals(permission, status.getPermission());
+  }
+
+  @Test
+  public void testFolderPermission() throws IOException {
+    path = new Path(testRoot, UUID.randomUUID().toString());
+    adlStore.getConf()
+        .set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000");
+    adlStore.mkdirs(path.getParent(),
+        new FsPermission(FsAction.ALL, FsAction.WRITE, FsAction.NONE));
+    adlStore.removeDefaultAcl(path.getParent());
+
+    adlStore.mkdirs(path, permission);
+    FileStatus status = adlStore.getFileStatus(path);
+    Assert.assertEquals(permission, status.getPermission());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSupportedCharsetInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSupportedCharsetInPath.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSupportedCharsetInPath.java
new file mode 100644
index 0000000..d80b6bf
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSupportedCharsetInPath.java
@@ -0,0 +1,334 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.common.Parallelized;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URISyntaxException;
+import java.util.*;
+
+/**
+ * Test supported ASCII, UTF-8 character set supported by Adl storage file
+ * system on file/folder operation.
+ */
+@RunWith(Parallelized.class)
+public class TestAdlSupportedCharsetInPath {
+
+  private static final String TEST_ROOT = "/test/";
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestAdlSupportedCharsetInPath.class);
+  private String path;
+
+  public TestAdlSupportedCharsetInPath(String filePath) {
+    path = filePath;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<Object[]> adlCharTestData()
+      throws UnsupportedEncodingException {
+
+    ArrayList<String> filePathList = new ArrayList<>();
+    for (int i = 32; i < 127; ++i) {
+      String specialChar = (char) i + "";
+      if (i >= 48 && i <= 57) {
+        continue;
+      }
+
+      if (i >= 65 && i <= 90) {
+        continue;
+      }
+
+      if (i >= 97 && i <= 122) {
+        continue;
+      }
+
+      // Special char at start of the path
+      if (i != 92 && i != 58 && i != 46 && i != 47) {
+        filePathList.add(specialChar + "");
+      }
+
+      // Special char at end of string
+      if (i != 92 && i != 47 && i != 58) {
+        filePathList.add("file " + i + " " + specialChar);
+      }
+
+      // Special char in between string
+      if (i != 47 && i != 58 && i != 92) {
+        filePathList.add("file " + i + " " + specialChar + "_name");
+      }
+    }
+
+    filePathList.add("a  ");
+    filePathList.add("a..b");
+    fillUnicodes(filePathList);
+    Collection<Object[]> result = new ArrayList<>();
+    for (String item : filePathList) {
+      result.add(new Object[] {item});
+    }
+    return result;
+  }
+
+  private static void fillUnicodes(ArrayList<String> filePathList) {
+    // Unicode characters
+    filePathList.add("\u0627\u0644\u0628\u064a\u0627\u0646\u0627\u062a \u0627\u0644\u0643\u0628\u064a\u0631\u0629"); // Arabic
+    filePathList.add("T� dh�nat i madh"); // Albanian
+    filePathList.add("\u0574\u0565\u056e \u057f\u057e\u0575\u0561\u056c\u0576\u0565\u0580\u0568"); // Armenian
+    filePathList.add("b�y�k data"); // Azerbaijani
+    filePathList.add("\u0432\u044f\u043b\u0456\u043a\u0456\u044f \u0434\u0430\u0434\u0437\u0435\u043d\u044b\u044f"); // Belarusian,
+    filePathList.add("\u09ac\u09bf\u0997 \u09a1\u09c7\u099f\u09be"); // Bengali
+    filePathList.add("veliki podataka"); // Bosnian
+    filePathList.add("\u0433\u043e\u043b\u044f\u043c\u0430 \u0434\u0430\u043d\u043d\u0438"); // Bulgarian
+    filePathList.add("\u5927\u6570\u636e"); // Chinese - Simplified
+    filePathList.add("\u5927\u6578\u64da"); // Chinese - Traditional
+    filePathList.add("\u10d3\u10d8\u10d3\u10d8 \u10db\u10dd\u10dc\u10d0\u10ea\u10d4\u10db\u10d7\u10d0"); // Georgian,
+    filePathList.add("gro�e Daten"); // German
+    filePathList.add("\u03bc\u03b5\u03b3\u03ac\u03bb\u03bf \u03b4\u03b5\u03b4\u03bf\u03bc\u03ad\u03bd\u03b1"); // Greek
+    filePathList.add("\u0aae\u0acb\u0a9f\u0abe \u0aae\u0abe\u0ab9\u0abf\u0aa4\u0ac0"); // Gujarati
+    filePathList.add("\u05e0\u05ea\u05d5\u05e0\u05d9\u05dd \u05d2\u05d3\u05d5\u05dc\u05d9\u05dd"); // Hebrew
+    filePathList.add("\u092c\u0921\u093c\u093e \u0921\u0947\u091f\u093e"); // Hindi
+    filePathList.add("st�r g�gn"); // Icelandic
+    filePathList.add("sonra� m�r"); // Irish
+    filePathList.add("\u30d3\u30c3\u30b0\u30c7\u30fc\u30bf"); // Japanese
+    filePathList.add("\u04af\u043b\u043a\u0435\u043d \u0434\u0435\u0440\u0435\u043a\u0442\u0435\u0440"); // Kazakh
+    filePathList.add("\u1791\u17b7\u1793\u17d2\u1793\u1793\u17d0\u1799\u1792\u17c6"); // Khmer
+    filePathList.add("\ube45 \ub370\uc774\ud130"); // Korean
+    filePathList.add("\u0e82\u0ecd\u0ec9\u0ea1\u0eb9\u0e99 \u0e82\u0eb0\u0eab\u0e99\u0eb2\u0e94\u0ec3\u0eab\u0e8d\u0ec8"); // Lao
+    filePathList.add("\u0433\u043e\u043b\u0435\u043c\u0438 \u043f\u043e\u0434\u0430\u0442\u043e\u0446\u0438"); // Macedonian
+    filePathList.add("\u0920\u0942\u0932\u094b \u0921\u093e\u091f\u093e"); // Nepali
+    filePathList.add("\u0d35\u0d32\u0d3f\u0d2f \u0d21\u0d3e\u0d31\u0d4d\u0d31"); // Malayalam
+    filePathList.add("\u092e\u094b\u0920\u0947 \u0921\u0947\u091f\u093e"); // Marathi
+    filePathList.add("\u0442\u043e\u043c \u043c\u044d\u0434\u044d\u044d\u043b\u044d\u043b"); // Mangolian
+    filePathList.add("\u0627\u0637\u0644\u0627\u0639\u0627\u062a \u0628\u0632\u0631\u06af"); // Persian
+    filePathList.add("\u0a35\u0a71\u0a21\u0a47 \u0a21\u0a3e\u0a1f\u0a47 \u0a28\u0a42\u0a70"); // Punjabi
+    filePathList.add("\u0431\u043e\u043b\u044c\u0448\u0438\u0435 \u0434\u0430\u043d\u043d\u044b\u0435"); // Russian
+    filePathList.add("\u0412\u0435\u043b\u0438\u043a\u0438 \u043f\u043e\u0434\u0430\u0442\u0430\u043a\u0430"); // Serbian
+    filePathList.add("\u0dc0\u0dd2\u0dc1\u0dcf\u0dbd \u0daf\u0dad\u0dca\u0dad"); // Sinhala
+    filePathList.add("big d�t"); // Slovak
+    filePathList.add("\u043c\u0430\u044a\u043b\u0443\u043c\u043e\u0442\u0438 \u043a\u0430\u043b\u043e\u043d"); // Tajik
+    filePathList.add("\u0baa\u0bc6\u0bb0\u0bbf\u0baf \u0ba4\u0bb0\u0bb5\u0bc1"); // Tamil
+    filePathList.add("\u0c2a\u0c46\u0c26\u0c4d\u0c26 \u0c21\u0c47\u0c1f\u0c3e"); // Telugu
+    filePathList.add("\u0e02\u0e49\u0e2d\u0e21\u0e39\u0e25\u0e43\u0e2b\u0e0d\u0e48"); // Thai
+    filePathList.add("b�y�k veri"); // Turkish
+    filePathList.add("\u0432\u0435\u043b\u0438\u043a\u0456 \u0434\u0430\u043d\u0456"); // Ukranian
+    filePathList.add("\u0628\u0691\u06d2 \u0627\u0639\u062f\u0627\u062f \u0648 \u0634\u0645\u0627\u0631"); // Urdu
+    filePathList.add("katta ma'lumotlar"); // Uzbek
+    filePathList.add("d\u1eef li\u1ec7u l\u1edbn"); // Vietanamese
+    filePathList.add("\u05d2\u05e8\u05d5\u05d9\u05e1 \u05d3\u05d0\u05b7\u05d8\u05df"); // Yiddish
+    filePathList.add("big idatha"); // Zulu
+    filePathList.add("rachel\u03c7");
+    filePathList.add("jessica\u03bf");
+    filePathList.add("sarah\u03b4");
+    filePathList.add("katie\u03bd");
+    filePathList.add("wendy\u03be");
+    filePathList.add("david\u03bc");
+    filePathList.add("priscilla\u03c5");
+    filePathList.add("oscar\u03b8");
+    filePathList.add("xavier\u03c7");
+    filePathList.add("gabriella\u03b8");
+    filePathList.add("david\u03c5");
+    filePathList.add("irene\u03bc");
+    filePathList.add("fred\u03c1");
+    filePathList.add("david\u03c4");
+    filePathList.add("ulysses\u03bd");
+    filePathList.add("gabriella\u03bc");
+    filePathList.add("zach\u03b6");
+    filePathList.add("gabriella\u03bb");
+    filePathList.add("ulysses\u03c6");
+    filePathList.add("david\u03c7");
+    filePathList.add("sarah\u03c3");
+    filePathList.add("holly\u03c8");
+    filePathList.add("nick\u03b1");
+    filePathList.add("ulysses\u03b9");
+    filePathList.add("mike\u03b2");
+    filePathList.add("priscilla\u03ba");
+    filePathList.add("wendy\u03b8");
+    filePathList.add("jessica\u03c2");
+    filePathList.add("fred\u03c7");
+    filePathList.add("fred\u03b6");
+    filePathList.add("sarah\u03ba");
+    filePathList.add("calvin\u03b7");
+    filePathList.add("xavier\u03c7");
+    filePathList.add("yuri\u03c7");
+    filePathList.add("ethan\u03bb");
+    filePathList.add("holly\u03b5");
+    filePathList.add("xavier\u03c3");
+    filePathList.add("victor\u03c4");
+    filePathList.add("wendy\u03b2");
+    filePathList.add("jessica\u03c2");
+    filePathList.add("quinn\u03c6");
+    filePathList.add("xavier\u03c5");
+    filePathList.add("nick\u03b9");
+    filePathList.add("rachel\u03c6");
+    filePathList.add("oscar\u03be");
+    filePathList.add("zach\u03b4");
+    filePathList.add("zach\u03bb");
+    filePathList.add("rachel\u03b1");
+    filePathList.add("jessica\u03c6");
+    filePathList.add("luke\u03c6");
+    filePathList.add("tom\u03b6");
+    filePathList.add("nick\u03be");
+    filePathList.add("nick\u03ba");
+    filePathList.add("ethan\u03b4");
+    filePathList.add("fred\u03c7");
+    filePathList.add("priscilla\u03b8");
+    filePathList.add("zach\u03be");
+    filePathList.add("xavier\u03be");
+    filePathList.add("zach\u03c8");
+    filePathList.add("ethan\u03b1");
+    filePathList.add("oscar\u03b9");
+    filePathList.add("irene\u03b4");
+    filePathList.add("irene\u03b6");
+    filePathList.add("victor\u03bf");
+    filePathList.add("wendy\u03b2");
+    filePathList.add("mike\u03c3");
+    filePathList.add("fred\u03bf");
+    filePathList.add("mike\u03b7");
+    filePathList.add("sarah\u03c1");
+    filePathList.add("quinn\u03b2");
+    filePathList.add("mike\u03c5");
+    filePathList.add("nick\u03b6");
+    filePathList.add("nick\u03bf");
+    filePathList.add("tom\u03ba");
+    filePathList.add("bob\u03bb");
+    filePathList.add("yuri\u03c0");
+    filePathList.add("david\u03c4");
+    filePathList.add("quinn\u03c0");
+    filePathList.add("mike\u03bb");
+    filePathList.add("david\u03b7");
+    filePathList.add("ethan\u03c4");
+    filePathList.add("nick\u03c6");
+    filePathList.add("yuri\u03bf");
+    filePathList.add("ethan\u03c5");
+    filePathList.add("bob\u03b8");
+    filePathList.add("david\u03bb");
+    filePathList.add("priscilla\u03be");
+    filePathList.add("nick\u03b3");
+    filePathList.add("luke\u03c5");
+    filePathList.add("irene\u03bb");
+    filePathList.add("xavier\u03bf");
+    filePathList.add("fred\u03c5");
+    filePathList.add("ulysses\u03bc");
+    filePathList.add("wendy\u03b3");
+    filePathList.add("zach\u03bb");
+    filePathList.add("rachel\u03c2");
+    filePathList.add("sarah\u03c0");
+    filePathList.add("alice\u03c8");
+    filePathList.add("bob\u03c4");
+  }
+
+  @AfterClass
+  public static void testReport() throws IOException, URISyntaxException {
+    if (!AdlStorageConfiguration.isContractTestEnabled()) {
+      return;
+    }
+
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+    fs.delete(new Path(TEST_ROOT), true);
+  }
+
+  @Test
+  public void testAllowedSpecialCharactersMkdir()
+      throws IOException, URISyntaxException {
+    Path parentPath = new Path(TEST_ROOT, UUID.randomUUID().toString() + "/");
+    Path specialFile = new Path(parentPath, path);
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+
+    Assert.assertTrue("Mkdir failed : " + specialFile, fs.mkdirs(specialFile));
+    Assert.assertTrue("File not Found after Mkdir success" + specialFile,
+        fs.exists(specialFile));
+    Assert.assertTrue("Not listed under parent " + parentPath,
+        contains(fs.listStatus(parentPath),
+            fs.makeQualified(specialFile).toString()));
+    Assert.assertTrue("Delete failed : " + specialFile,
+            fs.delete(specialFile, true));
+    Assert.assertFalse("File still exist after delete " + specialFile,
+        fs.exists(specialFile));
+  }
+
+  private boolean contains(FileStatus[] statuses, String remotePath) {
+    for (FileStatus status : statuses) {
+      if (status.getPath().toString().equals(remotePath)) {
+        return true;
+      }
+    }
+
+    Arrays.stream(statuses).forEach(s -> LOG.info(s.getPath().toString()));
+    return false;
+  }
+
+  @Before
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+
+  @Test
+  public void testAllowedSpecialCharactersRename()
+      throws IOException, URISyntaxException {
+
+    String parentPath = TEST_ROOT + UUID.randomUUID().toString() + "/";
+    Path specialFile = new Path(parentPath + path);
+    Path anotherLocation = new Path(parentPath + UUID.randomUUID().toString());
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+
+    Assert.assertTrue("Could not create " + specialFile.toString(),
+        fs.createNewFile(specialFile));
+    Assert.assertTrue(
+        "Failed to rename " + specialFile.toString() + " --> " + anotherLocation
+            .toString(), fs.rename(specialFile, anotherLocation));
+    Assert.assertFalse("File should not be present after successful rename : "
+        + specialFile.toString(), fs.exists(specialFile));
+    Assert.assertTrue("File should be present after successful rename : "
+        + anotherLocation.toString(), fs.exists(anotherLocation));
+    Assert.assertFalse(
+        "Listed under parent whereas expected not listed : " + parentPath,
+        contains(fs.listStatus(new Path(parentPath)),
+            fs.makeQualified(specialFile).toString()));
+
+    Assert.assertTrue(
+        "Failed to rename " + anotherLocation.toString() + " --> " + specialFile
+            .toString(), fs.rename(anotherLocation, specialFile));
+    Assert.assertTrue(
+        "File should be present after successful rename : " + "" + specialFile
+            .toString(), fs.exists(specialFile));
+    Assert.assertFalse("File should not be present after successful rename : "
+        + anotherLocation.toString(), fs.exists(anotherLocation));
+
+    Assert.assertTrue("Not listed under parent " + parentPath,
+        contains(fs.listStatus(new Path(parentPath)),
+            fs.makeQualified(specialFile).toString()));
+
+    Assert.assertTrue("Failed to delete " + parentPath,
+        fs.delete(new Path(parentPath), true));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
new file mode 100644
index 0000000..3b9e7da
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.UUID;
+
+/**
+ * This class is responsible for testing ContentSummary, ListStatus on
+ * file/folder.
+ */
+public class TestMetadata {
+
+  private FileSystem adlStore;
+  private Path parent;
+
+  public TestMetadata() {
+    parent = new Path("test");
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    adlStore = AdlStorageConfiguration.createStorageConnector();
+  }
+
+  @After
+  public void cleanUp() throws Exception {
+    if (AdlStorageConfiguration.isContractTestEnabled()) {
+      adlStore.delete(parent, true);
+    }
+  }
+
+  @Test
+  public void testContentSummaryOnFile() throws IOException {
+    Path child = new Path(UUID.randomUUID().toString());
+    Path testFile = new Path(parent, child);
+    OutputStream out = adlStore.create(testFile);
+
+    for (int i = 0; i < 1024; ++i) {
+      out.write(97);
+    }
+    out.close();
+
+    Assert.assertTrue(adlStore.isFile(testFile));
+    ContentSummary summary = adlStore.getContentSummary(testFile);
+    Assert.assertEquals(1024, summary.getSpaceConsumed());
+    Assert.assertEquals(1, summary.getFileCount());
+    Assert.assertEquals(0, summary.getDirectoryCount());
+    Assert.assertEquals(1024, summary.getLength());
+  }
+
+  @Test
+  public void testContentSummaryOnFolder() throws IOException {
+    Path child = new Path(UUID.randomUUID().toString());
+    Path testFile = new Path(parent, child);
+    OutputStream out = adlStore.create(testFile);
+
+    for (int i = 0; i < 1024; ++i) {
+      out.write(97);
+    }
+    out.close();
+
+    Assert.assertTrue(adlStore.isFile(testFile));
+    ContentSummary summary = adlStore.getContentSummary(parent);
+    Assert.assertEquals(1024, summary.getSpaceConsumed());
+    Assert.assertEquals(1, summary.getFileCount());
+    Assert.assertEquals(1, summary.getDirectoryCount());
+    Assert.assertEquals(1024, summary.getLength());
+  }
+
+  @Test
+  public void listStatusOnFile() throws IOException {
+    Path path = new Path(parent, "a.txt");
+    FileSystem fs = adlStore;
+    fs.createNewFile(path);
+    Assert.assertTrue(fs.isFile(path));
+    FileStatus[] statuses = fs.listStatus(path);
+    Assert
+        .assertEquals(path.makeQualified(fs.getUri(), fs.getWorkingDirectory()),
+            statuses[0].getPath());
+  }
+}
+


---------------------------------------------------------------------
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: HADOOP-13855. Fix a couple of the s3a statistic names to be consistent with the rest. Contributed by Steve Loughran

Posted by xg...@apache.org.
HADOOP-13855. Fix a couple of the s3a statistic names to be consistent with the rest. 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/51211a7d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/51211a7d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/51211a7d

Branch: refs/heads/YARN-5734
Commit: 51211a7d7aa342b93951fe61da3f624f0652e101
Parents: 5bd18c4
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Dec 2 13:48:15 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Fri Dec 2 14:01:42 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51211a7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index 36ec50b..789c6d7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -92,12 +92,12 @@ public enum Statistic {
       "Count of times the TCP stream was aborted"),
   STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations",
       "Number of executed seek operations which went backwards in a stream"),
-  STREAM_CLOSED("streamClosed", "Count of times the TCP stream was closed"),
+  STREAM_CLOSED("stream_closed", "Count of times the TCP stream was closed"),
   STREAM_CLOSE_OPERATIONS("stream_close_operations",
       "Total count of times an attempt to close a data stream was made"),
   STREAM_FORWARD_SEEK_OPERATIONS("stream_forward_seek_operations",
       "Number of executed seek operations which went forward in a stream"),
-  STREAM_OPENED("streamOpened",
+  STREAM_OPENED("stream_opened",
       "Total count of times an input stream to object store was opened"),
   STREAM_READ_EXCEPTIONS("stream_read_exceptions",
       "Number of seek operations invoked on input streams"),


---------------------------------------------------------------------
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: HDFS-8678. Bring back the feature to view chunks of files in the HDFS file browser. Contributed by Ivo Udelsmann.

Posted by xg...@apache.org.
HDFS-8678. Bring back the feature to view chunks of files in the HDFS file browser. Contributed by Ivo Udelsmann.


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

Branch: refs/heads/YARN-5734
Commit: 625df87c7b8ec2787e743d845fadde5e73479dc1
Parents: 51e6c1c
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Wed Nov 30 09:11:19 2016 -0800
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Wed Nov 30 09:12:15 2016 -0800

----------------------------------------------------------------------
 .../src/main/webapps/hdfs/explorer.html         | 13 +++++--
 .../src/main/webapps/hdfs/explorer.js           | 37 +++++++++++++++++---
 2 files changed, 43 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/625df87c/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index ad8c374..3700a5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -57,8 +57,17 @@
 	    <h4 class="modal-title" id="file-info-title">File information</h4>
 	  </div>
 	  <div class="modal-body" id="file-info-body">
-	    <a id="file-info-download">Download</a>
-        <!--<a id="file-info-preview" style="cursor:pointer">Tail the file (last 32K)</a>-->
+	    <div class=row>
+              <span class="col-xs-4">
+                <a id="file-info-download">Download</a>
+              </span>
+              <span class="col-xs-4">
+                <a id="file-info-preview-head" style="cursor:pointer">Head the file (first 32K)</a>
+              </span>
+              <span class="col-xs-4">
+                <a id="file-info-preview-tail" style="cursor:pointer">Tail the file (last 32K)</a>
+	      </span>
+	    </div>
 	    <hr />
 	    <div class="panel panel-success" id="file-info-blockinfo-panel">
 	      <div class="panel-heading">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/625df87c/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index 1739db2..3e276a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -192,13 +192,40 @@
       var download_url = '/webhdfs/v1' + abs_path + '?op=OPEN';
 
       $('#file-info-download').attr('href', download_url);
-      $('#file-info-preview').click(function() {
+
+      var processPreview = function(url) {
+        url += "&noredirect=true";
+        $.ajax({
+          type: 'GET',
+          url: url,
+          processData: false,
+          crossDomain: true
+        }).done(function(data) {
+          url = data.Location;
+          $.ajax({
+            type: 'GET',
+            url: url,
+            processData: false,
+            crossDomain: true
+          }).complete(function(data) {
+            $('#file-info-preview-body').val(data.responseText);
+            $('#file-info-tail').show();
+          }).error(function(jqXHR, textStatus, errorThrown) {
+            show_err_msg("Couldn't preview the file. " + errorThrown);
+          });
+        }).error(function(jqXHR, textStatus, errorThrown) {
+          show_err_msg("Couldn't find datanode to read file from. " + errorThrown);
+        });
+      }
+
+      $('#file-info-preview-tail').click(function() {
         var offset = d.fileLength - TAIL_CHUNK_SIZE;
         var url = offset > 0 ? download_url + '&offset=' + offset : download_url;
-        $.get(url, function(t) {
-          $('#file-info-preview-body').val(t);
-          $('#file-info-tail').show();
-        }, "text").error(network_error_handler(url));
+        processPreview(url);
+      });
+      $('#file-info-preview-head').click(function() {
+        var url = d.fileLength > TAIL_CHUNK_SIZE ? download_url + '&length=' + TAIL_CHUNK_SIZE : download_url;
+        processPreview(url);
       });
 
       if (d.fileLength > 0) {


---------------------------------------------------------------------
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: Revert due to an error "HDFS-10994. Support an XOR policy XOR-2-1-64k in HDFS. Contributed by Sammi Chen"

Posted by xg...@apache.org.
Revert due to an error "HDFS-10994. Support an XOR policy XOR-2-1-64k in HDFS. Contributed by Sammi Chen"

This reverts commit 5614f847b2ef2a5b70bd9a06edc4eba06174c6.


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

Branch: refs/heads/YARN-5734
Commit: cfd8076f81930c3ffea8ec2ef42926217b83ab1a
Parents: aeecfa2
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Nov 30 15:44:52 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Nov 30 15:44:52 2016 +0800

----------------------------------------------------------------------
 .../io/erasurecode/ErasureCodeConstants.java    |   3 -
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   1 -
 .../namenode/ErasureCodingPolicyManager.java    |  23 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   8 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  28 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |  50 +--
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  27 +-
 .../TestDFSStripedOutputStreamWithFailure.java  |  37 +-
 .../hdfs/TestDFSXORStripedInputStream.java      |  33 --
 .../hdfs/TestDFSXORStripedOutputStream.java     |  35 --
 ...estDFSXORStripedOutputStreamWithFailure.java |  36 --
 ...tyPreemptionPolicyForReservedContainers.java | 430 +++++++++++++++++++
 12 files changed, 471 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
index ffa0bce..8d6ff85 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
@@ -38,7 +38,4 @@ public final class ErasureCodeConstants {
 
   public static final ECSchema RS_6_3_LEGACY_SCHEMA = new ECSchema(
       RS_LEGACY_CODEC_NAME, 6, 3);
-
-  public static final ECSchema XOR_2_1_SCHEMA = new ECSchema(
-      XOR_CODEC_NAME, 2, 1);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index b55b4df..acbc8f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -147,7 +147,6 @@ public final class HdfsConstants {
   public static final byte RS_6_3_POLICY_ID = 0;
   public static final byte RS_3_2_POLICY_ID = 1;
   public static final byte RS_6_3_LEGACY_POLICY_ID = 2;
-  public static final byte XOR_2_1_POLICY_ID = 3;
 
   /* Hidden constructor */
   protected HdfsConstants() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 8a85d23..c4bc8de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -36,7 +36,7 @@ import java.util.TreeMap;
 public final class ErasureCodingPolicyManager {
 
   /**
-   * TODO: HDFS-8095.
+   * TODO: HDFS-8095
    */
   private static final int DEFAULT_CELLSIZE = 64 * 1024;
   private static final ErasureCodingPolicy SYS_POLICY1 =
@@ -48,14 +48,10 @@ public final class ErasureCodingPolicyManager {
   private static final ErasureCodingPolicy SYS_POLICY3 =
       new ErasureCodingPolicy(ErasureCodeConstants.RS_6_3_LEGACY_SCHEMA,
           DEFAULT_CELLSIZE, HdfsConstants.RS_6_3_LEGACY_POLICY_ID);
-  private static final ErasureCodingPolicy SYS_POLICY4 =
-      new ErasureCodingPolicy(ErasureCodeConstants.XOR_2_1_SCHEMA,
-          DEFAULT_CELLSIZE, HdfsConstants.XOR_2_1_POLICY_ID);
 
   //We may add more later.
   private static final ErasureCodingPolicy[] SYS_POLICIES =
-      new ErasureCodingPolicy[]{SYS_POLICY1, SYS_POLICY2, SYS_POLICY3,
-          SYS_POLICY4};
+      new ErasureCodingPolicy[]{SYS_POLICY1, SYS_POLICY2, SYS_POLICY3};
 
   // Supported storage policies for striped EC files
   private static final byte[] SUITABLE_STORAGE_POLICIES_FOR_EC_STRIPED_MODE = new byte[] {
@@ -101,19 +97,6 @@ public final class ErasureCodingPolicyManager {
   }
 
   /**
-   * Get system-wide policy by policy ID.
-   * @return ecPolicy
-   */
-  public static ErasureCodingPolicy getPolicyByPolicyID(byte id) {
-    for (ErasureCodingPolicy policy : SYS_POLICIES) {
-      if (policy.getId() == id) {
-        return policy;
-      }
-    }
-    return null;
-  }
-
-  /**
    * Get all policies that's available to use.
    * @return all policies
    */
@@ -158,7 +141,7 @@ public final class ErasureCodingPolicyManager {
   }
 
   /**
-   * Clear and clean up.
+   * Clear and clean up
    */
   public void clear() {
     activePoliciesByName.clear();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index a5dcee9..37f97db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -455,13 +455,9 @@ public class INodeFile extends INodeWithAdditionalFields
     if(!isStriped()){
       return max;
     }
-
+    // TODO support more policies based on policyId
     ErasureCodingPolicy ecPolicy =
-        ErasureCodingPolicyManager.getPolicyByPolicyID(
-            getErasureCodingPolicyID());
-    if (ecPolicy == null){
-      ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
-    }
+        ErasureCodingPolicyManager.getSystemDefaultPolicy();
     return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 1fbc1d9..13e2656 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -1888,41 +1888,21 @@ public class DFSTestUtil {
    * Creates the metadata of a file in striped layout. This method only
    * manipulates the NameNode state without injecting data to DataNode.
    * You should disable periodical heartbeat before use this.
-   * @param file Path of the file to create
+   *  @param file Path of the file to create
    * @param dir Parent path of the file
    * @param numBlocks Number of striped block groups to add to the file
    * @param numStripesPerBlk Number of striped cells in each block
    * @param toMkdir
    */
-  public static void createStripedFile(MiniDFSCluster cluster, Path file,
-      Path dir, int numBlocks, int numStripesPerBlk, boolean toMkdir)
-      throws Exception {
-    createStripedFile(cluster, file, dir, numBlocks, numStripesPerBlk,
-        toMkdir, null);
-  }
-
-  /**
-   * Creates the metadata of a file in striped layout. This method only
-   * manipulates the NameNode state without injecting data to DataNode.
-   * You should disable periodical heartbeat before use this.
-   * @param file Path of the file to create
-   * @param dir Parent path of the file
-   * @param numBlocks Number of striped block groups to add to the file
-   * @param numStripesPerBlk Number of striped cells in each block
-   * @param toMkdir
-   * @param ecPolicy erasure coding policy apply to created file. A null value
-   *                 means using default erasure coding policy.
-   */
-  public static void createStripedFile(MiniDFSCluster cluster, Path file,
-      Path dir, int numBlocks, int numStripesPerBlk, boolean toMkdir,
-      ErasureCodingPolicy ecPolicy) throws Exception {
+  public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir,
+      int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception {
     DistributedFileSystem dfs = cluster.getFileSystem();
     // If outer test already set EC policy, dir should be left as null
     if (toMkdir) {
       assert dir != null;
       dfs.mkdirs(dir);
       try {
-        dfs.getClient().setErasureCodingPolicy(dir.toString(), ecPolicy);
+        dfs.getClient().setErasureCodingPolicy(dir.toString(), null);
       } catch (IOException e) {
         if (!e.getMessage().contains("non-empty directory")) {
           throw e;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
index 121b9a4..3b46c66 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
@@ -64,34 +64,20 @@ public class TestDFSStripedInputStream {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
-  private ErasureCodingPolicy ecPolicy;
-  private short dataBlocks;
-  private short parityBlocks;
-  private int cellSize;
+  private final ErasureCodingPolicy ecPolicy =
+      ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  private final short dataBlocks = (short) ecPolicy.getNumDataUnits();
+  private final short parityBlocks = (short) ecPolicy.getNumParityUnits();
+  private final int cellSize = ecPolicy.getCellSize();
   private final int stripesPerBlock = 2;
-  private int blockSize;
-  private int blockGroupSize;
+  private final int blockSize = stripesPerBlock * cellSize;
+  private final int blockGroupSize =  dataBlocks * blockSize;
 
   @Rule
   public Timeout globalTimeout = new Timeout(300000);
 
-  public ErasureCodingPolicy getEcPolicy() {
-    return ErasureCodingPolicyManager.getSystemDefaultPolicy();
-  }
-
   @Before
   public void setup() throws IOException {
-    /*
-     * Initialize erasure coding policy.
-     */
-    ecPolicy = getEcPolicy();
-    dataBlocks = (short) ecPolicy.getNumDataUnits();
-    parityBlocks = (short) ecPolicy.getNumParityUnits();
-    cellSize = ecPolicy.getCellSize();
-    blockSize = stripesPerBlock * cellSize;
-    blockGroupSize =  dataBlocks * blockSize;
-    System.out.println("EC policy = " + ecPolicy);
-
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     if (ErasureCodeNative.isNativeCodeLoaded()) {
@@ -108,7 +94,7 @@ public class TestDFSStripedInputStream {
     }
     fs = cluster.getFileSystem();
     fs.mkdirs(dirPath);
-    fs.getClient().setErasureCodingPolicy(dirPath.toString(), ecPolicy);
+    fs.getClient().setErasureCodingPolicy(dirPath.toString(), null);
   }
 
   @After
@@ -120,13 +106,13 @@ public class TestDFSStripedInputStream {
   }
 
   /**
-   * Test {@link DFSStripedInputStream#getBlockAt(long)}.
+   * Test {@link DFSStripedInputStream#getBlockAt(long)}
    */
   @Test
   public void testRefreshBlock() throws Exception {
     final int numBlocks = 4;
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false, ecPolicy);
+        stripesPerBlock, false);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, blockGroupSize * numBlocks);
     final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
@@ -150,7 +136,7 @@ public class TestDFSStripedInputStream {
   public void testPread() throws Exception {
     final int numBlocks = 2;
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false, ecPolicy);
+        stripesPerBlock, false);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, blockGroupSize * numBlocks);
     int fileLen = blockGroupSize * numBlocks;
@@ -168,9 +154,7 @@ public class TestDFSStripedInputStream {
             bg.getBlock().getBlockPoolId());
       }
 
-      /**
-       * A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks
-       */
+      /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
       for (int i = 0; i < stripesPerBlock; i++) {
         for (int j = 0; j < dataBlocks; j++) {
           for (int k = 0; k < cellSize; k++) {
@@ -210,7 +194,7 @@ public class TestDFSStripedInputStream {
     final int numBlocks = 4;
     final int failedDNIdx = dataBlocks - 1;
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false, ecPolicy);
+        stripesPerBlock, false);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, blockGroupSize);
 
@@ -321,7 +305,7 @@ public class TestDFSStripedInputStream {
       setup();
     }
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false, ecPolicy);
+        stripesPerBlock, false);
     LocatedBlocks lbs = fs.getClient().namenode.
         getBlockLocations(filePath.toString(), 0, fileSize);
 
@@ -346,9 +330,7 @@ public class TestDFSStripedInputStream {
     byte[] expected = new byte[fileSize];
 
     for (LocatedBlock bg : lbs.getLocatedBlocks()) {
-      /**
-       * A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks
-       */
+      /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
       for (int i = 0; i < stripesPerBlock; i++) {
         for (int j = 0; j < dataBlocks; j++) {
           for (int k = 0; k < cellSize; k++) {
@@ -389,7 +371,7 @@ public class TestDFSStripedInputStream {
     final int numBlocks = 4;
     final int failedDNIdx = dataBlocks - 1;
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false, ecPolicy);
+        stripesPerBlock, false);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, blockGroupSize);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index 5bde16e..b686f28 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -47,36 +47,23 @@ public class TestDFSStripedOutputStream {
     GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
   }
 
-  private ErasureCodingPolicy ecPolicy;
-  private int dataBlocks;
-  private int parityBlocks;
+  private final ErasureCodingPolicy ecPolicy =
+      ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  private final int dataBlocks = ecPolicy.getNumDataUnits();
+  private final int parityBlocks = ecPolicy.getNumParityUnits();
 
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private Configuration conf;
-  private int cellSize;
+  private final int cellSize = ecPolicy.getCellSize();
   private final int stripesPerBlock = 4;
-  private int blockSize;
+  private final int blockSize = cellSize * stripesPerBlock;
 
   @Rule
   public Timeout globalTimeout = new Timeout(300000);
 
-  public ErasureCodingPolicy getEcPolicy() {
-    return ErasureCodingPolicyManager.getSystemDefaultPolicy();
-  }
-
   @Before
   public void setup() throws IOException {
-    /*
-     * Initialize erasure coding policy.
-     */
-    ecPolicy = getEcPolicy();
-    dataBlocks = (short) ecPolicy.getNumDataUnits();
-    parityBlocks = (short) ecPolicy.getNumParityUnits();
-    cellSize = ecPolicy.getCellSize();
-    blockSize = stripesPerBlock * cellSize;
-    System.out.println("EC policy = " + ecPolicy);
-
     int numDNs = dataBlocks + parityBlocks + 2;
     conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
@@ -89,7 +76,7 @@ public class TestDFSStripedOutputStream {
           NativeRSRawErasureCoderFactory.class.getCanonicalName());
     }
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", ecPolicy);
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index 0baf9cc..cde07a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Assume;
-import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -77,36 +76,18 @@ public class TestDFSStripedOutputStreamWithFailure {
         .getLogger().setLevel(Level.ALL);
   }
 
-  private ErasureCodingPolicy ecPolicy;
-  private int dataBlocks;
-  private int parityBlocks;
-  private int cellSize;
+  private final ErasureCodingPolicy ecPolicy =
+      ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  private final int dataBlocks = ecPolicy.getNumDataUnits();
+  private final int parityBlocks = ecPolicy.getNumParityUnits();
+  private final int cellSize = ecPolicy.getCellSize();
   private final int stripesPerBlock = 4;
-  private int blockSize;
-  private int blockGroupSize;
+  private final int blockSize = cellSize * stripesPerBlock;
+  private final int blockGroupSize = blockSize * dataBlocks;
 
   private static final int FLUSH_POS =
       9 * DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
 
-  public ErasureCodingPolicy getEcPolicy() {
-    return ErasureCodingPolicyManager.getSystemDefaultPolicy();
-  }
-
-  /*
-   * Initialize erasure coding policy.
-   */
-  @Before
-  public void init(){
-    ecPolicy = getEcPolicy();
-    dataBlocks = ecPolicy.getNumDataUnits();
-    parityBlocks = ecPolicy.getNumParityUnits();
-    cellSize = ecPolicy.getCellSize();
-    blockSize = cellSize * stripesPerBlock;
-    blockGroupSize = blockSize * dataBlocks;
-    dnIndexSuite = getDnIndexSuite();
-    lengths = newLengths();
-  }
-
   List<Integer> newLengths() {
     final List<Integer> lens = new ArrayList<>();
     lens.add(FLUSH_POS + 2);
@@ -123,7 +104,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     return lens;
   }
 
-  private int[][] dnIndexSuite;
+  private final int[][] dnIndexSuite = getDnIndexSuite();
 
   private int[][] getDnIndexSuite() {
     final int maxNumLevel = 2;
@@ -186,7 +167,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     return positions;
   }
 
-  private List<Integer> lengths;
+  private final List<Integer> lengths = newLengths();
 
   Integer getLength(int i) {
     return i >= 0 && i < lengths.size()? lengths.get(i): null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedInputStream.java
deleted file mode 100644
index 75062e0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedInputStream.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
-
-/**
- * This tests read operation of DFS striped file with XOR-2-1-64k erasure code
- * policy.
- */
-public class TestDFSXORStripedInputStream extends TestDFSStripedInputStream{
-
-  public ErasureCodingPolicy getEcPolicy() {
-    return ErasureCodingPolicyManager.getPolicyByPolicyID(
-        HdfsConstants.XOR_2_1_POLICY_ID);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStream.java
deleted file mode 100644
index 64bddb8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStream.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
-
-/**
- * This tests write operation of DFS striped file with XOR-2-1-64k erasure code
- * policy.
- */
-public class TestDFSXORStripedOutputStream extends TestDFSStripedOutputStream{
-
-  @Override
-  public ErasureCodingPolicy getEcPolicy() {
-    return ErasureCodingPolicyManager.getPolicyByPolicyID(
-        HdfsConstants.XOR_2_1_POLICY_ID);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStreamWithFailure.java
deleted file mode 100644
index ed361a8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStreamWithFailure.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
-
-/**
- * This tests write operation of DFS striped file with XOR-2-1-64k erasure code
- * policy when there is data node failure.
- */
-public class TestDFSXORStripedOutputStreamWithFailure
-    extends TestDFSStripedOutputStreamWithFailure{
-
-  @Override
-  public ErasureCodingPolicy getEcPolicy() {
-    return ErasureCodingPolicyManager.getPolicyByPolicyID(
-        HdfsConstants.XOR_2_1_POLICY_ID);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cfd8076f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForReservedContainers.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/monitor/capacity/TestProportionalCapacityPreemptionPolicyForReservedContainers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForReservedContainers.java
new file mode 100644
index 0000000..38b2e78
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForReservedContainers.java
@@ -0,0 +1,430 @@
+/**
+ * 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.yarn.server.resourcemanager.monitor.capacity;
+
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class TestProportionalCapacityPreemptionPolicyForReservedContainers
+    extends ProportionalCapacityPreemptionPolicyMockFramework {
+  @Before
+  public void setup() {
+    super.setup();
+    conf.setBoolean(
+        CapacitySchedulerConfiguration.PREEMPTION_SELECT_CANDIDATES_FOR_RESERVED_CONTAINERS,
+        true);
+    policy = new ProportionalCapacityPreemptionPolicy(rmContext, cs, mClock);
+  }
+
+  @Test
+  public void testPreemptionForSimpleReservedContainer() throws IOException {
+    /**
+     * The simplest test of reserved container, Queue structure is:
+     *
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     * Guaranteed resource of a/b are 50:50
+     * Total cluster resource = 100
+     * - A has 90 containers on two node, n1 has 45, n2 has 45, size of each
+     * container is 1.
+     * - B has am container at n1, and reserves 1 container with size = 9 at n1,
+     * so B needs to preempt 9 containers from A at n1 instead of randomly
+     * preempt from n1 and n2.
+     */
+    String labelsConfig =
+        "=100,true;";
+    String nodesConfig = // n1 / n2 has no label
+        "n1= res=50;" +
+        "n2= res=50";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 100 9 9]);" + //root
+            "-a(=[50 100 90 0]);" + // a
+            "-b(=[50 100 10 9 9])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+            + "(1,1,n1,,45,false)" // 45 in n1
+            + "(1,1,n2,,45,false);" + // 45 in n2
+        "b\t" // app2 in b
+            + "(1,1,n1,,1,false)" // AM container in n1
+            + "(1,9,n1,,1,true)"; // 1 container with size=9 reserved at n1
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // Total 5 preempted from app1 at n1, don't preempt container from other
+    // app/node
+    verify(mDisp, times(5)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+    verify(mDisp, times(5)).handle(
+        argThat(new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n1", 1))));
+    verify(mDisp, times(0)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(2))));
+  }
+
+  @Test
+  public void testUseReservedAndFifoSelectorTogether() throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     * Guaranteed resource of a/b are 30:70
+     * Total cluster resource = 100
+     * - A has 45 containers on two node, n1 has 10, n2 has 35, size of each
+     * container is 1.
+     * - B has 5 containers at n2, and reserves 1 container with size = 50 at n1,
+     *   B also has 20 pending resources.
+     * so B needs to preempt:
+     * - 10 containers from n1 (for reserved)
+     * - 5 containers from n2 for pending resources
+     */
+    String labelsConfig =
+        "=100,true;";
+    String nodesConfig = // n1 / n2 has no label
+        "n1= res=50;" +
+        "n2= res=50";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 100 70 10]);" + //root
+            "-a(=[30 100 45 0]);" + // a
+            "-b(=[70 100 55 70 50])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+            + "(1,1,n2,,35,false)" // 35 in n2
+            + "(1,1,n1,,10,false);" + // 10 in n1
+            "b\t" // app2 in b
+            + "(1,1,n2,,5,false)" // 5 in n2
+            + "(1,50,n1,,1,true)"; // 1 container with size=50 reserved at n1
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(15)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+    verify(mDisp, times(10)).handle(
+        argThat(new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n1", 1))));
+    verify(mDisp, times(5)).handle(
+        argThat(new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n2", 1))));
+    verify(mDisp, times(0)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(2))));
+  }
+
+  @Test
+  public void testReservedSelectorSkipsAMContainer() throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     * Guaranteed resource of a/b are 30:70
+     * Total cluster resource = 100
+     * - A has 45 containers on two node, n1 has 10, n2 has 35, size of each
+     * container is 1.
+     * - B has 5 containers at n2, and reserves 1 container with size = 50 at n1,
+     *   B also has 20 pending resources.
+     *
+     * Ideally B needs to preempt:
+     * - 10 containers from n1 (for reserved)
+     * - 5 containers from n2 for pending resources
+     *
+     * However, since one AM container is located at n1 (from queueA), we cannot
+     * preempt 10 containers from n1 for reserved container. Instead, we will
+     * preempt 15 containers from n2, since containers from queueA launched in n2
+     * are later than containers from queueA launched in n1 (FIFO order of containers)
+     */
+    String labelsConfig =
+        "=100,true;";
+    String nodesConfig = // n1 / n2 has no label
+        "n1= res=50;" +
+            "n2= res=50";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 100 70 10]);" + //root
+            "-a(=[30 100 45 0]);" + // a
+            "-b(=[70 100 55 70 50])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+            + "(1,1,n1,,10,false)" // 10 in n1
+            + "(1,1,n2,,35,false);" +// 35 in n2
+            "b\t" // app2 in b
+            + "(1,1,n2,,5,false)" // 5 in n2
+            + "(1,50,n1,,1,true)"; // 1 container with size=50 reserved at n1
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(15)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+    verify(mDisp, times(0)).handle(
+        argThat(new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n1", 1))));
+    verify(mDisp, times(15)).handle(
+        argThat(new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n2", 1))));
+    verify(mDisp, times(0)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(2))));
+  }
+
+  @Test
+  public void testPreemptionForReservedContainerRespectGuaranteedResource()
+      throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     * Guaranteed resource of a/b are 85:15
+     * Total cluster resource = 100
+     * - A has 90 containers on two node, n1 has 45, n2 has 45, size of each
+     * container is 1.
+     * - B has am container at n1, and reserves 1 container with size = 9 at n1,
+     *
+     * If we preempt 9 containers from queue-A, queue-A will be below its
+     * guaranteed resource = 90 - 9 = 81 < 85.
+     *
+     * So no preemption will take place
+     */
+    String labelsConfig =
+        "=100,true;";
+    String nodesConfig = // n1 / n2 has no label
+        "n1= res=50;" +
+            "n2= res=50";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 100 9 9]);" + //root
+            "-a(=[85 100 90 0]);" + // a
+            "-b(=[15 100 10 9 9])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+            + "(1,1,n1,,45,false)" // 45 in n1
+            + "(1,1,n2,,45,false);" + // 45 in n2
+            "b\t" // app2 in b
+            + "(1,1,n1,,1,false)" // AM container in n1
+            + "(1,9,n1,,1,true)"; // 1 container with size=9 reserved at n1
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(0)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+    verify(mDisp, times(0)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(2))));
+  }
+
+  @Test
+  public void testPreemptionForReservedContainerWhichHasAvailableResource()
+      throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     *
+     * Guaranteed resource of a/b are 50:50
+     * Total cluster resource = 100
+     * - A has 90 containers on two node, n1 has 45, n2 has 45, size of each
+     * container is 1.
+     * - B has am container at n1, and reserves 1 container with size = 9 at n1,
+     *
+     * So we can get 4 containers preempted after preemption.
+     * (reserved 5 + preempted 4) = 9
+     */
+    String labelsConfig =
+        "=100,true;";
+    String nodesConfig = // n1 / n2 has no label
+        "n1= res=50;" +
+            "n2= res=50";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 99 9 9]);" + //root
+            "-a(=[50 100 90 0]);" + // a
+            "-b(=[50 100 9 9 9])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+            + "(1,1,n1,,45,false)" // 45 in n1
+            + "(1,1,n2,,45,false);" + // 45 in n2
+            "b\t" // app2 in b
+            + "(1,9,n1,,1,true)"; // 1 container with size=9 reserved at n1
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // Total 4 preempted from app1 at n1, don't preempt container from other
+    // app/node
+    verify(mDisp, times(4)).handle(argThat(
+        new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n1", 1))));
+    verify(mDisp, times(0)).handle(argThat(
+        new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n2", 1))));
+  }
+
+  @Test
+  public void testPreemptionForReservedContainerWhichHasNondivisibleAvailableResource()
+      throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     *
+     * Guaranteed resource of a/b are 50:50
+     * Total cluster resource = 100
+     * - A has 45 containers on two node, size of each container is 2,
+     *   n1 has 23, n2 has 22
+     * - B reserves 1 container with size = 9 at n1,
+     *
+     * So we can get 4 containers (total-resource = 8) preempted after
+     * preemption. Actual required is 3.5, but we need to preempt integer
+     * number of containers
+     */
+    String labelsConfig =
+        "=100,true;";
+    String nodesConfig = // n1 / n2 has no label
+        "n1= res=50;" +
+            "n2= res=50";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 99 9 9]);" + //root
+            "-a(=[50 100 90 0]);" + // a
+            "-b(=[50 100 9 9 9])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+            + "(1,2,n1,,24,false)" // 48 in n1
+            + "(1,2,n2,,23,false);" + // 46 in n2
+            "b\t" // app2 in b
+            + "(1,9,n1,,1,true)"; // 1 container with size=9 reserved at n1
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // Total 4 preempted from app1 at n1, don't preempt container from other
+    // app/node
+    verify(mDisp, times(4)).handle(argThat(
+        new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n1", 1))));
+    verify(mDisp, times(0)).handle(argThat(
+        new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n2", 1))));
+  }
+
+  @Test
+  public void testPreemptionForReservedContainerRespectAvailableResources()
+      throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     *
+     * Guaranteed resource of a/b are 50:50
+     * Total cluster resource = 100, 4 nodes, 25 on each node
+     * - A has 10 containers on every node, size of container is 2
+     * - B reserves 1 container with size = 9 at n1,
+     *
+     * So even if we cannot allocate container for B now, no preemption should
+     * happen since there're plenty of available resources.
+     */
+    String labelsConfig =
+        "=100,true;";
+    String nodesConfig =
+        "n1= res=25;" +
+            "n2= res=25;" +
+            "n3= res=25;" +
+            "n4= res=25;";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 89 9 9]);" + //root
+            "-a(=[50 100 80 0]);" + // a
+            "-b(=[50 100 9 9 9])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+            + "(1,2,n1,,10,false)" // 10 in n1
+            + "(1,2,n2,,10,false)" // 10 in n2
+            + "(1,2,n3,,10,false)" // 10 in n3
+            + "(1,2,n4,,10,false);" + // 10 in n4
+            "b\t" // app2 in b
+            + "(1,9,n1,,1,true)"; // 1 container with size=5 reserved at n1
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // No preemption should happen
+    verify(mDisp, times(0)).handle(argThat(
+        new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n1", 1))));
+    verify(mDisp, times(0)).handle(argThat(
+        new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n2", 1))));
+    verify(mDisp, times(0)).handle(argThat(
+        new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n3", 1))));
+    verify(mDisp, times(0)).handle(argThat(
+        new IsPreemptionRequestForQueueAndNode(getAppAttemptId(1), "a",
+            NodeId.newInstance("n4", 1))));
+  }
+}


---------------------------------------------------------------------
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: HDFS-10994. Support an XOR policy XOR-2-1-64k in HDFS. Contributed by Sammi Chen

Posted by xg...@apache.org.
HDFS-10994. Support an XOR policy XOR-2-1-64k in HDFS. Contributed by Sammi Chen


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

Branch: refs/heads/YARN-5734
Commit: 51e6c1cc3f66f9908d2e816e7291ac34bee43f52
Parents: cfd8076
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Nov 30 15:52:56 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Nov 30 15:52:56 2016 +0800

----------------------------------------------------------------------
 .../io/erasurecode/ErasureCodeConstants.java    |  3 ++
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  1 +
 .../namenode/ErasureCodingPolicyManager.java    | 23 +++++++--
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  8 +++-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     | 28 +++++++++--
 .../hadoop/hdfs/TestDFSStripedInputStream.java  | 50 +++++++++++++-------
 .../hadoop/hdfs/TestDFSStripedOutputStream.java | 27 ++++++++---
 .../TestDFSStripedOutputStreamWithFailure.java  | 37 +++++++++++----
 .../hdfs/TestDFSXORStripedInputStream.java      | 33 +++++++++++++
 .../hdfs/TestDFSXORStripedOutputStream.java     | 35 ++++++++++++++
 ...estDFSXORStripedOutputStreamWithFailure.java | 36 ++++++++++++++
 11 files changed, 240 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
index 8d6ff85..ffa0bce 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
@@ -38,4 +38,7 @@ public final class ErasureCodeConstants {
 
   public static final ECSchema RS_6_3_LEGACY_SCHEMA = new ECSchema(
       RS_LEGACY_CODEC_NAME, 6, 3);
+
+  public static final ECSchema XOR_2_1_SCHEMA = new ECSchema(
+      XOR_CODEC_NAME, 2, 1);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index acbc8f6..b55b4df 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -147,6 +147,7 @@ public final class HdfsConstants {
   public static final byte RS_6_3_POLICY_ID = 0;
   public static final byte RS_3_2_POLICY_ID = 1;
   public static final byte RS_6_3_LEGACY_POLICY_ID = 2;
+  public static final byte XOR_2_1_POLICY_ID = 3;
 
   /* Hidden constructor */
   protected HdfsConstants() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index c4bc8de..8a85d23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -36,7 +36,7 @@ import java.util.TreeMap;
 public final class ErasureCodingPolicyManager {
 
   /**
-   * TODO: HDFS-8095
+   * TODO: HDFS-8095.
    */
   private static final int DEFAULT_CELLSIZE = 64 * 1024;
   private static final ErasureCodingPolicy SYS_POLICY1 =
@@ -48,10 +48,14 @@ public final class ErasureCodingPolicyManager {
   private static final ErasureCodingPolicy SYS_POLICY3 =
       new ErasureCodingPolicy(ErasureCodeConstants.RS_6_3_LEGACY_SCHEMA,
           DEFAULT_CELLSIZE, HdfsConstants.RS_6_3_LEGACY_POLICY_ID);
+  private static final ErasureCodingPolicy SYS_POLICY4 =
+      new ErasureCodingPolicy(ErasureCodeConstants.XOR_2_1_SCHEMA,
+          DEFAULT_CELLSIZE, HdfsConstants.XOR_2_1_POLICY_ID);
 
   //We may add more later.
   private static final ErasureCodingPolicy[] SYS_POLICIES =
-      new ErasureCodingPolicy[]{SYS_POLICY1, SYS_POLICY2, SYS_POLICY3};
+      new ErasureCodingPolicy[]{SYS_POLICY1, SYS_POLICY2, SYS_POLICY3,
+          SYS_POLICY4};
 
   // Supported storage policies for striped EC files
   private static final byte[] SUITABLE_STORAGE_POLICIES_FOR_EC_STRIPED_MODE = new byte[] {
@@ -97,6 +101,19 @@ public final class ErasureCodingPolicyManager {
   }
 
   /**
+   * Get system-wide policy by policy ID.
+   * @return ecPolicy
+   */
+  public static ErasureCodingPolicy getPolicyByPolicyID(byte id) {
+    for (ErasureCodingPolicy policy : SYS_POLICIES) {
+      if (policy.getId() == id) {
+        return policy;
+      }
+    }
+    return null;
+  }
+
+  /**
    * Get all policies that's available to use.
    * @return all policies
    */
@@ -141,7 +158,7 @@ public final class ErasureCodingPolicyManager {
   }
 
   /**
-   * Clear and clean up
+   * Clear and clean up.
    */
   public void clear() {
     activePoliciesByName.clear();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 37f97db..a5dcee9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -455,9 +455,13 @@ public class INodeFile extends INodeWithAdditionalFields
     if(!isStriped()){
       return max;
     }
-    // TODO support more policies based on policyId
+
     ErasureCodingPolicy ecPolicy =
-        ErasureCodingPolicyManager.getSystemDefaultPolicy();
+        ErasureCodingPolicyManager.getPolicyByPolicyID(
+            getErasureCodingPolicyID());
+    if (ecPolicy == null){
+      ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    }
     return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 13e2656..1fbc1d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -1888,21 +1888,41 @@ public class DFSTestUtil {
    * Creates the metadata of a file in striped layout. This method only
    * manipulates the NameNode state without injecting data to DataNode.
    * You should disable periodical heartbeat before use this.
-   *  @param file Path of the file to create
+   * @param file Path of the file to create
    * @param dir Parent path of the file
    * @param numBlocks Number of striped block groups to add to the file
    * @param numStripesPerBlk Number of striped cells in each block
    * @param toMkdir
    */
-  public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir,
-      int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception {
+  public static void createStripedFile(MiniDFSCluster cluster, Path file,
+      Path dir, int numBlocks, int numStripesPerBlk, boolean toMkdir)
+      throws Exception {
+    createStripedFile(cluster, file, dir, numBlocks, numStripesPerBlk,
+        toMkdir, null);
+  }
+
+  /**
+   * Creates the metadata of a file in striped layout. This method only
+   * manipulates the NameNode state without injecting data to DataNode.
+   * You should disable periodical heartbeat before use this.
+   * @param file Path of the file to create
+   * @param dir Parent path of the file
+   * @param numBlocks Number of striped block groups to add to the file
+   * @param numStripesPerBlk Number of striped cells in each block
+   * @param toMkdir
+   * @param ecPolicy erasure coding policy apply to created file. A null value
+   *                 means using default erasure coding policy.
+   */
+  public static void createStripedFile(MiniDFSCluster cluster, Path file,
+      Path dir, int numBlocks, int numStripesPerBlk, boolean toMkdir,
+      ErasureCodingPolicy ecPolicy) throws Exception {
     DistributedFileSystem dfs = cluster.getFileSystem();
     // If outer test already set EC policy, dir should be left as null
     if (toMkdir) {
       assert dir != null;
       dfs.mkdirs(dir);
       try {
-        dfs.getClient().setErasureCodingPolicy(dir.toString(), null);
+        dfs.getClient().setErasureCodingPolicy(dir.toString(), ecPolicy);
       } catch (IOException e) {
         if (!e.getMessage().contains("non-empty directory")) {
           throw e;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
index 3b46c66..121b9a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
@@ -64,20 +64,34 @@ public class TestDFSStripedInputStream {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
-  private final ErasureCodingPolicy ecPolicy =
-      ErasureCodingPolicyManager.getSystemDefaultPolicy();
-  private final short dataBlocks = (short) ecPolicy.getNumDataUnits();
-  private final short parityBlocks = (short) ecPolicy.getNumParityUnits();
-  private final int cellSize = ecPolicy.getCellSize();
+  private ErasureCodingPolicy ecPolicy;
+  private short dataBlocks;
+  private short parityBlocks;
+  private int cellSize;
   private final int stripesPerBlock = 2;
-  private final int blockSize = stripesPerBlock * cellSize;
-  private final int blockGroupSize =  dataBlocks * blockSize;
+  private int blockSize;
+  private int blockGroupSize;
 
   @Rule
   public Timeout globalTimeout = new Timeout(300000);
 
+  public ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  }
+
   @Before
   public void setup() throws IOException {
+    /*
+     * Initialize erasure coding policy.
+     */
+    ecPolicy = getEcPolicy();
+    dataBlocks = (short) ecPolicy.getNumDataUnits();
+    parityBlocks = (short) ecPolicy.getNumParityUnits();
+    cellSize = ecPolicy.getCellSize();
+    blockSize = stripesPerBlock * cellSize;
+    blockGroupSize =  dataBlocks * blockSize;
+    System.out.println("EC policy = " + ecPolicy);
+
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     if (ErasureCodeNative.isNativeCodeLoaded()) {
@@ -94,7 +108,7 @@ public class TestDFSStripedInputStream {
     }
     fs = cluster.getFileSystem();
     fs.mkdirs(dirPath);
-    fs.getClient().setErasureCodingPolicy(dirPath.toString(), null);
+    fs.getClient().setErasureCodingPolicy(dirPath.toString(), ecPolicy);
   }
 
   @After
@@ -106,13 +120,13 @@ public class TestDFSStripedInputStream {
   }
 
   /**
-   * Test {@link DFSStripedInputStream#getBlockAt(long)}
+   * Test {@link DFSStripedInputStream#getBlockAt(long)}.
    */
   @Test
   public void testRefreshBlock() throws Exception {
     final int numBlocks = 4;
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false);
+        stripesPerBlock, false, ecPolicy);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, blockGroupSize * numBlocks);
     final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
@@ -136,7 +150,7 @@ public class TestDFSStripedInputStream {
   public void testPread() throws Exception {
     final int numBlocks = 2;
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false);
+        stripesPerBlock, false, ecPolicy);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, blockGroupSize * numBlocks);
     int fileLen = blockGroupSize * numBlocks;
@@ -154,7 +168,9 @@ public class TestDFSStripedInputStream {
             bg.getBlock().getBlockPoolId());
       }
 
-      /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
+      /**
+       * A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks
+       */
       for (int i = 0; i < stripesPerBlock; i++) {
         for (int j = 0; j < dataBlocks; j++) {
           for (int k = 0; k < cellSize; k++) {
@@ -194,7 +210,7 @@ public class TestDFSStripedInputStream {
     final int numBlocks = 4;
     final int failedDNIdx = dataBlocks - 1;
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false);
+        stripesPerBlock, false, ecPolicy);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, blockGroupSize);
 
@@ -305,7 +321,7 @@ public class TestDFSStripedInputStream {
       setup();
     }
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false);
+        stripesPerBlock, false, ecPolicy);
     LocatedBlocks lbs = fs.getClient().namenode.
         getBlockLocations(filePath.toString(), 0, fileSize);
 
@@ -330,7 +346,9 @@ public class TestDFSStripedInputStream {
     byte[] expected = new byte[fileSize];
 
     for (LocatedBlock bg : lbs.getLocatedBlocks()) {
-      /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
+      /**
+       * A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks
+       */
       for (int i = 0; i < stripesPerBlock; i++) {
         for (int j = 0; j < dataBlocks; j++) {
           for (int k = 0; k < cellSize; k++) {
@@ -371,7 +389,7 @@ public class TestDFSStripedInputStream {
     final int numBlocks = 4;
     final int failedDNIdx = dataBlocks - 1;
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        stripesPerBlock, false);
+        stripesPerBlock, false, ecPolicy);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, blockGroupSize);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index b686f28..5bde16e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -47,23 +47,36 @@ public class TestDFSStripedOutputStream {
     GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
   }
 
-  private final ErasureCodingPolicy ecPolicy =
-      ErasureCodingPolicyManager.getSystemDefaultPolicy();
-  private final int dataBlocks = ecPolicy.getNumDataUnits();
-  private final int parityBlocks = ecPolicy.getNumParityUnits();
+  private ErasureCodingPolicy ecPolicy;
+  private int dataBlocks;
+  private int parityBlocks;
 
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private Configuration conf;
-  private final int cellSize = ecPolicy.getCellSize();
+  private int cellSize;
   private final int stripesPerBlock = 4;
-  private final int blockSize = cellSize * stripesPerBlock;
+  private int blockSize;
 
   @Rule
   public Timeout globalTimeout = new Timeout(300000);
 
+  public ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  }
+
   @Before
   public void setup() throws IOException {
+    /*
+     * Initialize erasure coding policy.
+     */
+    ecPolicy = getEcPolicy();
+    dataBlocks = (short) ecPolicy.getNumDataUnits();
+    parityBlocks = (short) ecPolicy.getNumParityUnits();
+    cellSize = ecPolicy.getCellSize();
+    blockSize = stripesPerBlock * cellSize;
+    System.out.println("EC policy = " + ecPolicy);
+
     int numDNs = dataBlocks + parityBlocks + 2;
     conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
@@ -76,7 +89,7 @@ public class TestDFSStripedOutputStream {
           NativeRSRawErasureCoderFactory.class.getCanonicalName());
     }
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", ecPolicy);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index cde07a4..0baf9cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Assume;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -76,18 +77,36 @@ public class TestDFSStripedOutputStreamWithFailure {
         .getLogger().setLevel(Level.ALL);
   }
 
-  private final ErasureCodingPolicy ecPolicy =
-      ErasureCodingPolicyManager.getSystemDefaultPolicy();
-  private final int dataBlocks = ecPolicy.getNumDataUnits();
-  private final int parityBlocks = ecPolicy.getNumParityUnits();
-  private final int cellSize = ecPolicy.getCellSize();
+  private ErasureCodingPolicy ecPolicy;
+  private int dataBlocks;
+  private int parityBlocks;
+  private int cellSize;
   private final int stripesPerBlock = 4;
-  private final int blockSize = cellSize * stripesPerBlock;
-  private final int blockGroupSize = blockSize * dataBlocks;
+  private int blockSize;
+  private int blockGroupSize;
 
   private static final int FLUSH_POS =
       9 * DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
 
+  public ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  }
+
+  /*
+   * Initialize erasure coding policy.
+   */
+  @Before
+  public void init(){
+    ecPolicy = getEcPolicy();
+    dataBlocks = ecPolicy.getNumDataUnits();
+    parityBlocks = ecPolicy.getNumParityUnits();
+    cellSize = ecPolicy.getCellSize();
+    blockSize = cellSize * stripesPerBlock;
+    blockGroupSize = blockSize * dataBlocks;
+    dnIndexSuite = getDnIndexSuite();
+    lengths = newLengths();
+  }
+
   List<Integer> newLengths() {
     final List<Integer> lens = new ArrayList<>();
     lens.add(FLUSH_POS + 2);
@@ -104,7 +123,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     return lens;
   }
 
-  private final int[][] dnIndexSuite = getDnIndexSuite();
+  private int[][] dnIndexSuite;
 
   private int[][] getDnIndexSuite() {
     final int maxNumLevel = 2;
@@ -167,7 +186,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     return positions;
   }
 
-  private final List<Integer> lengths = newLengths();
+  private List<Integer> lengths;
 
   Integer getLength(int i) {
     return i >= 0 && i < lengths.size()? lengths.get(i): null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedInputStream.java
new file mode 100644
index 0000000..75062e0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedInputStream.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;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+
+/**
+ * This tests read operation of DFS striped file with XOR-2-1-64k erasure code
+ * policy.
+ */
+public class TestDFSXORStripedInputStream extends TestDFSStripedInputStream{
+
+  public ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getPolicyByPolicyID(
+        HdfsConstants.XOR_2_1_POLICY_ID);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStream.java
new file mode 100644
index 0000000..64bddb8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStream.java
@@ -0,0 +1,35 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+
+/**
+ * This tests write operation of DFS striped file with XOR-2-1-64k erasure code
+ * policy.
+ */
+public class TestDFSXORStripedOutputStream extends TestDFSStripedOutputStream{
+
+  @Override
+  public ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getPolicyByPolicyID(
+        HdfsConstants.XOR_2_1_POLICY_ID);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51e6c1cc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStreamWithFailure.java
new file mode 100644
index 0000000..ed361a8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStreamWithFailure.java
@@ -0,0 +1,36 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+
+/**
+ * This tests write operation of DFS striped file with XOR-2-1-64k erasure code
+ * policy when there is data node failure.
+ */
+public class TestDFSXORStripedOutputStreamWithFailure
+    extends TestDFSStripedOutputStreamWithFailure{
+
+  @Override
+  public ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getPolicyByPolicyID(
+        HdfsConstants.XOR_2_1_POLICY_ID);
+  }
+}
\ No newline at end of file


---------------------------------------------------------------------
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-13847. KMSWebApp should close KeyProviderCryptoExtension. Contributed by John Zhuge.

Posted by xg...@apache.org.
HADOOP-13847. KMSWebApp should close KeyProviderCryptoExtension. 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/291df5c7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/291df5c7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/291df5c7

Branch: refs/heads/YARN-5734
Commit: 291df5c7fb713d5442ee29eb3f272127afb05a3c
Parents: c51bfd2
Author: Xiao Chen <xi...@apache.org>
Authored: Mon Dec 5 09:34:39 2016 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Mon Dec 5 09:35:17 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/crypto/key/KeyProviderCryptoExtension.java  | 5 +++--
 .../org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java    | 7 ++++++-
 2 files changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/291df5c7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
index 1ecd9f6..0543222 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
@@ -427,8 +427,9 @@ public class KeyProviderCryptoExtension extends
 
   @Override
   public void close() throws IOException {
-    if (getKeyProvider() != null) {
-      getKeyProvider().close();
+    KeyProvider provider = getKeyProvider();
+    if (provider != null && provider != this) {
+      provider.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/291df5c7/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
index cd773dd..40ae19f 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
@@ -40,9 +40,9 @@ import javax.servlet.ServletContextEvent;
 import javax.servlet.ServletContextListener;
 
 import java.io.File;
+import java.io.IOException;
 import java.net.URI;
 import java.net.URL;
-import java.util.List;
 
 @InterfaceAudience.Private
 public class KMSWebApp implements ServletContextListener {
@@ -215,6 +215,11 @@ public class KMSWebApp implements ServletContextListener {
 
   @Override
   public void contextDestroyed(ServletContextEvent sce) {
+    try {
+      keyProviderCryptoExtension.close();
+    } catch (IOException ioe) {
+      LOG.error("Error closing KeyProviderCryptoExtension", ioe);
+    }
     kmsAudit.shutdown();
     kmsAcls.stopReloader();
     jmxReporter.stop();


---------------------------------------------------------------------
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: HDFS-11180. Intermittent deadlock in NameNode when failover happens.

Posted by xg...@apache.org.
HDFS-11180. Intermittent deadlock in NameNode when failover happens.


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

Branch: refs/heads/YARN-5734
Commit: e0fa49234fd37aca88e1caa95bac77bca192bae4
Parents: 1f7613b
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 1 23:08:59 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 1 23:08:59 2016 +0900

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         | 27 ++++++++
 .../hadoop/hdfs/server/namenode/FSEditLog.java  | 72 +++++++++++++++++---
 .../hadoop/hdfs/server/namenode/FSImage.java    | 15 +++-
 .../hdfs/server/namenode/FSNamesystem.java      | 27 ++++++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  2 +-
 .../server/namenode/ha/StandbyCheckpointer.java |  4 +-
 .../server/namenode/TestFSNamesystemMBean.java  | 24 +++++++
 7 files changed, 148 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0fa4923/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 426fb72..e6e4057 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -109,6 +109,33 @@
         <Field name="journalSet" />
         <Bug pattern="IS2_INCONSISTENT_SYNC" />
     </Match>
+    <!--
+      FSEditLog#getTotalSyncCount is not synchronized because this method is
+      used by metrics. NullPointerException can happen and it is ignored.
+    -->
+    <Match>
+      <Class name="org.apache.hadoop.hdfs.server.namenode.FSEditLog" />
+      <Field name="editLogStream" />
+      <Bug pattern="IS2_INCONSISTENT_SYNC" />
+    </Match>
+    <!--
+      FSEditLog#isOpenForWriteWithoutLock and FSEditLog#isSegmentOpenWithoutLock
+      are not synchronized because these methods are used by metrics.
+    -->
+    <Match>
+      <Class name="org.apache.hadoop.hdfs.server.namenode.FSEditLog" />
+      <Field name="state" />
+      <Bug pattern="IS2_INCONSISTENT_SYNC" />
+    </Match>
+    <!--
+      All of the threads which update/increment txid are synchronized,
+      so make txid volatile instead of AtomicLong.
+    -->
+    <Match>
+      <Class name="org.apache.hadoop.hdfs.server.namenode.FSEditLog" />
+      <Field name="txid" />
+      <Bug pattern="VO_VOLATILE_INCREMENT" />
+    </Match>
      <!--
       This method isn't performance-critical and is much clearer to write as it's written.
       -->

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0fa4923/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index ef9eb68..c9ee32b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -155,14 +155,16 @@ public class FSEditLog implements LogsPurgeable {
   private EditLogOutputStream editLogStream = null;
 
   // a monotonically increasing counter that represents transactionIds.
-  private long txid = 0;
+  // All of the threads which update/increment txid are synchronized,
+  // so make txid volatile instead of AtomicLong.
+  private volatile long txid = 0;
 
   // stores the last synced transactionId.
   private long synctxid = 0;
 
   // the first txid of the log that's currently open for writing.
   // If this value is N, we are currently writing to edits_inprogress_N
-  private long curSegmentTxId = HdfsServerConstants.INVALID_TXID;
+  private volatile long curSegmentTxId = HdfsServerConstants.INVALID_TXID;
 
   // the time of printing the statistics to the log file.
   private long lastPrintTime;
@@ -338,7 +340,18 @@ public class FSEditLog implements LogsPurgeable {
     return state == State.IN_SEGMENT ||
       state == State.BETWEEN_LOG_SEGMENTS;
   }
-  
+
+  /**
+   * Return true if the log is currently open in write mode.
+   * This method is not synchronized and must be used only for metrics.
+   * @return true if the log is currently open in write mode, regardless
+   * of whether it actually has an open segment.
+   */
+  boolean isOpenForWriteWithoutLock() {
+    return state == State.IN_SEGMENT ||
+        state == State.BETWEEN_LOG_SEGMENTS;
+  }
+
   /**
    * @return true if the log is open in write mode and has a segment open
    * ready to take edits.
@@ -348,6 +361,16 @@ public class FSEditLog implements LogsPurgeable {
   }
 
   /**
+   * Return true the state is IN_SEGMENT.
+   * This method is not synchronized and must be used only for metrics.
+   * @return true if the log is open in write mode and has a segment open
+   * ready to take edits.
+   */
+  boolean isSegmentOpenWithoutLock() {
+    return state == State.IN_SEGMENT;
+  }
+
+  /**
    * @return true if the log is open in read mode.
    */
   public synchronized boolean isOpenForRead() {
@@ -522,7 +545,16 @@ public class FSEditLog implements LogsPurgeable {
   public synchronized long getLastWrittenTxId() {
     return txid;
   }
-  
+
+  /**
+   * Return the transaction ID of the last transaction written to the log.
+   * This method is not synchronized and must be used only for metrics.
+   * @return The transaction ID of the last transaction written to the log
+   */
+  long getLastWrittenTxIdWithoutLock() {
+    return txid;
+  }
+
   /**
    * @return the first transaction ID in the current log segment
    */
@@ -531,7 +563,16 @@ public class FSEditLog implements LogsPurgeable {
         "Bad state: %s", state);
     return curSegmentTxId;
   }
-  
+
+  /**
+   * Return the first transaction ID in the current log segment.
+   * This method is not synchronized and must be used only for metrics.
+   * @return The first transaction ID in the current log segment
+   */
+  long getCurSegmentTxIdWithoutLock() {
+    return curSegmentTxId;
+  }
+
   /**
    * Set the transaction ID to use for the next transaction written.
    */
@@ -1182,7 +1223,9 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * Get all the journals this edit log is currently operating on.
    */
-  synchronized List<JournalAndStream> getJournals() {
+  List<JournalAndStream> getJournals() {
+    // The list implementation is CopyOnWriteArrayList,
+    // so we don't need to synchronize this method.
     return journalSet.getAllJournalStreams();
   }
   
@@ -1190,7 +1233,7 @@ public class FSEditLog implements LogsPurgeable {
    * Used only by tests.
    */
   @VisibleForTesting
-  synchronized public JournalSet getJournalSet() {
+  public JournalSet getJournalSet() {
     return journalSet;
   }
   
@@ -1731,11 +1774,18 @@ public class FSEditLog implements LogsPurgeable {
    * Return total number of syncs happened on this edit log.
    * @return long - count
    */
-  public synchronized long getTotalSyncCount() {
-    if (editLogStream != null) {
-      return editLogStream.getNumSync();
-    } else {
+  public long getTotalSyncCount() {
+    // Avoid NPE as possible.
+    if (editLogStream == null) {
       return 0;
     }
+    long count = 0;
+    try {
+      count = editLogStream.getNumSync();
+    } catch (NullPointerException ignore) {
+      // This method is used for metrics, so we don't synchronize it.
+      // Therefore NPE can happen even if there is a null check before.
+    }
+    return count;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0fa4923/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index d02a6ca..c40c626 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -924,7 +924,7 @@ public class FSImage implements Closeable {
       Canceler canceler) throws IOException {
     FSImageCompression compression =
         FSImageCompression.createCompression(conf);
-    long txid = getLastAppliedOrWrittenTxId();
+    long txid = getCorrectLastAppliedOrWrittenTxId();
     SaveNamespaceContext ctx = new SaveNamespaceContext(source, txid,
         canceler);
     FSImageFormat.Saver saver = new FSImageFormat.Saver(ctx);
@@ -1019,7 +1019,7 @@ public class FSImage implements Closeable {
       final long checkpointTxId = image.getCheckpointTxId();
       final long checkpointAge = Time.now() - imageFile.lastModified();
       if (checkpointAge <= timeWindow * 1000 &&
-          checkpointTxId >= this.getLastAppliedOrWrittenTxId() - txGap) {
+          checkpointTxId >= this.getCorrectLastAppliedOrWrittenTxId() - txGap) {
         return false;
       }
     }
@@ -1046,7 +1046,7 @@ public class FSImage implements Closeable {
     if (editLogWasOpen) {
       editLog.endCurrentLogSegment(true);
     }
-    long imageTxId = getLastAppliedOrWrittenTxId();
+    long imageTxId = getCorrectLastAppliedOrWrittenTxId();
     if (!addToCheckpointing(imageTxId)) {
       throw new IOException(
           "FS image is being downloaded from another NN at txid " + imageTxId);
@@ -1418,6 +1418,15 @@ public class FSImage implements Closeable {
 
   public long getLastAppliedOrWrittenTxId() {
     return Math.max(lastAppliedTxId,
+        editLog != null ? editLog.getLastWrittenTxIdWithoutLock() : 0);
+  }
+
+  /**
+   * This method holds a lock of FSEditLog to get the correct value.
+   * This method must not be used for metrics.
+   */
+  public long getCorrectLastAppliedOrWrittenTxId() {
+    return Math.max(lastAppliedTxId,
         editLog != null ? editLog.getLastWrittenTxId() : 0);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0fa4923/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 c78ef46..8a750a0 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
@@ -3654,7 +3654,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       //create ha status
       final NNHAStatusHeartbeat haState = new NNHAStatusHeartbeat(
           haContext.getState().getServiceState(),
-          getFSImage().getLastAppliedOrWrittenTxId());
+          getFSImage().getCorrectLastAppliedOrWrittenTxId());
 
       return new HeartbeatResponse(cmds, haState, rollingUpgradeInfo,
           blockReportLeaseId);
@@ -3779,7 +3779,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     public void run() {
       while (fsRunning && shouldRun) {
         try {
-          long numEdits = getTransactionsSinceLastLogRoll();
+          long numEdits = getCorrectTransactionsSinceLastLogRoll();
           if (numEdits > rollThreshold) {
             FSNamesystem.LOG.info("NameNode rolling its own edit log because"
                 + " number of edits in open segment exceeds threshold of "
@@ -3928,17 +3928,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   @Metric({"TransactionsSinceLastLogRoll",
       "Number of transactions since last edit log roll"})
   public long getTransactionsSinceLastLogRoll() {
+    if (isInStandbyState() || !getEditLog().isSegmentOpenWithoutLock()) {
+      return 0;
+    } else {
+      return getEditLog().getLastWrittenTxIdWithoutLock() -
+          getEditLog().getCurSegmentTxIdWithoutLock() + 1;
+    }
+  }
+
+  /**
+   * Get the correct number of transactions since last edit log roll.
+   * This method holds a lock of FSEditLog and must not be used for metrics.
+   */
+  private long getCorrectTransactionsSinceLastLogRoll() {
     if (isInStandbyState() || !getEditLog().isSegmentOpen()) {
       return 0;
     } else {
       return getEditLog().getLastWrittenTxId() -
-        getEditLog().getCurSegmentTxId() + 1;
+          getEditLog().getCurSegmentTxId() + 1;
     }
   }
-  
+
   @Metric({"LastWrittenTransactionId", "Transaction ID written to the edit log"})
   public long getLastWrittenTransactionId() {
-    return getEditLog().getLastWrittenTxId();
+    return getEditLog().getLastWrittenTxIdWithoutLock();
   }
   
   @Metric({"LastCheckpointTime",
@@ -5613,7 +5626,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     List<Map<String, String>> jasList = new ArrayList<Map<String, String>>();
     FSEditLog log = getFSImage().getEditLog();
     if (log != null) {
-      boolean openForWrite = log.isOpenForWrite();
+      // This flag can be false because we cannot hold a lock of FSEditLog
+      // for metrics.
+      boolean openForWrite = log.isOpenForWriteWithoutLock();
       for (JournalAndStream jas : log.getJournals()) {
         final Map<String, String> jasMap = new HashMap<String, String>();
         String manager = jas.getManager().toString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0fa4923/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index c55c4d1..0fc3e60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -1215,7 +1215,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     checkNNStartup();
     namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.checkSuperuserPrivilege();
-    return namesystem.getFSImage().getLastAppliedOrWrittenTxId();
+    return namesystem.getFSImage().getCorrectLastAppliedOrWrittenTxId();
   }
   
   @Override // NamenodeProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0fa4923/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
index 099a6aa..569c6bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
@@ -172,7 +172,7 @@ public class StandbyCheckpointer {
       FSImage img = namesystem.getFSImage();
 
       long prevCheckpointTxId = img.getStorage().getMostRecentCheckpointTxId();
-      long thisCheckpointTxId = img.getLastAppliedOrWrittenTxId();
+      long thisCheckpointTxId = img.getCorrectLastAppliedOrWrittenTxId();
       assert thisCheckpointTxId >= prevCheckpointTxId;
       if (thisCheckpointTxId == prevCheckpointTxId) {
         LOG.info("A checkpoint was triggered but the Standby Node has not " +
@@ -315,7 +315,7 @@ public class StandbyCheckpointer {
 
   private long countUncheckpointedTxns() {
     FSImage img = namesystem.getFSImage();
-    return img.getLastAppliedOrWrittenTxId() -
+    return img.getCorrectLastAppliedOrWrittenTxId() -
       img.getStorage().getMostRecentCheckpointTxId();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0fa4923/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
index 4687cfd..d311494 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
@@ -162,6 +162,30 @@ public class TestFSNamesystemMBean {
     }
   }
 
+  // The test makes sure JMX request can be processed even if FSEditLog
+  // is synchronized.
+  @Test
+  public void testWithFSEditLogLock() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      synchronized (cluster.getNameNode().getFSImage().getEditLog()) {
+        MBeanClient client = new MBeanClient();
+        client.start();
+        client.join(20000);
+        assertTrue("JMX calls are blocked when FSEditLog" +
+            " is synchronized by another thread", client.succeeded);
+        client.interrupt();
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   @Test(timeout = 120000)
   public void testFsEditLogMetrics() throws Exception {
     final Configuration conf = new Configuration();


---------------------------------------------------------------------
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: YARN-5942. "Overridden" is misspelled as "overriden" in FairScheduler.md (Contributed by Heather Sutherland via Daniel Templeton)

Posted by xg...@apache.org.
YARN-5942. "Overridden" is misspelled as "overriden" in FairScheduler.md
(Contributed by Heather Sutherland via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: 4fca94fbdad16e845e670758939aabb7a97154d9
Parents: be5a757
Author: Daniel Templeton <te...@apache.org>
Authored: Wed Nov 30 11:22:21 2016 -0800
Committer: Daniel Templeton <te...@apache.org>
Committed: Wed Nov 30 11:23:51 2016 -0800

----------------------------------------------------------------------
 .../hadoop-yarn-site/src/site/markdown/FairScheduler.md      | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fca94fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
index ecbb309..ae4c3ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
@@ -129,13 +129,13 @@ The allocation file must be in XML format. The format contains five types of ele
 
 * **A defaultFairSharePreemptionThreshold element**: which sets the fair share preemption threshold for the root queue; overridden by fairSharePreemptionThreshold element in root queue.
 
-* **A queueMaxAppsDefault element**: which sets the default running app limit for queues; overriden by maxRunningApps element in each queue.
+* **A queueMaxAppsDefault element**: which sets the default running app limit for queues; overridden by maxRunningApps element in each queue.
 
-* **A queueMaxResourcesDefault element**: which sets the default max resource limit for queue; overriden by maxResources element in each queue.
+* **A queueMaxResourcesDefault element**: which sets the default max resource limit for queue; overridden by maxResources element in each queue.
 
-* **A queueMaxAMShareDefault element**: which sets the default AM resource limit for queue; overriden by maxAMShare element in each queue.
+* **A queueMaxAMShareDefault element**: which sets the default AM resource limit for queue; overridden by maxAMShare element in each queue.
 
-* **A defaultQueueSchedulingPolicy element**: which sets the default scheduling policy for queues; overriden by the schedulingPolicy element in each queue if specified. Defaults to "fair".
+* **A defaultQueueSchedulingPolicy element**: which sets the default scheduling policy for queues; overridden by the schedulingPolicy element in each queue if specified. Defaults to "fair".
 
 * **A queuePlacementPolicy element**: which contains a list of rule elements that tell the scheduler how to place incoming apps into queues. Rules are applied in the order that they are listed. Rules may take arguments. All rules accept the "create" argument, which indicates whether the rule can create a new queue. "Create" defaults to true; if set to false and the rule would place the app in a queue that is not configured in the allocations file, we continue on to the next rule. The last rule must be one that can never issue a continue. Valid rules are:
 


---------------------------------------------------------------------
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-5517. Lower the default maximum number of blocks per file. Contributed by Aaron T. Myers and Andrew Wang.

Posted by xg...@apache.org.
HDFS-5517. Lower the default maximum number of blocks per file. Contributed by Aaron T. Myers 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/7226a71b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7226a71b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7226a71b

Branch: refs/heads/YARN-5734
Commit: 7226a71b1f684f562bd88ee121f1dd7aa8b73816
Parents: 69fb70c
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Nov 30 15:58:31 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Nov 30 15:58:31 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java  |  2 +-
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml      |  2 +-
 .../hdfs/server/datanode/TestDirectoryScanner.java       | 11 +++++++++--
 .../server/namenode/metrics/TestNameNodeMetrics.java     |  2 +-
 4 files changed, 12 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7226a71b/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 d7d3c9d..df21857 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
@@ -399,7 +399,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_MIN_BLOCK_SIZE_KEY = "dfs.namenode.fs-limits.min-block-size";
   public static final long    DFS_NAMENODE_MIN_BLOCK_SIZE_DEFAULT = 1024*1024;
   public static final String  DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY = "dfs.namenode.fs-limits.max-blocks-per-file";
-  public static final long    DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT = 1024*1024;
+  public static final long    DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT = 10*1000;
   public static final String  DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY = "dfs.namenode.fs-limits.max-xattrs-per-inode";
   public static final int     DFS_NAMENODE_MAX_XATTRS_PER_INODE_DEFAULT = 32;
   public static final String  DFS_NAMENODE_MAX_XATTR_SIZE_KEY = "dfs.namenode.fs-limits.max-xattr-size";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7226a71b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 671c98c..086f667 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -372,7 +372,7 @@
 
 <property>
     <name>dfs.namenode.fs-limits.max-blocks-per-file</name>
-    <value>1048576</value>
+    <value>10000</value>
     <description>Maximum number of blocks per file, enforced by the Namenode on
         write. This prevents the creation of extremely large files which can
         degrade performance.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7226a71b/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 f08b579..d7c8383 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
@@ -590,8 +590,15 @@ public class TestDirectoryScanner {
           100);
       DataNode dataNode = cluster.getDataNodes().get(0);
 
-      createFile(GenericTestUtils.getMethodName(),
-          BLOCK_LENGTH * blocks, false);
+      final int maxBlocksPerFile = (int) DFSConfigKeys
+          .DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT;
+      int numBlocksToCreate = blocks;
+      while (numBlocksToCreate > 0) {
+        final int toCreate = Math.min(maxBlocksPerFile, numBlocksToCreate);
+        createFile(GenericTestUtils.getMethodName() + numBlocksToCreate,
+            BLOCK_LENGTH * toCreate, false);
+        numBlocksToCreate -= toCreate;
+      }
 
       float ratio = 0.0f;
       int retries = maxRetries;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7226a71b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index 0ad6130..b9d25b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -571,7 +571,7 @@ public class TestNameNodeMetrics {
     Path file1_Path = new Path(TEST_ROOT_DIR_PATH, "ReadData.dat");
 
     //Perform create file operation
-    createFile(file1_Path, 1024 * 1024,(short)2);
+    createFile(file1_Path, 1024, (short) 2);
 
     // Perform read file operation on earlier created file
     readFile(fs, file1_Path);


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