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 tm...@apache.org on 2018/08/11 05:37:05 UTC

[01/50] [abbrv] hadoop git commit: HDFS-13447. Fix Typos - Node Not Chosen. Contributed by Beluga Behr. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HADOOP-15407 64dbd179e -> 7262485ae (forced update)


HDFS-13447. Fix Typos - Node Not Chosen. Contributed by Beluga Behr.


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

Branch: refs/heads/HADOOP-15407
Commit: 36c0d742d484f8bf01d7cb01c7b1c9e3627625dc
Parents: 5b898c1
Author: Márton Elek <el...@apache.org>
Authored: Wed Aug 8 17:27:57 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Wed Aug 8 17:31:55 2018 +0200

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/36c0d742/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index 6985f55..d00f961 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -72,11 +72,11 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       .withInitial(() -> new HashMap<NodeNotChosenReason, Integer>());
 
   private enum NodeNotChosenReason {
-    NOT_IN_SERVICE("the node isn't in service"),
+    NOT_IN_SERVICE("the node is not in service"),
     NODE_STALE("the node is stale"),
     NODE_TOO_BUSY("the node is too busy"),
     TOO_MANY_NODES_ON_RACK("the rack has too many chosen nodes"),
-    NOT_ENOUGH_STORAGE_SPACE("no enough storage space to place the block");
+    NOT_ENOUGH_STORAGE_SPACE("not enough storage space to place the block");
 
     private final String text;
 


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


[26/50] [abbrv] hadoop git commit: HDDS-245. Handle ContainerReports in the SCM. Contributed by Elek Marton.

Posted by tm...@apache.org.
HDDS-245. Handle ContainerReports in the SCM. Contributed by Elek Marton.


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

Branch: refs/heads/HADOOP-15407
Commit: f5dbbfe2e97a8c11e3df0f95ae4a493f11fdbc28
Parents: b2517dd
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Aug 9 16:55:13 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Aug 9 16:55:39 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hdds/server/events/EventQueue.java   |   7 +-
 .../scm/container/ContainerReportHandler.java   | 107 +++++-
 .../replication/ReplicationActivityStatus.java  |  86 +++++
 .../ReplicationActivityStatusMXBean.java        |  28 ++
 .../replication/ReplicationRequest.java         |  28 +-
 .../hadoop/hdds/scm/events/SCMEvents.java       |   9 +
 .../hdds/scm/node/states/Node2ContainerMap.java |  10 +-
 .../hdds/scm/node/states/ReportResult.java      |  18 +-
 .../scm/server/StorageContainerManager.java     |  27 +-
 .../container/TestContainerReportHandler.java   | 228 +++++++++++++
 .../scm/node/states/Node2ContainerMapTest.java  | 308 -----------------
 .../scm/node/states/TestNode2ContainerMap.java  | 328 +++++++++++++++++++
 12 files changed, 859 insertions(+), 325 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
index f93c54b..b2b0df2 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
@@ -147,7 +147,12 @@ public class EventQueue implements EventPublisher, AutoCloseable {
 
         for (EventHandler handler : executorAndHandlers.getValue()) {
           queuedCount.incrementAndGet();
-
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Delivering event {} to executor/handler {}: {}",
+                event.getName(),
+                executorAndHandlers.getKey().getName(),
+                payload);
+          }
           executorAndHandlers.getKey()
               .onMessage(handler, payload, this);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
index 486162e..b26eed2 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
@@ -18,30 +18,131 @@
 
 package org.apache.hadoop.hdds.scm.container;
 
+import java.io.IOException;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.replication
+    .ReplicationActivityStatus;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.states.Node2ContainerMap;
+import org.apache.hadoop.hdds.scm.node.states.ReportResult;
 import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
     .ContainerReportFromDatanode;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * Handles container reports from datanode.
  */
 public class ContainerReportHandler implements
     EventHandler<ContainerReportFromDatanode> {
 
-  private final Mapping containerMapping;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerReportHandler.class);
+
   private final Node2ContainerMap node2ContainerMap;
 
+  private final Mapping containerMapping;
+
+  private ContainerStateManager containerStateManager;
+
+  private ReplicationActivityStatus replicationStatus;
+
+
   public ContainerReportHandler(Mapping containerMapping,
-                                Node2ContainerMap node2ContainerMap) {
+      Node2ContainerMap node2ContainerMap,
+      ReplicationActivityStatus replicationActivityStatus) {
+    Preconditions.checkNotNull(containerMapping);
+    Preconditions.checkNotNull(node2ContainerMap);
+    Preconditions.checkNotNull(replicationActivityStatus);
     this.containerMapping = containerMapping;
     this.node2ContainerMap = node2ContainerMap;
+    this.containerStateManager = containerMapping.getStateManager();
+    this.replicationStatus = replicationActivityStatus;
   }
 
   @Override
   public void onMessage(ContainerReportFromDatanode containerReportFromDatanode,
                         EventPublisher publisher) {
-    // TODO: process container report.
+
+    DatanodeDetails datanodeOrigin =
+        containerReportFromDatanode.getDatanodeDetails();
+
+    ContainerReportsProto containerReport =
+        containerReportFromDatanode.getReport();
+    try {
+
+      //update state in container db and trigger close container events
+      containerMapping.processContainerReports(datanodeOrigin, containerReport);
+
+      Set<ContainerID> containerIds = containerReport.getReportsList().stream()
+          .map(containerProto -> containerProto.getContainerID())
+          .map(ContainerID::new)
+          .collect(Collectors.toSet());
+
+      ReportResult reportResult = node2ContainerMap
+          .processReport(datanodeOrigin.getUuid(), containerIds);
+
+      //we have the report, so we can update the states for the next iteration.
+      node2ContainerMap
+          .setContainersForDatanode(datanodeOrigin.getUuid(), containerIds);
+
+      for (ContainerID containerID : reportResult.getMissingContainers()) {
+        containerStateManager
+            .removeContainerReplica(containerID, datanodeOrigin);
+        emitReplicationRequestEvent(containerID, publisher);
+      }
+
+      for (ContainerID containerID : reportResult.getNewContainers()) {
+        containerStateManager.addContainerReplica(containerID, datanodeOrigin);
+
+        emitReplicationRequestEvent(containerID, publisher);
+      }
+
+    } catch (IOException e) {
+      //TODO: stop all the replication?
+      LOG.error("Error on processing container report from datanode {}",
+          datanodeOrigin, e);
+    }
+
+  }
+
+  private void emitReplicationRequestEvent(ContainerID containerID,
+      EventPublisher publisher) throws SCMException {
+    ContainerInfo container = containerStateManager.getContainer(containerID);
+
+    if (container == null) {
+      //warning unknown container
+      LOG.warn(
+          "Container is missing from containerStateManager. Can't request "
+              + "replication. {}",
+          containerID);
+    }
+    if (replicationStatus.isReplicationEnabled()) {
+
+      int existingReplicas =
+          containerStateManager.getContainerReplicas(containerID).size();
+
+      int expectedReplicas = container.getReplicationFactor().getNumber();
+
+      if (existingReplicas != expectedReplicas) {
+
+        publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER,
+            new ReplicationRequest(containerID.getId(), existingReplicas,
+                container.getReplicationFactor().getNumber()));
+      }
+    }
+
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatus.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatus.java
new file mode 100644
index 0000000..4a9888c
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatus.java
@@ -0,0 +1,86 @@
+/**
+ * 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.hdds.scm.container.replication;
+
+import javax.management.ObjectName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hdds.server.events.EventHandler;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+import org.apache.hadoop.metrics2.util.MBeans;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Event listener to track the current state of replication.
+ */
+public class ReplicationActivityStatus
+    implements EventHandler<Boolean>, ReplicationActivityStatusMXBean,
+    Closeable {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReplicationActivityStatus.class);
+
+  private AtomicBoolean replicationEnabled = new AtomicBoolean();
+
+  private ObjectName jmxObjectName;
+
+  public boolean isReplicationEnabled() {
+    return replicationEnabled.get();
+  }
+
+  @VisibleForTesting
+  public void setReplicationEnabled(boolean enabled) {
+    replicationEnabled.set(enabled);
+  }
+
+  @VisibleForTesting
+  public void enableReplication() {
+    replicationEnabled.set(true);
+  }
+
+  /**
+   * The replication status could be set by async events.
+   */
+  @Override
+  public void onMessage(Boolean enabled, EventPublisher publisher) {
+    replicationEnabled.set(enabled);
+  }
+
+  public void start() {
+    try {
+      this.jmxObjectName =
+          MBeans.register(
+              "StorageContainerManager", "ReplicationActivityStatus", this);
+    } catch (Exception ex) {
+      LOG.error("JMX bean for ReplicationActivityStatus can't be registered",
+          ex);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.jmxObjectName != null) {
+      MBeans.unregister(jmxObjectName);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatusMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatusMXBean.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatusMXBean.java
new file mode 100644
index 0000000..164bd24
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatusMXBean.java
@@ -0,0 +1,28 @@
+/**
+ * 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.hdds.scm.container.replication;
+
+/**
+ * JMX interface to monitor replication status.
+ */
+public interface ReplicationActivityStatusMXBean {
+
+  boolean isReplicationEnabled();
+
+  void setReplicationEnabled(boolean enabled);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationRequest.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationRequest.java
index ef7c546..d40cd9c 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationRequest.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationRequest.java
@@ -29,18 +29,24 @@ import org.apache.commons.lang3.builder.HashCodeBuilder;
 public class ReplicationRequest implements Comparable<ReplicationRequest>,
     Serializable {
   private final long containerId;
-  private final short replicationCount;
-  private final short expecReplicationCount;
+  private final int replicationCount;
+  private final int expecReplicationCount;
   private final long timestamp;
 
-  public ReplicationRequest(long containerId, short replicationCount,
-      long timestamp, short expecReplicationCount) {
+  public ReplicationRequest(long containerId, int replicationCount,
+      long timestamp, int expecReplicationCount) {
     this.containerId = containerId;
     this.replicationCount = replicationCount;
     this.timestamp = timestamp;
     this.expecReplicationCount = expecReplicationCount;
   }
 
+  public ReplicationRequest(long containerId, int replicationCount,
+      int expecReplicationCount) {
+    this(containerId, replicationCount, System.currentTimeMillis(),
+        expecReplicationCount);
+  }
+
   /**
    * Compares this object with the specified object for order.  Returns a
    * negative integer, zero, or a positive integer as this object is less
@@ -93,7 +99,7 @@ public class ReplicationRequest implements Comparable<ReplicationRequest>,
     return containerId;
   }
 
-  public short getReplicationCount() {
+  public int getReplicationCount() {
     return replicationCount;
   }
 
@@ -101,7 +107,17 @@ public class ReplicationRequest implements Comparable<ReplicationRequest>,
     return timestamp;
   }
 
-  public short getExpecReplicationCount() {
+  public int getExpecReplicationCount() {
     return expecReplicationCount;
   }
+
+  @Override
+  public String toString() {
+    return "ReplicationRequest{" +
+        "containerId=" + containerId +
+        ", replicationCount=" + replicationCount +
+        ", expecReplicationCount=" + expecReplicationCount +
+        ", timestamp=" + timestamp +
+        '}';
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
index d49dd4f..70b1e96 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
@@ -174,6 +174,15 @@ public final class SCMEvents {
       new TypedEvent<>(ReplicationCompleted.class);
 
   /**
+   * Signal for all the components (but especially for the replication
+   * manager and container report handler) that the replication could be
+   * started. Should be send only if (almost) all the container state are
+   * available from the datanodes.
+   */
+  public static final TypedEvent<Boolean> START_REPLICATION =
+      new TypedEvent<>(Boolean.class);
+
+  /**
    * Private Ctor. Never Constructed.
    */
   private SCMEvents() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
index 1960604..8ed6d59 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
@@ -68,7 +69,8 @@ public class Node2ContainerMap {
       throws SCMException {
     Preconditions.checkNotNull(containerIDs);
     Preconditions.checkNotNull(datanodeID);
-    if(dn2ContainerMap.putIfAbsent(datanodeID, containerIDs) != null) {
+    if (dn2ContainerMap.putIfAbsent(datanodeID, new HashSet<>(containerIDs))
+        != null) {
       throw new SCMException("Node already exists in the map",
                   DUPLICATE_DATANODE);
     }
@@ -82,11 +84,13 @@ public class Node2ContainerMap {
    * @throws SCMException - if we don't know about this datanode, for new DN
    *                      use insertNewDatanode.
    */
-  public void updateDatanodeMap(UUID datanodeID, Set<ContainerID> containers)
+  public void setContainersForDatanode(UUID datanodeID, Set<ContainerID> containers)
       throws SCMException {
     Preconditions.checkNotNull(datanodeID);
     Preconditions.checkNotNull(containers);
-    if(dn2ContainerMap.computeIfPresent(datanodeID, (k, v) -> v) == null){
+    if (dn2ContainerMap
+        .computeIfPresent(datanodeID, (k, v) -> new HashSet<>(containers))
+        == null) {
       throw new SCMException("No such datanode", NO_SUCH_DATANODE);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java
index cb06cb3..2697629 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java
@@ -21,10 +21,13 @@ package org.apache.hadoop.hdds.scm.node.states;
 
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 
+import java.util.Collections;
 import java.util.Set;
 
+import com.google.common.base.Preconditions;
+
 /**
- * A Container Report gets processsed by the Node2Container and returns the
+ * A Container Report gets processsed by the Node2Container and returns
  * Report Result class.
  */
 public class ReportResult {
@@ -36,6 +39,8 @@ public class ReportResult {
       Set<ContainerID> missingContainers,
       Set<ContainerID> newContainers) {
     this.status = status;
+    Preconditions.checkNotNull(missingContainers);
+    Preconditions.checkNotNull(newContainers);
     this.missingContainers = missingContainers;
     this.newContainers = newContainers;
   }
@@ -80,7 +85,16 @@ public class ReportResult {
     }
 
     ReportResult build() {
-      return new ReportResult(status, missingContainers, newContainers);
+
+      Set<ContainerID> nullSafeMissingContainers = this.missingContainers;
+      Set<ContainerID> nullSafeNewContainers = this.newContainers;
+      if (nullSafeNewContainers == null) {
+        nullSafeNewContainers = Collections.emptySet();
+      }
+      if (nullSafeMissingContainers == null) {
+        nullSafeMissingContainers = Collections.emptySet();
+      }
+      return new ReportResult(status, nullSafeMissingContainers, nullSafeNewContainers);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index 9cb1318..47a9100 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -40,6 +40,8 @@ import org.apache.hadoop.hdds.scm.container.ContainerActionsHandler;
 import org.apache.hadoop.hdds.scm.container.ContainerMapping;
 import org.apache.hadoop.hdds.scm.container.ContainerReportHandler;
 import org.apache.hadoop.hdds.scm.container.Mapping;
+import org.apache.hadoop.hdds.scm.container.replication
+    .ReplicationActivityStatus;
 import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms
@@ -164,9 +166,13 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
    * Key = DatanodeUuid, value = ContainerStat.
    */
   private Cache<String, ContainerStat> containerReportCache;
+
   private final ReplicationManager replicationManager;
+
   private final LeaseManager<Long> commandWatcherLeaseManager;
 
+  private final ReplicationActivityStatus replicationStatus;
+
   /**
    * Creates a new StorageContainerManager. Configuration will be updated
    * with information on the
@@ -199,19 +205,26 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
 
     Node2ContainerMap node2ContainerMap = new Node2ContainerMap();
 
+    replicationStatus = new ReplicationActivityStatus();
+
     CloseContainerEventHandler closeContainerHandler =
         new CloseContainerEventHandler(scmContainerManager);
     NodeReportHandler nodeReportHandler =
         new NodeReportHandler(scmNodeManager);
-    ContainerReportHandler containerReportHandler =
-        new ContainerReportHandler(scmContainerManager, node2ContainerMap);
+
     CommandStatusReportHandler cmdStatusReportHandler =
         new CommandStatusReportHandler();
+
     NewNodeHandler newNodeHandler = new NewNodeHandler(node2ContainerMap);
     StaleNodeHandler staleNodeHandler = new StaleNodeHandler(node2ContainerMap);
     DeadNodeHandler deadNodeHandler = new DeadNodeHandler(node2ContainerMap);
     ContainerActionsHandler actionsHandler = new ContainerActionsHandler();
 
+    ContainerReportHandler containerReportHandler =
+        new ContainerReportHandler(scmContainerManager, node2ContainerMap,
+            replicationStatus);
+
+
     eventQueue.addHandler(SCMEvents.DATANODE_COMMAND, scmNodeManager);
     eventQueue.addHandler(SCMEvents.NODE_REPORT, nodeReportHandler);
     eventQueue.addHandler(SCMEvents.CONTAINER_REPORT, containerReportHandler);
@@ -221,6 +234,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
     eventQueue.addHandler(SCMEvents.STALE_NODE, staleNodeHandler);
     eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
     eventQueue.addHandler(SCMEvents.CMD_STATUS_REPORT, cmdStatusReportHandler);
+    eventQueue.addHandler(SCMEvents.START_REPLICATION, replicationStatus);
 
     long watcherTimeout =
         conf.getTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT,
@@ -580,6 +594,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
         "server", getDatanodeProtocolServer().getDatanodeRpcAddress()));
     getDatanodeProtocolServer().start();
 
+    replicationStatus.start();
     httpServer.start();
     scmBlockManager.start();
     replicationManager.start();
@@ -592,6 +607,14 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
   public void stop() {
 
     try {
+      LOG.info("Stopping Replication Activity Status tracker.");
+      replicationStatus.close();
+    } catch (Exception ex) {
+      LOG.error("Replication Activity Status tracker stop failed.", ex);
+    }
+
+
+    try {
       LOG.info("Stopping Replication Manager Service.");
       replicationManager.stop();
     } catch (Exception ex) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
new file mode 100644
index 0000000..363db99
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
@@ -0,0 +1,228 @@
+/**
+ * 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.hdds.scm.container;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.scm.TestUtils;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo
+    .Builder;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.replication
+    .ReplicationActivityStatus;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
+import org.apache.hadoop.hdds.scm.node.states.Node2ContainerMap;
+import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
+import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
+    .ContainerReportFromDatanode;
+import org.apache.hadoop.hdds.server.events.Event;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import static org.mockito.Matchers.anyLong;
+import org.mockito.Mockito;
+import static org.mockito.Mockito.when;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test the behaviour of the ContainerReportHandler.
+ */
+public class TestContainerReportHandler implements EventPublisher {
+
+  private List<Object> publishedEvents = new ArrayList<>();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestContainerReportHandler.class);
+
+  @Before
+  public void resetEventCollector() {
+    publishedEvents.clear();
+  }
+
+  @Test
+  public void test() throws IOException {
+
+    //given
+
+    OzoneConfiguration conf = new OzoneConfiguration();
+    Node2ContainerMap node2ContainerMap = new Node2ContainerMap();
+    Mapping mapping = Mockito.mock(Mapping.class);
+
+    when(mapping.getContainer(anyLong()))
+        .thenAnswer(
+            (Answer<ContainerInfo>) invocation ->
+                new Builder()
+                    .setReplicationFactor(ReplicationFactor.THREE)
+                    .setContainerID((Long) invocation.getArguments()[0])
+                    .build()
+        );
+
+    ContainerStateManager containerStateManager =
+        new ContainerStateManager(conf, mapping);
+
+    when(mapping.getStateManager()).thenReturn(containerStateManager);
+
+    ReplicationActivityStatus replicationActivityStatus =
+        new ReplicationActivityStatus();
+
+    ContainerReportHandler reportHandler =
+        new ContainerReportHandler(mapping, node2ContainerMap,
+            replicationActivityStatus);
+
+    DatanodeDetails dn1 = TestUtils.randomDatanodeDetails();
+    DatanodeDetails dn2 = TestUtils.randomDatanodeDetails();
+    DatanodeDetails dn3 = TestUtils.randomDatanodeDetails();
+    DatanodeDetails dn4 = TestUtils.randomDatanodeDetails();
+    node2ContainerMap.insertNewDatanode(dn1.getUuid(), new HashSet<>());
+    node2ContainerMap.insertNewDatanode(dn2.getUuid(), new HashSet<>());
+    node2ContainerMap.insertNewDatanode(dn3.getUuid(), new HashSet<>());
+    node2ContainerMap.insertNewDatanode(dn4.getUuid(), new HashSet<>());
+    PipelineSelector pipelineSelector = Mockito.mock(PipelineSelector.class);
+
+    Pipeline pipeline = new Pipeline("leader", LifeCycleState.CLOSED,
+        ReplicationType.STAND_ALONE, ReplicationFactor.THREE, "pipeline1");
+
+    when(pipelineSelector.getReplicationPipeline(ReplicationType.STAND_ALONE,
+        ReplicationFactor.THREE)).thenReturn(pipeline);
+
+    long c1 = containerStateManager
+        .allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
+            ReplicationFactor.THREE, "root").getContainerInfo()
+        .getContainerID();
+
+    long c2 = containerStateManager
+        .allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
+            ReplicationFactor.THREE, "root").getContainerInfo()
+        .getContainerID();
+
+    //when
+
+    //initial reports before replication is enabled. 2 containers w 3 replicas.
+    reportHandler.onMessage(
+        new ContainerReportFromDatanode(dn1,
+            createContainerReport(new long[] {c1, c2})), this);
+
+    reportHandler.onMessage(
+        new ContainerReportFromDatanode(dn2,
+            createContainerReport(new long[] {c1, c2})), this);
+
+    reportHandler.onMessage(
+        new ContainerReportFromDatanode(dn3,
+            createContainerReport(new long[] {c1, c2})), this);
+
+    reportHandler.onMessage(
+        new ContainerReportFromDatanode(dn4,
+            createContainerReport(new long[] {})), this);
+
+    Assert.assertEquals(0, publishedEvents.size());
+
+    replicationActivityStatus.enableReplication();
+
+    //no problem here
+    reportHandler.onMessage(
+        new ContainerReportFromDatanode(dn1,
+            createContainerReport(new long[] {c1, c2})), this);
+
+    Assert.assertEquals(0, publishedEvents.size());
+
+    //container is missing from d2
+    reportHandler.onMessage(
+        new ContainerReportFromDatanode(dn2,
+            createContainerReport(new long[] {c1})), this);
+
+    Assert.assertEquals(1, publishedEvents.size());
+    ReplicationRequest replicationRequest =
+        (ReplicationRequest) publishedEvents.get(0);
+
+    Assert.assertEquals(c2, replicationRequest.getContainerId());
+    Assert.assertEquals(3, replicationRequest.getExpecReplicationCount());
+    Assert.assertEquals(2, replicationRequest.getReplicationCount());
+
+    //container was replicated to dn4
+    reportHandler.onMessage(
+        new ContainerReportFromDatanode(dn4,
+            createContainerReport(new long[] {c2})), this);
+
+    //no more event, everything is perfect
+    Assert.assertEquals(1, publishedEvents.size());
+
+    //c2 was found at dn2 (it was missing before, magic)
+    reportHandler.onMessage(
+        new ContainerReportFromDatanode(dn2,
+            createContainerReport(new long[] {c1, c2})), this);
+
+    //c2 is over replicated (dn1,dn2,dn3,dn4)
+    Assert.assertEquals(2, publishedEvents.size());
+
+    replicationRequest =
+        (ReplicationRequest) publishedEvents.get(1);
+
+    Assert.assertEquals(c2, replicationRequest.getContainerId());
+    Assert.assertEquals(3, replicationRequest.getExpecReplicationCount());
+    Assert.assertEquals(4, replicationRequest.getReplicationCount());
+
+  }
+
+  private ContainerReportsProto createContainerReport(long[] containerIds) {
+
+    ContainerReportsProto.Builder crBuilder =
+        ContainerReportsProto.newBuilder();
+
+    for (long containerId : containerIds) {
+      org.apache.hadoop.hdds.protocol.proto
+          .StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder
+          ciBuilder = org.apache.hadoop.hdds.protocol.proto
+          .StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
+      ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea46d2")
+          .setSize(5368709120L)
+          .setUsed(2000000000L)
+          .setKeyCount(100000000L)
+          .setReadCount(100000000L)
+          .setWriteCount(100000000L)
+          .setReadBytes(2000000000L)
+          .setWriteBytes(2000000000L)
+          .setContainerID(containerId)
+          .setDeleteTransactionId(0);
+
+      crBuilder.addReports(ciBuilder.build());
+    }
+
+    return crBuilder.build();
+  }
+
+  @Override
+  public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void fireEvent(
+      EVENT_TYPE event, PAYLOAD payload) {
+    LOG.info("Event is published: {}", payload);
+    publishedEvents.add(payload);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMapTest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMapTest.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMapTest.java
deleted file mode 100644
index 79f1b40..0000000
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMapTest.java
+++ /dev/null
@@ -1,308 +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.hdds.scm.node.states;
-
-import org.apache.hadoop.hdds.scm.container.ContainerID;
-import org.apache.hadoop.hdds.scm.exceptions.SCMException;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * Test classes for Node2ContainerMap.
- */
-public class Node2ContainerMapTest {
-  private final static int DATANODE_COUNT = 300;
-  private final static int CONTAINER_COUNT = 1000;
-  private final Map<UUID, TreeSet<ContainerID>> testData = new
-      ConcurrentHashMap<>();
-
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  private void generateData() {
-    for (int dnIndex = 1; dnIndex <= DATANODE_COUNT; dnIndex++) {
-      TreeSet<ContainerID> currentSet = new TreeSet<>();
-      for (int cnIndex = 1; cnIndex <= CONTAINER_COUNT; cnIndex++) {
-        long currentCnIndex = (dnIndex * CONTAINER_COUNT) + cnIndex;
-        currentSet.add(new ContainerID(currentCnIndex));
-      }
-      testData.put(UUID.randomUUID(), currentSet);
-    }
-  }
-
-  private UUID getFirstKey() {
-    return testData.keySet().iterator().next();
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    generateData();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-  }
-
-  @Test
-  public void testIsKnownDatanode() throws SCMException {
-    Node2ContainerMap map = new Node2ContainerMap();
-    UUID knownNode = getFirstKey();
-    UUID unknownNode = UUID.randomUUID();
-    Set<ContainerID> containerIDs = testData.get(knownNode);
-    map.insertNewDatanode(knownNode, containerIDs);
-    Assert.assertTrue("Not able to detect a known node",
-        map.isKnownDatanode(knownNode));
-    Assert.assertFalse("Unknown node detected",
-        map.isKnownDatanode(unknownNode));
-  }
-
-  @Test
-  public void testInsertNewDatanode() throws SCMException {
-    Node2ContainerMap map = new Node2ContainerMap();
-    UUID knownNode = getFirstKey();
-    Set<ContainerID> containerIDs = testData.get(knownNode);
-    map.insertNewDatanode(knownNode, containerIDs);
-    Set<ContainerID> readSet = map.getContainers(knownNode);
-
-    // Assert that all elements are present in the set that we read back from
-    // node map.
-    Set newSet = new TreeSet((readSet));
-    Assert.assertTrue(newSet.removeAll(containerIDs));
-    Assert.assertTrue(newSet.size() == 0);
-
-    thrown.expect(SCMException.class);
-    thrown.expectMessage("already exists");
-    map.insertNewDatanode(knownNode, containerIDs);
-
-    map.removeDatanode(knownNode);
-    map.insertNewDatanode(knownNode, containerIDs);
-
-  }
-
-  @Test
-  public void testProcessReportCheckOneNode() throws SCMException {
-    UUID key = getFirstKey();
-    Set<ContainerID> values = testData.get(key);
-    Node2ContainerMap map = new Node2ContainerMap();
-    map.insertNewDatanode(key, values);
-    Assert.assertTrue(map.isKnownDatanode(key));
-    ReportResult result = map.processReport(key, values);
-    Assert.assertEquals(result.getStatus(),
-        Node2ContainerMap.ReportStatus.ALL_IS_WELL);
-  }
-
-  @Test
-  public void testProcessReportInsertAll() throws SCMException {
-    Node2ContainerMap map = new Node2ContainerMap();
-
-    for (Map.Entry<UUID, TreeSet<ContainerID>> keyEntry : testData.entrySet()) {
-      map.insertNewDatanode(keyEntry.getKey(), keyEntry.getValue());
-    }
-    // Assert all Keys are known datanodes.
-    for (UUID key : testData.keySet()) {
-      Assert.assertTrue(map.isKnownDatanode(key));
-    }
-  }
-
-  /*
-  For ProcessReport we have to test the following scenarios.
-
-  1. New Datanode - A new datanode appears and we have to add that to the
-  SCM's Node2Container Map.
-
-  2.  New Container - A Datanode exists, but a new container is added to that
-   DN. We need to detect that and return a list of added containers.
-
-  3. Missing Container - A Datanode exists, but one of the expected container
-   on that datanode is missing. We need to detect that.
-
-   4. We get a container report that has both the missing and new containers.
-    We need to return separate lists for these.
-   */
-
-  /**
-   * Assert that we are able to detect the addition of a new datanode.
-   *
-   * @throws SCMException
-   */
-  @Test
-  public void testProcessReportDetectNewDataNode() throws SCMException {
-    Node2ContainerMap map = new Node2ContainerMap();
-    // If we attempt to process a node that is not present in the map,
-    // we get a result back that says, NEW_NODE_FOUND.
-    UUID key = getFirstKey();
-    TreeSet<ContainerID> values = testData.get(key);
-    ReportResult result = map.processReport(key, values);
-    Assert.assertEquals(Node2ContainerMap.ReportStatus.NEW_DATANODE_FOUND,
-        result.getStatus());
-    Assert.assertEquals(result.getNewContainers().size(), values.size());
-  }
-
-  /**
-   * This test asserts that processReport is able to detect new containers
-   * when it is added to a datanode. For that we populate the DN with a list
-   * of containerIDs and then add few more containers and make sure that we
-   * are able to detect them.
-   *
-   * @throws SCMException
-   */
-  @Test
-  public void testProcessReportDetectNewContainers() throws SCMException {
-    Node2ContainerMap map = new Node2ContainerMap();
-    UUID key = getFirstKey();
-    TreeSet<ContainerID> values = testData.get(key);
-    map.insertNewDatanode(key, values);
-
-    final int newCount = 100;
-    // This is not a mistake, the treeset seems to be reverse sorted.
-    ContainerID last = values.pollFirst();
-    TreeSet<ContainerID> addedContainers = new TreeSet<>();
-    for (int x = 1; x <= newCount; x++) {
-      long cTemp = last.getId() + x;
-      addedContainers.add(new ContainerID(cTemp));
-    }
-
-    // This set is the super set of existing containers and new containers.
-    TreeSet<ContainerID> newContainersSet = new TreeSet<>(values);
-    newContainersSet.addAll(addedContainers);
-
-    ReportResult result = map.processReport(key, newContainersSet);
-
-    //Assert that expected size of missing container is same as addedContainers
-    Assert.assertEquals(Node2ContainerMap.ReportStatus.NEW_CONTAINERS_FOUND,
-        result.getStatus());
-
-    Assert.assertEquals(addedContainers.size(),
-        result.getNewContainers().size());
-
-    // Assert that the Container IDs are the same as we added new.
-    Assert.assertTrue("All objects are not removed.",
-        result.getNewContainers().removeAll(addedContainers));
-  }
-
-  /**
-   * This test asserts that processReport is able to detect missing containers
-   * if they are misssing from a list.
-   *
-   * @throws SCMException
-   */
-  @Test
-  public void testProcessReportDetectMissingContainers() throws SCMException {
-    Node2ContainerMap map = new Node2ContainerMap();
-    UUID key = getFirstKey();
-    TreeSet<ContainerID> values = testData.get(key);
-    map.insertNewDatanode(key, values);
-
-    final int removeCount = 100;
-    Random r = new Random();
-
-    ContainerID first = values.pollLast();
-    TreeSet<ContainerID> removedContainers = new TreeSet<>();
-
-    // Pick a random container to remove it is ok to collide no issues.
-    for (int x = 0; x < removeCount; x++) {
-      int startBase = (int) first.getId();
-      long cTemp = r.nextInt(values.size());
-      removedContainers.add(new ContainerID(cTemp + startBase));
-    }
-
-    // This set is a new set with some containers removed.
-    TreeSet<ContainerID> newContainersSet = new TreeSet<>(values);
-    newContainersSet.removeAll(removedContainers);
-
-    ReportResult result = map.processReport(key, newContainersSet);
-
-
-    //Assert that expected size of missing container is same as addedContainers
-    Assert.assertEquals(Node2ContainerMap.ReportStatus.MISSING_CONTAINERS,
-        result.getStatus());
-    Assert.assertEquals(removedContainers.size(),
-        result.getMissingContainers().size());
-
-    // Assert that the Container IDs are the same as we added new.
-    Assert.assertTrue("All missing containers not found.",
-        result.getMissingContainers().removeAll(removedContainers));
-  }
-
-  @Test
-  public void testProcessReportDetectNewAndMissingContainers() throws
-      SCMException {
-    Node2ContainerMap map = new Node2ContainerMap();
-    UUID key = getFirstKey();
-    TreeSet<ContainerID> values = testData.get(key);
-    map.insertNewDatanode(key, values);
-
-    Set<ContainerID> insertedSet = new TreeSet<>();
-    // Insert nodes from 1..30
-    for (int x = 1; x <= 30; x++) {
-      insertedSet.add(new ContainerID(x));
-    }
-
-
-    final int removeCount = 100;
-    Random r = new Random();
-
-    ContainerID first = values.pollLast();
-    TreeSet<ContainerID> removedContainers = new TreeSet<>();
-
-    // Pick a random container to remove it is ok to collide no issues.
-    for (int x = 0; x < removeCount; x++) {
-      int startBase = (int) first.getId();
-      long cTemp = r.nextInt(values.size());
-      removedContainers.add(new ContainerID(cTemp + startBase));
-    }
-
-    Set<ContainerID> newSet = new TreeSet<>(values);
-    newSet.addAll(insertedSet);
-    newSet.removeAll(removedContainers);
-
-    ReportResult result = map.processReport(key, newSet);
-
-
-    Assert.assertEquals(
-        Node2ContainerMap.ReportStatus.MISSING_AND_NEW_CONTAINERS_FOUND,
-        result.getStatus());
-    Assert.assertEquals(removedContainers.size(),
-        result.getMissingContainers().size());
-
-
-    // Assert that the Container IDs are the same as we added new.
-    Assert.assertTrue("All missing containers not found.",
-        result.getMissingContainers().removeAll(removedContainers));
-
-    Assert.assertEquals(insertedSet.size(),
-        result.getNewContainers().size());
-
-    // Assert that the Container IDs are the same as we added new.
-    Assert.assertTrue("All inserted containers are not found.",
-        result.getNewContainers().removeAll(insertedSet));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5dbbfe2/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/TestNode2ContainerMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/TestNode2ContainerMap.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/TestNode2ContainerMap.java
new file mode 100644
index 0000000..633653b
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/TestNode2ContainerMap.java
@@ -0,0 +1,328 @@
+/*
+ * 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.hdds.scm.node.states;
+
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Test classes for Node2ContainerMap.
+ */
+public class TestNode2ContainerMap {
+  private final static int DATANODE_COUNT = 300;
+  private final static int CONTAINER_COUNT = 1000;
+  private final Map<UUID, TreeSet<ContainerID>> testData = new
+      ConcurrentHashMap<>();
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private void generateData() {
+    for (int dnIndex = 1; dnIndex <= DATANODE_COUNT; dnIndex++) {
+      TreeSet<ContainerID> currentSet = new TreeSet<>();
+      for (int cnIndex = 1; cnIndex <= CONTAINER_COUNT; cnIndex++) {
+        long currentCnIndex = (dnIndex * CONTAINER_COUNT) + cnIndex;
+        currentSet.add(new ContainerID(currentCnIndex));
+      }
+      testData.put(UUID.randomUUID(), currentSet);
+    }
+  }
+
+  private UUID getFirstKey() {
+    return testData.keySet().iterator().next();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    generateData();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public void testIsKnownDatanode() throws SCMException {
+    Node2ContainerMap map = new Node2ContainerMap();
+    UUID knownNode = getFirstKey();
+    UUID unknownNode = UUID.randomUUID();
+    Set<ContainerID> containerIDs = testData.get(knownNode);
+    map.insertNewDatanode(knownNode, containerIDs);
+    Assert.assertTrue("Not able to detect a known node",
+        map.isKnownDatanode(knownNode));
+    Assert.assertFalse("Unknown node detected",
+        map.isKnownDatanode(unknownNode));
+  }
+
+  @Test
+  public void testInsertNewDatanode() throws SCMException {
+    Node2ContainerMap map = new Node2ContainerMap();
+    UUID knownNode = getFirstKey();
+    Set<ContainerID> containerIDs = testData.get(knownNode);
+    map.insertNewDatanode(knownNode, containerIDs);
+    Set<ContainerID> readSet = map.getContainers(knownNode);
+
+    // Assert that all elements are present in the set that we read back from
+    // node map.
+    Set newSet = new TreeSet((readSet));
+    Assert.assertTrue(newSet.removeAll(containerIDs));
+    Assert.assertTrue(newSet.size() == 0);
+
+    thrown.expect(SCMException.class);
+    thrown.expectMessage("already exists");
+    map.insertNewDatanode(knownNode, containerIDs);
+
+    map.removeDatanode(knownNode);
+    map.insertNewDatanode(knownNode, containerIDs);
+
+  }
+
+  @Test
+  public void testProcessReportCheckOneNode() throws SCMException {
+    UUID key = getFirstKey();
+    Set<ContainerID> values = testData.get(key);
+    Node2ContainerMap map = new Node2ContainerMap();
+    map.insertNewDatanode(key, values);
+    Assert.assertTrue(map.isKnownDatanode(key));
+    ReportResult result = map.processReport(key, values);
+    Assert.assertEquals(result.getStatus(),
+        Node2ContainerMap.ReportStatus.ALL_IS_WELL);
+  }
+
+  @Test
+  public void testUpdateDatanodeMap() throws SCMException {
+    UUID datanodeId = getFirstKey();
+    Set<ContainerID> values = testData.get(datanodeId);
+    Node2ContainerMap map = new Node2ContainerMap();
+    map.insertNewDatanode(datanodeId, values);
+    Assert.assertTrue(map.isKnownDatanode(datanodeId));
+    Assert.assertEquals(CONTAINER_COUNT, map.getContainers(datanodeId).size());
+
+    //remove one container
+    values.remove(values.iterator().next());
+    Assert.assertEquals(CONTAINER_COUNT - 1, values.size());
+    Assert.assertEquals(CONTAINER_COUNT, map.getContainers(datanodeId).size());
+
+    map.setContainersForDatanode(datanodeId, values);
+
+    Assert.assertEquals(values.size(), map.getContainers(datanodeId).size());
+    Assert.assertEquals(values, map.getContainers(datanodeId));
+  }
+
+  @Test
+  public void testProcessReportInsertAll() throws SCMException {
+    Node2ContainerMap map = new Node2ContainerMap();
+
+    for (Map.Entry<UUID, TreeSet<ContainerID>> keyEntry : testData.entrySet()) {
+      map.insertNewDatanode(keyEntry.getKey(), keyEntry.getValue());
+    }
+    // Assert all Keys are known datanodes.
+    for (UUID key : testData.keySet()) {
+      Assert.assertTrue(map.isKnownDatanode(key));
+    }
+  }
+
+  /*
+  For ProcessReport we have to test the following scenarios.
+
+  1. New Datanode - A new datanode appears and we have to add that to the
+  SCM's Node2Container Map.
+
+  2.  New Container - A Datanode exists, but a new container is added to that
+   DN. We need to detect that and return a list of added containers.
+
+  3. Missing Container - A Datanode exists, but one of the expected container
+   on that datanode is missing. We need to detect that.
+
+   4. We get a container report that has both the missing and new containers.
+    We need to return separate lists for these.
+   */
+
+  /**
+   * Assert that we are able to detect the addition of a new datanode.
+   *
+   * @throws SCMException
+   */
+  @Test
+  public void testProcessReportDetectNewDataNode() throws SCMException {
+    Node2ContainerMap map = new Node2ContainerMap();
+    // If we attempt to process a node that is not present in the map,
+    // we get a result back that says, NEW_NODE_FOUND.
+    UUID key = getFirstKey();
+    TreeSet<ContainerID> values = testData.get(key);
+    ReportResult result = map.processReport(key, values);
+    Assert.assertEquals(Node2ContainerMap.ReportStatus.NEW_DATANODE_FOUND,
+        result.getStatus());
+    Assert.assertEquals(result.getNewContainers().size(), values.size());
+  }
+
+  /**
+   * This test asserts that processReport is able to detect new containers
+   * when it is added to a datanode. For that we populate the DN with a list
+   * of containerIDs and then add few more containers and make sure that we
+   * are able to detect them.
+   *
+   * @throws SCMException
+   */
+  @Test
+  public void testProcessReportDetectNewContainers() throws SCMException {
+    Node2ContainerMap map = new Node2ContainerMap();
+    UUID key = getFirstKey();
+    TreeSet<ContainerID> values = testData.get(key);
+    map.insertNewDatanode(key, values);
+
+    final int newCount = 100;
+    // This is not a mistake, the treeset seems to be reverse sorted.
+    ContainerID last = values.first();
+    TreeSet<ContainerID> addedContainers = new TreeSet<>();
+    for (int x = 1; x <= newCount; x++) {
+      long cTemp = last.getId() + x;
+      addedContainers.add(new ContainerID(cTemp));
+    }
+
+    // This set is the super set of existing containers and new containers.
+    TreeSet<ContainerID> newContainersSet = new TreeSet<>(values);
+    newContainersSet.addAll(addedContainers);
+
+    ReportResult result = map.processReport(key, newContainersSet);
+
+    //Assert that expected size of missing container is same as addedContainers
+    Assert.assertEquals(Node2ContainerMap.ReportStatus.NEW_CONTAINERS_FOUND,
+        result.getStatus());
+
+    Assert.assertEquals(addedContainers.size(),
+        result.getNewContainers().size());
+
+    // Assert that the Container IDs are the same as we added new.
+    Assert.assertTrue("All objects are not removed.",
+        result.getNewContainers().removeAll(addedContainers));
+  }
+
+  /**
+   * This test asserts that processReport is able to detect missing containers
+   * if they are misssing from a list.
+   *
+   * @throws SCMException
+   */
+  @Test
+  public void testProcessReportDetectMissingContainers() throws SCMException {
+    Node2ContainerMap map = new Node2ContainerMap();
+    UUID key = getFirstKey();
+    TreeSet<ContainerID> values = testData.get(key);
+    map.insertNewDatanode(key, values);
+
+    final int removeCount = 100;
+    Random r = new Random();
+
+    ContainerID first = values.last();
+    TreeSet<ContainerID> removedContainers = new TreeSet<>();
+
+    // Pick a random container to remove it is ok to collide no issues.
+    for (int x = 0; x < removeCount; x++) {
+      int startBase = (int) first.getId();
+      long cTemp = r.nextInt(values.size());
+      removedContainers.add(new ContainerID(cTemp + startBase));
+    }
+
+    // This set is a new set with some containers removed.
+    TreeSet<ContainerID> newContainersSet = new TreeSet<>(values);
+    newContainersSet.removeAll(removedContainers);
+
+    ReportResult result = map.processReport(key, newContainersSet);
+
+
+    //Assert that expected size of missing container is same as addedContainers
+    Assert.assertEquals(Node2ContainerMap.ReportStatus.MISSING_CONTAINERS,
+        result.getStatus());
+    Assert.assertEquals(removedContainers.size(),
+        result.getMissingContainers().size());
+
+    // Assert that the Container IDs are the same as we added new.
+    Assert.assertTrue("All missing containers not found.",
+        result.getMissingContainers().removeAll(removedContainers));
+  }
+
+  @Test
+  public void testProcessReportDetectNewAndMissingContainers() throws
+      SCMException {
+    Node2ContainerMap map = new Node2ContainerMap();
+    UUID key = getFirstKey();
+    TreeSet<ContainerID> values = testData.get(key);
+    map.insertNewDatanode(key, values);
+
+    Set<ContainerID> insertedSet = new TreeSet<>();
+    // Insert nodes from 1..30
+    for (int x = 1; x <= 30; x++) {
+      insertedSet.add(new ContainerID(x));
+    }
+
+
+    final int removeCount = 100;
+    Random r = new Random();
+
+    ContainerID first = values.last();
+    TreeSet<ContainerID> removedContainers = new TreeSet<>();
+
+    // Pick a random container to remove it is ok to collide no issues.
+    for (int x = 0; x < removeCount; x++) {
+      int startBase = (int) first.getId();
+      long cTemp = r.nextInt(values.size());
+      removedContainers.add(new ContainerID(cTemp + startBase));
+    }
+
+    Set<ContainerID> newSet = new TreeSet<>(values);
+    newSet.addAll(insertedSet);
+    newSet.removeAll(removedContainers);
+
+    ReportResult result = map.processReport(key, newSet);
+
+
+    Assert.assertEquals(
+        Node2ContainerMap.ReportStatus.MISSING_AND_NEW_CONTAINERS_FOUND,
+        result.getStatus());
+    Assert.assertEquals(removedContainers.size(),
+        result.getMissingContainers().size());
+
+
+    // Assert that the Container IDs are the same as we added new.
+    Assert.assertTrue("All missing containers not found.",
+        result.getMissingContainers().removeAll(removedContainers));
+
+    Assert.assertEquals(insertedSet.size(),
+        result.getNewContainers().size());
+
+    // Assert that the Container IDs are the same as we added new.
+    Assert.assertTrue("All inserted containers are not found.",
+        result.getNewContainers().removeAll(insertedSet));
+  }
+}
\ 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


[46/50] [abbrv] hadoop git commit: HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit.

Posted by tm...@apache.org.
HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit.

Contributed by Shane Mainali, Thomas Marquardt, Zichen Sun, Georgi Chalakov, Esfandiar Manii, Amit Singh, Dana Kaban, Da Zhou, Junhua Gu, Saher Ahwal, Saurabh Pant, James Baker, Shaoyu Zhang, Lawrence Chen, Kevin Chen and Steve Loughran


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

Branch: refs/heads/HADOOP-15407
Commit: fdf5f4c3243e8c8d76f443cb15aeb19a4edb4bb6
Parents: a2a8c48
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jun 15 18:14:13 2018 +0100
Committer: Thomas Marquardt <tm...@microsoft.com>
Committed: Sat Aug 11 03:42:27 2018 +0000

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 .../src/main/resources/core-default.xml         |  10 +
 .../conf/TestCommonConfigurationFields.java     |   3 +
 hadoop-project/pom.xml                          |  11 +
 hadoop-tools/hadoop-azure/pom.xml               |  61 +-
 .../src/config/checkstyle-suppressions.xml      |  47 ++
 .../org/apache/hadoop/fs/azurebfs/Abfs.java     |  48 ++
 .../org/apache/hadoop/fs/azurebfs/Abfss.java    |  48 ++
 .../hadoop/fs/azurebfs/AzureBlobFileSystem.java | 612 ++++++++++++++++
 .../fs/azurebfs/SecureAzureBlobFileSystem.java  |  41 ++
 .../azurebfs/constants/AbfsHttpConstants.java   |  76 ++
 .../azurebfs/constants/ConfigurationKeys.java   |  57 ++
 .../constants/FileSystemConfigurations.java     |  59 ++
 .../constants/FileSystemUriSchemes.java         |  42 ++
 .../constants/HttpHeaderConfigurations.java     |  57 ++
 .../fs/azurebfs/constants/HttpQueryParams.java  |  40 ++
 .../fs/azurebfs/constants/package-info.java     |  22 +
 .../ConfigurationValidationAnnotations.java     | 104 +++
 .../contracts/annotations/package-info.java     |  22 +
 .../diagnostics/ConfigurationValidator.java     |  37 +
 .../contracts/diagnostics/package-info.java     |  22 +
 .../exceptions/AbfsRestOperationException.java  |  84 +++
 .../AzureBlobFileSystemException.java           |  56 ++
 .../ConfigurationPropertyNotFoundException.java |  32 +
 .../FileSystemOperationUnhandledException.java  |  33 +
 .../InvalidAbfsRestOperationException.java      |  40 ++
 .../InvalidConfigurationValueException.java     |  37 +
 .../InvalidFileSystemPropertyException.java     |  33 +
 .../InvalidUriAuthorityException.java           |  33 +
 .../exceptions/InvalidUriException.java         |  33 +
 .../exceptions/ServiceResolutionException.java  |  36 +
 .../contracts/exceptions/TimeoutException.java  |  33 +
 .../contracts/exceptions/package-info.java      |  22 +
 .../fs/azurebfs/contracts/package-info.java     |  22 +
 .../services/AbfsHttpClientFactory.java         |  39 ++
 .../contracts/services/AbfsHttpService.java     | 162 +++++
 .../contracts/services/AbfsServiceProvider.java |  40 ++
 .../services/AzureServiceErrorCode.java         | 112 +++
 .../services/ConfigurationService.java          | 143 ++++
 .../contracts/services/InjectableService.java   |  30 +
 .../services/ListResultEntrySchema.java         | 160 +++++
 .../contracts/services/ListResultSchema.java    |  58 ++
 .../contracts/services/ReadBufferStatus.java    |  29 +
 .../contracts/services/TracingService.java      |  66 ++
 .../contracts/services/package-info.java        |  22 +
 ...Base64StringConfigurationBasicValidator.java |  50 ++
 .../BooleanConfigurationBasicValidator.java     |  50 ++
 .../ConfigurationBasicValidator.java            |  67 ++
 .../IntegerConfigurationBasicValidator.java     |  68 ++
 .../LongConfigurationBasicValidator.java        |  65 ++
 .../StringConfigurationBasicValidator.java      |  45 ++
 .../fs/azurebfs/diagnostics/package-info.java   |  22 +
 .../org/apache/hadoop/fs/azurebfs/package.html  |  31 +
 .../hadoop/fs/azurebfs/services/AbfsClient.java | 402 +++++++++++
 .../services/AbfsHttpClientFactoryImpl.java     | 116 ++++
 .../fs/azurebfs/services/AbfsHttpHeader.java    |  40 ++
 .../fs/azurebfs/services/AbfsHttpOperation.java | 430 ++++++++++++
 .../azurebfs/services/AbfsHttpServiceImpl.java  | 693 +++++++++++++++++++
 .../fs/azurebfs/services/AbfsInputStream.java   | 382 ++++++++++
 .../fs/azurebfs/services/AbfsOutputStream.java  | 335 +++++++++
 .../fs/azurebfs/services/AbfsRestOperation.java | 178 +++++
 .../services/AbfsServiceInjectorImpl.java       |  81 +++
 .../services/AbfsServiceProviderImpl.java       |  96 +++
 .../azurebfs/services/AbfsUriQueryBuilder.java  |  58 ++
 .../services/ConfigurationServiceImpl.java      | 317 +++++++++
 .../services/ExponentialRetryPolicy.java        | 141 ++++
 .../azurebfs/services/LoggerSpanReceiver.java   |  74 ++
 .../hadoop/fs/azurebfs/services/ReadBuffer.java | 139 ++++
 .../fs/azurebfs/services/ReadBufferManager.java | 391 +++++++++++
 .../fs/azurebfs/services/ReadBufferWorker.java  |  72 ++
 .../azurebfs/services/SharedKeyCredentials.java | 507 ++++++++++++++
 .../azurebfs/services/TracingServiceImpl.java   | 134 ++++
 .../fs/azurebfs/services/package-info.java      |  22 +
 .../hadoop/fs/azurebfs/utils/UriUtils.java      |  77 +++
 .../hadoop/fs/azurebfs/utils/package-info.java  |  22 +
 .../fs/azurebfs/DependencyInjectedTest.java     | 225 ++++++
 .../ITestAzureBlobFileSystemAppend.java         |  78 +++
 .../ITestAzureBlobFileSystemBackCompat.java     |  83 +++
 .../azurebfs/ITestAzureBlobFileSystemCopy.java  | 100 +++
 .../ITestAzureBlobFileSystemCreate.java         | 110 +++
 .../ITestAzureBlobFileSystemDelete.java         | 119 ++++
 .../azurebfs/ITestAzureBlobFileSystemE2E.java   | 150 ++++
 .../ITestAzureBlobFileSystemE2EScale.java       | 133 ++++
 .../ITestAzureBlobFileSystemFileStatus.java     |  64 ++
 .../azurebfs/ITestAzureBlobFileSystemFlush.java | 204 ++++++
 .../ITestAzureBlobFileSystemInitAndCreate.java  |  50 ++
 .../ITestAzureBlobFileSystemListStatus.java     | 132 ++++
 .../azurebfs/ITestAzureBlobFileSystemMkDir.java |  88 +++
 .../azurebfs/ITestAzureBlobFileSystemOpen.java  |  41 ++
 .../ITestAzureBlobFileSystemRandomRead.java     | 582 ++++++++++++++++
 .../ITestAzureBlobFileSystemRename.java         | 152 ++++
 .../azurebfs/ITestFileSystemInitialization.java |  78 +++
 .../azurebfs/ITestFileSystemRegistration.java   |  82 +++
 .../fs/azurebfs/ITestWasbAbfsCompatibility.java | 202 ++++++
 .../constants/TestConfigurationKeys.java        |  37 +
 .../fs/azurebfs/constants/package-info.java     |  22 +
 .../DependencyInjectedContractTest.java         |  63 ++
 .../contract/ITestAbfsFileSystemContract.java   |  54 ++
 .../ITestAbfsFileSystemContractAppend.java      |  70 ++
 .../ITestAbfsFileSystemContractConcat.java      |  62 ++
 .../ITestAbfsFileSystemContractCreate.java      |  63 ++
 .../ITestAbfsFileSystemContractDelete.java      |  63 ++
 .../ITestAbfsFileSystemContractDistCp.java      |  44 ++
 ...TestAbfsFileSystemContractGetFileStatus.java |  62 ++
 .../ITestAbfsFileSystemContractMkdir.java       |  63 ++
 .../ITestAbfsFileSystemContractOpen.java        |  63 ++
 .../ITestAbfsFileSystemContractRename.java      |  63 ++
 ...TestAbfsFileSystemContractRootDirectory.java |  67 ++
 ...ITestAbfsFileSystemContractSecureDistCp.java |  44 ++
 .../ITestAbfsFileSystemContractSeek.java        |  63 ++
 .../ITestAbfsFileSystemContractSetTimes.java    |  62 ++
 .../ITestAzureBlobFileSystemBasics.java         |  91 +++
 .../fs/azurebfs/contract/package-info.java      |  22 +
 .../TestConfigurationValidators.java            | 124 ++++
 .../fs/azurebfs/diagnostics/package-info.java   |  22 +
 .../apache/hadoop/fs/azurebfs/package-info.java |  22 +
 .../services/ITestAbfsHttpServiceImpl.java      | 122 ++++
 .../services/ITestReadWriteAndSeek.java         |  80 +++
 .../services/ITestTracingServiceImpl.java       |  79 +++
 .../services/MockAbfsHttpClientFactoryImpl.java |  69 ++
 .../services/MockAbfsServiceInjectorImpl.java   |  50 ++
 .../services/MockServiceProviderImpl.java       |  36 +
 ...estConfigurationServiceFieldsValidation.java | 149 ++++
 .../fs/azurebfs/services/package-info.java      |  22 +
 .../hadoop/fs/azurebfs/utils/TestUriUtils.java  |  48 ++
 .../hadoop/fs/azurebfs/utils/package-info.java  |  22 +
 .../hadoop-azure/src/test/resources/abfs.xml    |  64 ++
 .../src/test/resources/azure-bfs-test.xml       |  76 ++
 .../src/test/resources/log4j.properties         |   2 +
 129 files changed, 12606 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index d555036..ac245c7 100644
--- a/.gitignore
+++ b/.gitignore
@@ -22,6 +22,7 @@ make-build-debug
 # Filesystem contract test options and credentials
 auth-keys.xml
 azure-auth-keys.xml
+azure-bfs-auth-keys.xml
 
 # External tool builders
 */.externalToolBuilders

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 29c2bc2..93b8ddd 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1616,6 +1616,16 @@
   </description>
 </property>
 <property>
+  <name>fs.abfs.impl</name>
+  <value>org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem</value>
+  <description>The implementation class of the Azure Blob Filesystem</description>
+</property>
+<property>
+  <name>fs.abfss.impl</name>
+  <value>org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem</value>
+  <description>The implementation class of the Secure Azure Blob Filesystem</description>
+</property>
+<property>
   <name>fs.azure.local.sas.key.mode</name>
   <value>false</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
index 023c831..e10617d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
@@ -113,6 +113,9 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     xmlPrefixToSkipCompare.add("fs.wasb.impl");
     xmlPrefixToSkipCompare.add("fs.wasbs.impl");
     xmlPrefixToSkipCompare.add("fs.azure.");
+    xmlPrefixToSkipCompare.add("fs.abfs.impl");
+    xmlPrefixToSkipCompare.add("fs.abfss.impl");
+
 
     // ADL properties are in a different subtree
     // - org.apache.hadoop.hdfs.web.ADLConfKeys

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 7603842..64aa43e 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1207,6 +1207,11 @@
       </dependency>
       <dependency>
         <groupId>org.apache.htrace</groupId>
+        <artifactId>htrace-core</artifactId>
+        <version>3.1.0-incubating</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.htrace</groupId>
         <artifactId>htrace-core4</artifactId>
         <version>4.1.0-incubating</version>
       </dependency>
@@ -1341,6 +1346,12 @@
      </dependency>
 
       <dependency>
+        <groupId>org.threadly</groupId>
+        <artifactId>threadly</artifactId>
+        <version>4.9.0</version>
+      </dependency>
+
+      <dependency>
         <groupId>com.aliyun.oss</groupId>
         <artifactId>aliyun-sdk-oss</artifactId>
         <version>2.8.3</version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml
index 52b5b72..d4046ef 100644
--- a/hadoop-tools/hadoop-azure/pom.xml
+++ b/hadoop-tools/hadoop-azure/pom.xml
@@ -67,6 +67,7 @@
             <!-- To run with the default Sun ruleset, 
                  comment out the configLocation line -->
             <configLocation>src/config/checkstyle.xml</configLocation>
+            <suppressionsLocation>src/config/checkstyle-suppressions.xml</suppressionsLocation>
           </configuration>
           
       </plugin>
@@ -149,10 +150,22 @@
     </dependency>
 
     <dependency>
+      <groupId>org.threadly</groupId>
+      <artifactId>threadly</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-core</artifactId>
       <scope>compile</scope>
     </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+      <scope>compile</scope>
+    </dependency>
     
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>
@@ -173,8 +186,38 @@
     </dependency>
 
     <dependency>
+      <groupId>org.apache.htrace</groupId>
+      <artifactId>htrace-core</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.htrace</groupId>
+      <artifactId>htrace-core4</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.inject</groupId>
+      <artifactId>guice</artifactId>
+      <scope>compile</scope>
+      <!-- we have a dependency on a lower version -->
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
       <scope>compile</scope>
     </dependency>
 
@@ -183,15 +226,8 @@
       <artifactId>jetty-util-ajax</artifactId>
       <scope>compile</scope>
     </dependency>
-    
-    
-    <!-- dependencies use for test only -->
-    <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-      <scope>test</scope>
-    </dependency>
 
+    <!-- dependencies use for test only -->
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
@@ -229,10 +265,6 @@
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
 
   </dependencies>
 
@@ -398,8 +430,10 @@
                     <exclude>**/ITestNativeAzureFileSystemConcurrencyLive.java</exclude>
                     <exclude>**/ITestNativeAzureFileSystemLive.java</exclude>
                     <exclude>**/ITestNativeAzureFSPageBlobLive.java</exclude>
+                    <exclude>**/ITestAzureBlobFileSystemRandomRead.java</exclude>
                     <exclude>**/ITestWasbRemoteCallHelper.java</exclude>
                     <exclude>**/ITestBlockBlobInputStream.java</exclude>
+                    <exclude>**/ITestWasbAbfsCompatibility.java</exclude>
                   </excludes>
                 </configuration>
               </execution>
@@ -424,12 +458,15 @@
                     <fs.azure.scale.test.list.performance.files>${fs.azure.scale.test.list.performance.files}</fs.azure.scale.test.list.performance.files>
                   </systemPropertyVariables>
                   <includes>
+                    <include>**/ITestWasbAbfsCompatibility.java</include>
                     <include>**/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java</include>
                     <include>**/ITestFileSystemOperationsWithThreads.java</include>
                     <include>**/ITestOutOfBandAzureBlobOperationsLive.java</include>
                     <include>**/ITestNativeAzureFileSystemAuthorizationWithOwner.java</include>
                     <include>**/ITestNativeAzureFileSystemConcurrencyLive.java</include>
                     <include>**/ITestNativeAzureFileSystemLive.java</include>
+                    <include>**/ITestNativeAzureFSPageBlobLive.java</include>
+                    <include>**/ITestAzureBlobFileSystemRandomRead.java</include>
                     <include>**/ITestWasbRemoteCallHelper.java</include>
                     <include>**/ITestBlockBlobInputStream.java</include>
                   </includes>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
new file mode 100644
index 0000000..0204355
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
@@ -0,0 +1,47 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE suppressions PUBLIC
+        "-//Puppy Crawl//DTD Suppressions 1.0//EN"
+        "http://www.puppycrawl.com/dtds/suppressions_1_0.dtd">
+
+
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!--
+  Checkstyle configuration that checks the sun coding conventions from:
+
+    - the Java Language Specification at
+      http://java.sun.com/docs/books/jls/second_edition/html/index.html
+
+    - the Sun Code Conventions at http://java.sun.com/docs/codeconv/
+
+    - the Javadoc guidelines at
+      http://java.sun.com/j2se/javadoc/writingdoccomments/index.html
+
+    - the JDK Api documentation http://java.sun.com/j2se/docs/api/index.html
+
+    - some best practices
+
+  Checkstyle is very configurable. Be sure to read the documentation at
+  http://checkstyle.sf.net (or in your downloaded distribution).
+
+  Most Checks are configurable, be sure to consult the documentation.
+  To completely disable a check, just comment it out or delete it from the file.
+  Finally, it is worth reading the documentation.
+-->
+
+<suppressions>
+    <suppress checks="ParameterNumber|MagicNumber"
+              files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]services[\\/]AbfsHttpServiceImpl.java"/>
+</suppressions>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java
new file mode 100644
index 0000000..707e264
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+
+/**
+ * Azure Blob File System implementation of AbstractFileSystem.
+ * This impl delegates to the old FileSystem
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class Abfs extends DelegateToFileSystem {
+
+  Abfs(final URI theUri, final Configuration conf) throws IOException,
+      URISyntaxException {
+    super(theUri, new AzureBlobFileSystem(), conf, FileSystemUriSchemes.ABFS_SCHEME, false);
+  }
+
+  @Override
+  public int getUriDefaultPort() {
+    return -1;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java
new file mode 100644
index 0000000..19c0f7a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+
+/**
+ * Azure Blob File System implementation of AbstractFileSystem.
+ * This impl delegates to the old FileSystem
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class Abfss extends DelegateToFileSystem {
+
+  Abfss(final URI theUri, final Configuration conf) throws IOException,
+      URISyntaxException {
+    super(theUri, new SecureAzureBlobFileSystem(), conf, FileSystemUriSchemes.ABFS_SECURE_SCHEME, false);
+  }
+
+  @Override
+  public int getUriDefaultPort() {
+    return -1;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
new file mode 100644
index 0000000..707c81e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
@@ -0,0 +1,612 @@
+/**
+ * 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.azurebfs;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.threadly.util.ExceptionUtils;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.apache.htrace.core.TraceScope;
+
+/**
+ * A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
+ * href="http://store.azure.com/">Windows Azure</a>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AzureBlobFileSystem extends FileSystem {
+  public static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystem.class);
+  private URI uri;
+  private Path workingDir;
+  private UserGroupInformation userGroupInformation;
+  private String user;
+  private String primaryUserGroup;
+  private AbfsServiceProvider abfsServiceProvider;
+  private TracingService tracingService;
+  private AbfsHttpService abfsHttpService;
+  private ConfigurationService configurationService;
+  private boolean isClosed;
+
+  @Override
+  public void initialize(URI uri, Configuration configuration)
+      throws IOException {
+    uri = ensureAuthority(uri, configuration);
+    super.initialize(uri, configuration);
+
+    setConf(configuration);
+
+    try {
+      this.abfsServiceProvider = AbfsServiceProviderImpl.create(configuration);
+      this.tracingService = abfsServiceProvider.get(TracingService.class);
+      this.abfsHttpService = abfsServiceProvider.get(AbfsHttpService.class);
+      this.configurationService = abfsServiceProvider.get(ConfigurationService.class);
+    } catch (AzureBlobFileSystemException exception) {
+      throw new IOException(exception);
+    }
+
+    this.LOG.debug(
+        "Initializing AzureBlobFileSystem for {}", uri);
+
+    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+    this.userGroupInformation = UserGroupInformation.getCurrentUser();
+    this.user = userGroupInformation.getUserName();
+    this.primaryUserGroup = userGroupInformation.getPrimaryGroupName();
+
+    this.LOG.debug(
+        "Initializing NativeAzureFileSystem for {}", uri);
+
+    this.setWorkingDirectory(this.getHomeDirectory());
+
+    if (this.configurationService.getCreateRemoteFileSystemDuringInitialization()) {
+      this.createFileSystem();
+    }
+
+    this.mkdirs(this.workingDir);
+  }
+
+  public boolean isSecure() {
+    return false;
+  }
+
+  @Override
+  public URI getUri() {
+    return this.uri;
+  }
+
+  @Override
+  public FSDataInputStream open(final Path path, final int bufferSize) throws IOException {
+    this.LOG.debug(
+        "AzureBlobFileSystem.open path: {} bufferSize: {}", path.toString(), bufferSize);
+
+    try {
+      InputStream inputStream = abfsHttpService.openFileForRead(this, makeQualified(path), statistics);
+      return new FSDataInputStream(inputStream);
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+      return null;
+    }
+  }
+
+  @Override
+  public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize,
+      final short replication, final long blockSize, final Progressable progress) throws IOException {
+    this.LOG.debug(
+        "AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}",
+        f.toString(),
+        permission,
+        overwrite,
+        blockSize);
+
+    try {
+      OutputStream outputStream = abfsHttpService.createFile(this, makeQualified(f), overwrite);
+      return new FSDataOutputStream(outputStream, statistics);
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(f, ex);
+      return null;
+    }
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission,
+      final boolean overwrite, final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress) throws IOException {
+
+    final Path parent = f.getParent();
+    final FileStatus parentFileStatus = tryGetFileStatus(parent);
+
+    if (parentFileStatus == null) {
+      throw new FileNotFoundException("Cannot create file "
+          + f.getName() + " because parent folder does not exist.");
+    }
+
+    return create(f, permission, overwrite, bufferSize, replication, blockSize, progress);
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission,
+      final EnumSet<CreateFlag> flags, final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress) throws IOException {
+
+    // Check if file should be appended or overwritten. Assume that the file
+    // is overwritten on if the CREATE and OVERWRITE create flags are set.
+    final EnumSet<CreateFlag> createflags =
+        EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE);
+    final boolean overwrite = flags.containsAll(createflags);
+
+    // Delegate the create non-recursive call.
+    return this.createNonRecursive(f, permission, overwrite,
+        bufferSize, replication, blockSize, progress);
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public FSDataOutputStream createNonRecursive(final Path f,
+      final boolean overwrite, final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress) throws IOException {
+    return this.createNonRecursive(f, FsPermission.getFileDefault(),
+        overwrite, bufferSize, replication, blockSize, progress);
+  }
+
+  @Override
+  public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException {
+    this.LOG.debug(
+        "AzureBlobFileSystem.append path: {} bufferSize: {}",
+        f.toString(),
+        bufferSize);
+
+    try {
+      OutputStream outputStream = abfsHttpService.openFileForWrite(this, makeQualified(f), false);
+      return new FSDataOutputStream(outputStream, statistics);
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(f, ex);
+      return null;
+    }
+  }
+
+  public boolean rename(final Path src, final Path dst) throws IOException {
+    this.LOG.debug(
+        "AzureBlobFileSystem.rename src: {} dst: {}", src.toString(), dst.toString());
+
+    Path parentFolder = src.getParent();
+    if (parentFolder == null) {
+      return false;
+    }
+
+    final FileStatus dstFileStatus = tryGetFileStatus(dst);
+    try {
+      String sourceFileName = src.getName();
+      Path adjustedDst = dst;
+
+      if (dstFileStatus != null) {
+        if (!dstFileStatus.isDirectory()) {
+          return src.equals(dst);
+        }
+
+        adjustedDst = new Path(dst, sourceFileName);
+      }
+
+      abfsHttpService.rename(this, makeQualified(src), makeQualified(adjustedDst));
+      return true;
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(
+              src,
+              ex,
+              AzureServiceErrorCode.PATH_ALREADY_EXISTS,
+              AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH,
+              AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND,
+              AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE,
+              AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND);
+      return false;
+    }
+
+  }
+
+  @Override
+  public boolean delete(final Path f, final boolean recursive) throws IOException {
+    this.LOG.debug(
+        "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive);
+
+    if (f.isRoot()) {
+      if (!recursive) {
+        return false;
+      }
+
+      return deleteRoot();
+    }
+
+    try {
+      abfsHttpService.delete(this, makeQualified(f), recursive);
+      return true;
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND);
+      return false;
+    }
+
+  }
+
+  @Override
+  public FileStatus[] listStatus(final Path f) throws IOException {
+    this.LOG.debug(
+        "AzureBlobFileSystem.listStatus path: {}", f.toString());
+
+    try {
+      FileStatus[] result = abfsHttpService.listStatus(this, makeQualified(f));
+      return result;
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(f, ex);
+      return null;
+    }
+  }
+
+  @Override
+  public boolean mkdirs(final Path f, final FsPermission permission) throws IOException {
+    this.LOG.debug(
+        "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f.toString(), permission);
+
+    final Path parentFolder = f.getParent();
+    if (parentFolder == null) {
+      // Cannot create root
+      return true;
+    }
+
+    try {
+      abfsHttpService.createDirectory(this, makeQualified(f));
+      return true;
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(f, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS);
+      return true;
+    }
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (isClosed) {
+      return;
+    }
+
+    super.close();
+    this.LOG.debug("AzureBlobFileSystem.close");
+
+    try {
+      abfsHttpService.closeFileSystem(this);
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(null, ex);
+      this.isClosed = true;
+    }
+  }
+
+  @Override
+  public FileStatus getFileStatus(final Path f) throws IOException {
+    this.LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f.toString());
+
+    try {
+      return abfsHttpService.getFileStatus(this, makeQualified(f));
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(f, ex);
+      return null;
+    }
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return this.workingDir;
+  }
+
+  @Override
+  public void setWorkingDirectory(final Path newDir) {
+    if (newDir.isAbsolute()) {
+      this.workingDir = newDir;
+    } else {
+      this.workingDir = new Path(workingDir, newDir);
+    }
+  }
+
+  @Override
+  public String getScheme() {
+    return FileSystemUriSchemes.ABFS_SCHEME;
+  }
+
+  @Override
+  public Path getHomeDirectory() {
+    return makeQualified(new Path(
+            FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX + "/" + this.userGroupInformation.getShortUserName()));
+  }
+
+  /**
+   * Return an array containing hostnames, offset and size of
+   * portions of the given file. For ABFS we'll just lie and give
+   * fake hosts to make sure we get many splits in MR jobs.
+   */
+  @Override
+  public BlockLocation[] getFileBlockLocations(FileStatus file,
+      long start, long len) throws IOException {
+    if (file == null) {
+      return null;
+    }
+
+    if ((start < 0) || (len < 0)) {
+      throw new IllegalArgumentException("Invalid start or len parameter");
+    }
+
+    if (file.getLen() < start) {
+      return new BlockLocation[0];
+    }
+    final String blobLocationHost = this.configurationService.getAzureBlockLocationHost();
+
+    final String[] name = { blobLocationHost };
+    final String[] host = { blobLocationHost };
+    long blockSize = file.getBlockSize();
+    if (blockSize <= 0) {
+      throw new IllegalArgumentException(
+          "The block size for the given file is not a positive number: "
+              + blockSize);
+    }
+    int numberOfLocations = (int) (len / blockSize)
+        + ((len % blockSize == 0) ? 0 : 1);
+    BlockLocation[] locations = new BlockLocation[numberOfLocations];
+    for (int i = 0; i < locations.length; i++) {
+      long currentOffset = start + (i * blockSize);
+      long currentLength = Math.min(blockSize, start + len - currentOffset);
+      locations[i] = new BlockLocation(name, host, currentOffset, currentLength);
+    }
+
+    return locations;
+  }
+
+  public String getOwnerUser() {
+    return user;
+  }
+
+  public String getOwnerUserPrimaryGroup() {
+    return primaryUserGroup;
+  }
+
+  private boolean deleteRoot() throws IOException {
+    this.LOG.debug("Deleting root content");
+
+    final ExecutorService executorService = Executors.newFixedThreadPool(10);
+
+    try {
+      final FileStatus[] ls = listStatus(makeQualified(new Path(File.separator)));
+      final ArrayList<Future> deleteTasks = new ArrayList<>();
+      for (final FileStatus fs : ls) {
+        final Future deleteTask = executorService.submit(new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            delete(fs.getPath(), fs.isDirectory());
+            return null;
+          }
+        });
+        deleteTasks.add(deleteTask);
+      }
+
+      for (final Future deleteTask : deleteTasks) {
+        execute("deleteRoot", new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            deleteTask.get();
+            return null;
+          }
+        });
+      }
+    }
+    finally {
+      executorService.shutdownNow();
+    }
+
+    return true;
+  }
+
+  private FileStatus tryGetFileStatus(final Path f) {
+    try {
+      return getFileStatus(f);
+    }
+    catch (IOException ex) {
+      this.LOG.debug("File not found {}", f.toString());
+      return null;
+    }
+  }
+
+  private void createFileSystem() throws IOException {
+    this.LOG.debug(
+        "AzureBlobFileSystem.createFileSystem uri: {}", uri);
+    try {
+      abfsHttpService.createFilesystem(this);
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS);
+    }
+
+
+  }
+
+  private URI ensureAuthority(URI uri, final Configuration conf) {
+
+    Preconditions.checkNotNull(uri, "uri");
+
+    if (uri.getAuthority() == null) {
+      final URI defaultUri = FileSystem.getDefaultUri(conf);
+
+      if (defaultUri != null && isAbfsScheme(defaultUri.getScheme())) {
+        try {
+          // Reconstruct the URI with the authority from the default URI.
+          uri = new URI(
+              uri.getScheme(),
+              defaultUri.getAuthority(),
+              uri.getPath(),
+              uri.getQuery(),
+              uri.getFragment());
+        } catch (URISyntaxException e) {
+          // This should never happen.
+          throw new IllegalArgumentException(new InvalidUriException(uri.toString()));
+        }
+      }
+    }
+
+    if (uri.getAuthority() == null) {
+      throw new IllegalArgumentException(new InvalidUriAuthorityException(uri.toString()));
+    }
+
+    return uri;
+  }
+
+  private boolean isAbfsScheme(final String scheme) {
+    if (scheme == null) {
+      return false;
+    }
+
+    if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME) || scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  @VisibleForTesting
+  <T> FileSystemOperation execute(
+      final String scopeDescription,
+      final Callable<T> callableFileOperation) throws IOException {
+    return execute(scopeDescription, callableFileOperation, null);
+  }
+
+  @VisibleForTesting
+  <T> FileSystemOperation execute(
+      final String scopeDescription,
+      final Callable<T> callableFileOperation,
+      T defaultResultValue) throws IOException {
+
+    final TraceScope traceScope = tracingService.traceBegin(scopeDescription);
+    try {
+      final T executionResult = callableFileOperation.call();
+      return new FileSystemOperation(executionResult, null);
+    } catch (AbfsRestOperationException abfsRestOperationException) {
+      return new FileSystemOperation(defaultResultValue, abfsRestOperationException);
+    } catch (AzureBlobFileSystemException azureBlobFileSystemException) {
+      tracingService.traceException(traceScope, azureBlobFileSystemException);
+      throw new IOException(azureBlobFileSystemException);
+    } catch (Exception exception) {
+      if (exception instanceof ExecutionException) {
+        exception = (Exception) ExceptionUtils.getRootCause(exception);
+      }
+
+      final FileSystemOperationUnhandledException fileSystemOperationUnhandledException = new FileSystemOperationUnhandledException(exception);
+      tracingService.traceException(traceScope, fileSystemOperationUnhandledException);
+      throw new IOException(fileSystemOperationUnhandledException);
+    } finally {
+      tracingService.traceEnd(traceScope);
+    }
+  }
+
+  private void checkException(final Path path,
+                              final AzureBlobFileSystemException exception,
+                              final AzureServiceErrorCode... allowedErrorCodesList) throws IOException {
+    if (exception instanceof AbfsRestOperationException) {
+      AbfsRestOperationException ere = (AbfsRestOperationException) exception;
+
+      if (ArrayUtils.contains(allowedErrorCodesList, ere.getErrorCode())) {
+        return;
+      }
+      int statusCode = ere.getStatusCode();
+
+      //AbfsRestOperationException.getMessage() contains full error info including path/uri.
+      if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) {
+        throw new FileNotFoundException(ere.getMessage());
+      } else if (statusCode == HttpURLConnection.HTTP_CONFLICT) {
+        throw new FileAlreadyExistsException(ere.getMessage());
+      } else {
+        throw ere;
+      }
+    } else {
+      if (path == null) {
+        throw exception;
+      }
+      // record info of path
+      throw new PathIOException(path.toString(), exception);
+    }
+  }
+
+  @VisibleForTesting
+  FileSystem.Statistics getFsStatistics() {
+    return this.statistics;
+  }
+
+  @VisibleForTesting
+  static class FileSystemOperation<T> {
+    private final T result;
+    private final AbfsRestOperationException exception;
+
+    FileSystemOperation(final T result, final AbfsRestOperationException exception) {
+      this.result = result;
+      this.exception = exception;
+    }
+
+    public boolean failed() {
+      return this.exception != null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java
new file mode 100644
index 0000000..a6ad829
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java
@@ -0,0 +1,41 @@
+/**
+ * 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.azurebfs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+
+/**
+ * A secure {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
+ * href="http://store.azure.com/">Windows Azure</a>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class SecureAzureBlobFileSystem extends AzureBlobFileSystem {
+  @Override
+  public boolean isSecure() {
+    return true;
+  }
+
+  @Override
+  public String getScheme() {
+    return FileSystemUriSchemes.ABFS_SECURE_SCHEME;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
new file mode 100644
index 0000000..2ec4db0
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
@@ -0,0 +1,76 @@
+/**
+ * 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.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all constant keys used in abfs rest client here
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class AbfsHttpConstants {
+  // Abfs Http client constants
+  public static final String FILESYSTEM = "filesystem";
+  public static final String FILE = "file";
+  public static final String DIRECTORY = "directory";
+  public static final String APPEND_ACTION = "append";
+  public static final String FLUSH_ACTION = "flush";
+  public static final String SET_PROPERTIES_ACTION = "setProperties";
+  public static final String DEFAULT_TIMEOUT = "90";
+
+  public static final String JAVA_VERSION = "java.version";
+  public static final String OS_NAME = "os.name";
+  public static final String OS_VERSION = "os.version";
+
+  public static final String CLIENT_VERSION = "Azure Blob FS/1.0";
+
+  // Abfs Http Verb
+  public static final String HTTP_METHOD_DELETE = "DELETE";
+  public static final String HTTP_METHOD_GET = "GET";
+  public static final String HTTP_METHOD_HEAD = "HEAD";
+  public static final String HTTP_METHOD_PATCH = "PATCH";
+  public static final String HTTP_METHOD_POST = "POST";
+  public static final String HTTP_METHOD_PUT = "PUT";
+
+  // Abfs generic constants
+  public static final String SINGLE_WHITE_SPACE = " ";
+  public static final String EMPTY_STRING = "";
+  public static final String FORWARD_SLASH = "/";
+  public static final String DOT = ".";
+  public static final String PLUS = "+";
+  public static final String STAR = "*";
+  public static final String COMMA = ",";
+  public static final String EQUAL = "=";
+  public static final String QUESTION_MARK = "?";
+  public static final String AND_MARK = "&";
+  public static final String SEMICOLON = ";";
+  public static final String HTTP_HEADER_PREFIX = "x-ms-";
+
+  public static final String PLUS_ENCODE = "%20";
+  public static final String FORWARD_SLASH_ENCODE = "%2F";
+  public static final String AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER = "@";
+  public static final String UTF_8 = "utf-8";
+  public static final String GMT_TIMEZONE = "GMT";
+  public static final String APPLICATION_JSON = "application/json";
+  public static final String APPLICATION_OCTET_STREAM = "application/octet-stream";
+
+  private AbfsHttpConstants() {}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
new file mode 100644
index 0000000..ead1003
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
@@ -0,0 +1,57 @@
+/**
+ * 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.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ConfigurationKeys {
+  public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key.";
+  public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX = "fs\\.azure\\.account\\.key\\.(.*)";
+  public static final String FS_AZURE_SECURE_MODE = "fs.azure.secure.mode";
+
+  // Retry strategy defined by the user
+  public static final String AZURE_MIN_BACKOFF_INTERVAL = "fs.azure.io.retry.min.backoff.interval";
+  public static final String AZURE_MAX_BACKOFF_INTERVAL = "fs.azure.io.retry.max.backoff.interval";
+  public static final String AZURE_BACKOFF_INTERVAL = "fs.azure.io.retry.backoff.interval";
+  public static final String AZURE_MAX_IO_RETRIES = "fs.azure.io.retry.max.retries";
+
+  // Remove this and use common azure storage emulator property for public release.
+  public static final String FS_AZURE_EMULATOR_ENABLED = "fs.azure.abfs.emulator.enabled";
+
+  // Read and write buffer sizes defined by the user
+  public static final String AZURE_WRITE_BUFFER_SIZE = "fs.azure.write.request.size";
+  public static final String AZURE_READ_BUFFER_SIZE = "fs.azure.read.request.size";
+  public static final String AZURE_BLOCK_SIZE_PROPERTY_NAME = "fs.azure.block.size";
+  public static final String AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME = "fs.azure.block.location.impersonatedhost";
+  public static final String AZURE_CONCURRENT_CONNECTION_VALUE_OUT = "fs.azure.concurrentRequestCount.out";
+  public static final String AZURE_CONCURRENT_CONNECTION_VALUE_IN = "fs.azure.concurrentRequestCount.in";
+  public static final String AZURE_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append";
+  public static final String AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = "fs.azure.createRemoteFileSystemDuringInitialization";
+  public static final String FS_AZURE_AUTOTHROTTLING_ENABLE = "fs.azure.autothrottling.enable";
+  public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key";
+  public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth";
+
+  private ConfigurationKeys() {}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
new file mode 100644
index 0000000..482158c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
@@ -0,0 +1,59 @@
+/**
+ * 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.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all the Azure Blob File System related configurations.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class FileSystemConfigurations {
+  public static final String USER_HOME_DIRECTORY_PREFIX = "/user";
+
+  // Retry parameter defaults.
+  public static final int DEFAULT_MIN_BACKOFF_INTERVAL = 3 * 1000;  // 3s
+  public static final int DEFAULT_MAX_BACKOFF_INTERVAL = 30 * 1000;  // 30s
+  public static final int DEFAULT_BACKOFF_INTERVAL = 3 * 1000;  // 3s
+  public static final int DEFAULT_MAX_RETRY_ATTEMPTS = 30;
+
+  private static final int ONE_KB = 1024;
+  private static final int ONE_MB = ONE_KB * ONE_KB;
+
+  // Default upload and download buffer size
+  public static final int DEFAULT_WRITE_BUFFER_SIZE = 4 * ONE_MB;  // 4 MB
+  public static final int DEFAULT_READ_BUFFER_SIZE = 4 * ONE_MB;  // 4 MB
+  public static final int MIN_BUFFER_SIZE = 16 * ONE_KB;  // 16 KB
+  public static final int MAX_BUFFER_SIZE = 100 * ONE_MB;  // 100 MB
+  public static final long MAX_AZURE_BLOCK_SIZE = 512 * 1024 * 1024L;
+  public static final String AZURE_BLOCK_LOCATION_HOST_DEFAULT = "localhost";
+
+  public static final int MAX_CONCURRENT_READ_THREADS = 12;
+  public static final int MAX_CONCURRENT_WRITE_THREADS = 8;
+  public static final boolean DEFAULT_READ_TOLERATE_CONCURRENT_APPEND = false;
+  public static final boolean DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = false;
+
+  public static final String DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES = "/hbase";
+
+  public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1;
+
+  private FileSystemConfigurations() {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java
new file mode 100644
index 0000000..c7a0cda
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java
@@ -0,0 +1,42 @@
+/**
+ * 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.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all Azure Blob File System valid URI schemes.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class FileSystemUriSchemes {
+  public static final String ABFS_SCHEME = "abfs";
+  public static final String ABFS_SECURE_SCHEME = "abfss";
+  public static final String ABFS_DNS_PREFIX = "dfs";
+
+  public static final String HTTP_SCHEME = "http";
+  public static final String HTTPS_SCHEME = "https";
+
+  public static final String WASB_SCHEME = "wasb";
+  public static final String WASB_SECURE_SCHEME = "wasbs";
+  public static final String WASB_DNS_PREFIX = "blob";
+
+  private FileSystemUriSchemes() {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
new file mode 100644
index 0000000..9b7f9bc
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
@@ -0,0 +1,57 @@
+/**
+ * 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.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all abfs http headers here
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class HttpHeaderConfigurations {
+  public static final String ACCEPT = "Accept";
+  public static final String ACCEPT_CHARSET = "Accept-Charset";
+  public static final String AUTHORIZATION = "Authorization";
+  public static final String IF_MODIFIED_SINCE = "If-Modified-Since";
+  public static final String IF_UNMODIFIED_SINCE = "If-Unmodified-Since";
+  public static final String IF_MATCH = "If-Match";
+  public static final String IF_NONE_MATCH = "If-None-Match";
+  public static final String CONTENT_LENGTH = "Content-Length";
+  public static final String CONTENT_ENCODING = "Content-Encoding";
+  public static final String CONTENT_LANGUAGE = "Content-Language";
+  public static final String CONTENT_MD5 = "Content-MD5";
+  public static final String CONTENT_TYPE = "Content-Type";
+  public static final String RANGE = "Range";
+  public static final String TRANSFER_ENCODING = "Transfer-Encoding";
+  public static final String USER_AGENT = "User-Agent";
+  public static final String X_HTTP_METHOD_OVERRIDE = "X-HTTP-Method-Override";
+  public static final String X_MS_CLIENT_REQUEST_ID = "x-ms-client-request-id";
+  public static final String X_MS_DATE = "x-ms-date";
+  public static final String X_MS_REQUEST_ID = "x-ms-request-id";
+  public static final String X_MS_VERSION = "x-ms-version";
+  public static final String X_MS_RESOURCE_TYPE = "x-ms-resource-type";
+  public static final String X_MS_CONTINUATION = "x-ms-continuation";
+  public static final String ETAG = "ETag";
+  public static final String X_MS_PROPERTIES = "x-ms-properties";
+  public static final String X_MS_RENAME_SOURCE = "x-ms-rename-source";
+  public static final String LAST_MODIFIED = "Last-Modified";
+
+  private HttpHeaderConfigurations() {}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
new file mode 100644
index 0000000..a9f7d33
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
@@ -0,0 +1,40 @@
+/**
+ * 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.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all Http Query params here
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class HttpQueryParams {
+  public static final String QUERY_PARAM_RESOURCE = "resource";
+  public static final String QUERY_PARAM_DIRECTORY = "directory";
+  public static final String QUERY_PARAM_CONTINUATION = "continuation";
+  public static final String QUERY_PARAM_RECURSIVE = "recursive";
+  public static final String QUERY_PARAM_MAXRESULTS = "maxResults";
+  public static final String QUERY_PARAM_ACTION = "action";
+  public static final String QUERY_PARAM_POSITION = "position";
+  public static final String QUERY_PARAM_TIMEOUT = "timeout";
+  public static final String QUERY_PARAM_RETAIN_UNCOMMITTED_DATA = "retainUncommittedData";
+
+  private HttpQueryParams() {}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java
new file mode 100644
index 0000000..e6a471b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.constants;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java
new file mode 100644
index 0000000..462ebbc
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java
@@ -0,0 +1,104 @@
+/**
+ * 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.azurebfs.contracts.annotations;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Definitions of Annotations for all types of the validators
+ */
+@InterfaceStability.Evolving
+public class ConfigurationValidationAnnotations {
+  /**
+   * Describes the requirements when validating the annotated int field
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface IntegerConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    int MaxValue() default Integer.MAX_VALUE;
+
+    int MinValue() default Integer.MIN_VALUE;
+
+    int DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+
+  /**
+   * Describes the requirements when validating the annotated long field
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface LongConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    long MaxValue() default Long.MAX_VALUE;
+
+    long MinValue() default Long.MIN_VALUE;
+
+    long DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+
+  /**
+   * Describes the requirements when validating the annotated String field
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface StringConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    String DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+
+  /**
+   * Describes the requirements when validating the annotated String field
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Base64StringConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    String DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+
+  /**
+   * Describes the requirements when validating the annotated boolean field
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface BooleanConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    boolean DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java
new file mode 100644
index 0000000..0fc4deb
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts.annotations;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java
new file mode 100644
index 0000000..796f785
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java
@@ -0,0 +1,37 @@
+/**
+ * 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.azurebfs.contracts.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * ConfigurationValidator to validate the value of a configuration key
+ * @param <T> the type of the validator and the validated value
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface ConfigurationValidator<T> {
+  /**
+   * Validates the configValue
+   * @return validated value of type T
+   */
+  T validate(String configValue) throws InvalidConfigurationValueException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java
new file mode 100644
index 0000000..f8d27b2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts.diagnostics;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java
new file mode 100644
index 0000000..f0b69ef
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java
@@ -0,0 +1,84 @@
+/**
+ * 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.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+
+/**
+ * Exception to wrap Azure service error responses.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class AbfsRestOperationException extends AzureBlobFileSystemException {
+  private final int statusCode;
+  private final AzureServiceErrorCode errorCode;
+  private final String errorMessage;
+
+  public AbfsRestOperationException(
+      final int statusCode,
+      final String errorCode,
+      final String errorMessage,
+      final Exception innerException) {
+    super("Status code: " + statusCode + " error code: " + errorCode + " error message: " + errorMessage, innerException);
+
+    this.statusCode = statusCode;
+    this.errorCode = AzureServiceErrorCode.getAzureServiceCode(this.statusCode, errorCode);
+    this.errorMessage = errorMessage;
+  }
+
+  public AbfsRestOperationException(
+      final int statusCode,
+      final String errorCode,
+      final String errorMessage,
+      final Exception innerException,
+      final AbfsHttpOperation abfsHttpOperation) {
+    super(formatMessage(abfsHttpOperation));
+
+    this.statusCode = statusCode;
+    this.errorCode = AzureServiceErrorCode.getAzureServiceCode(this.statusCode, errorCode);
+    this.errorMessage = errorMessage;
+  }
+
+  public int getStatusCode() {
+    return this.statusCode;
+  }
+
+  public AzureServiceErrorCode getErrorCode() {
+    return this.errorCode;
+  }
+
+  public String getErrorMessage() {
+    return this.errorMessage;
+  }
+
+  private static String formatMessage(final AbfsHttpOperation abfsHttpOperation) {
+    return String.format(
+        "%1$s %2$s%nStatusCode=%3$s%nStatusDescription=%4$s%nErrorCode=%5$s%nErrorMessage=%6$s",
+        abfsHttpOperation.getMethod(),
+        abfsHttpOperation.getUrl().toString(),
+        abfsHttpOperation.getStatusCode(),
+        abfsHttpOperation.getStatusDescription(),
+        abfsHttpOperation.getStorageErrorCode(),
+        abfsHttpOperation.getStorageErrorMessage());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java
new file mode 100644
index 0000000..f31c680
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java
@@ -0,0 +1,56 @@
+/**
+ * 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.azurebfs.contracts.exceptions;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Base exception for any Azure Blob File System driver exceptions. All the exceptions must inherit this class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class AzureBlobFileSystemException extends IOException {
+  public AzureBlobFileSystemException(final String message) {
+    super(message);
+  }
+
+  public AzureBlobFileSystemException(final String message, final Exception innerException) {
+    super(message, innerException);
+  }
+
+  @Override
+  public String toString() {
+    if (this.getMessage() == null && this.getCause() == null) {
+      return "";
+    }
+
+    if (this.getCause() == null) {
+      return this.getMessage();
+    }
+
+    if (this.getMessage() == null) {
+      return this.getCause().toString();
+    }
+
+    return this.getMessage() + this.getCause().toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java
new file mode 100644
index 0000000..bf3b2f3
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java
@@ -0,0 +1,32 @@
+/**
+ * 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.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when a searched for element is not found
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ConfigurationPropertyNotFoundException extends AzureBlobFileSystemException {
+  public ConfigurationPropertyNotFoundException(String property) {
+    super("Configuration property " + property + "not found.");
+  }
+}


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


[03/50] [abbrv] hadoop git commit: HDFS-13658. Expose HighestPriorityLowRedundancy blocks statistics. Contributed by Kitti Nanasi.

Posted by tm...@apache.org.
HDFS-13658. Expose HighestPriorityLowRedundancy blocks statistics. Contributed by Kitti Nanasi.


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

Branch: refs/heads/HADOOP-15407
Commit: 9499df7b81b55b488a32fd59798a543dafef4ef8
Parents: ff06bd1
Author: Xiao Chen <xi...@apache.org>
Authored: Wed Aug 8 10:36:44 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Wed Aug 8 10:40:20 2018 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md  |  2 +
 .../hadoop/hdfs/protocol/ECBlockGroupStats.java | 27 +++++++++++-
 .../hdfs/protocol/ReplicatedBlockStats.java     | 28 ++++++++++++-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 21 ++++++++++
 .../src/main/proto/ClientNamenodeProtocol.proto |  3 ++
 .../federation/metrics/NamenodeBeanMetrics.java | 10 +++++
 .../server/federation/router/ErasureCoding.java | 13 ++++++
 .../server/blockmanagement/BlockManager.java    |  8 ++++
 .../blockmanagement/LowRedundancyBlocks.java    | 28 +++++++++++++
 .../hdfs/server/namenode/FSNamesystem.java      | 20 ++++++++-
 .../hdfs/server/namenode/NameNodeMXBean.java    | 18 ++++++++
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  | 10 +++++
 .../TestLowRedundancyBlockQueues.java           | 43 +++++++++++++-------
 .../namenode/metrics/TestNameNodeMetrics.java   | 12 ++++++
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  | 32 +++++++++++----
 15 files changed, 247 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index 4313640..83ad40a 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -244,6 +244,8 @@ Each metrics record contains tags such as HAState and Hostname as additional inf
 | `StaleDataNodes` | Current number of DataNodes marked stale due to delayed heartbeat |
 | `NumStaleStorages` | Number of storages marked as content stale (after NameNode restart/failover before first block report is received) |
 | `MissingReplOneBlocks` | Current number of missing blocks with replication factor 1 |
+| `HighestPriorityLowRedundancyReplicatedBlocks` | Current number of non-corrupt, low redundancy replicated blocks with the highest risk of loss (have 0 or 1 replica). Will be recovered with the highest priority. |
+| `HighestPriorityLowRedundancyECBlocks` | Current number of non-corrupt, low redundancy EC blocks with the highest risk of loss. Will be recovered with the highest priority. |
 | `NumFilesUnderConstruction` | Current number of files under construction |
 | `NumActiveClients` | Current number of active clients holding lease |
 | `HAState` | (HA-only) Current state of the NameNode: initializing or active or standby or stopping state |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
index 9a8ad8c..3dde604 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
@@ -34,15 +34,26 @@ public final class ECBlockGroupStats {
   private final long missingBlockGroups;
   private final long bytesInFutureBlockGroups;
   private final long pendingDeletionBlocks;
+  private final Long highestPriorityLowRedundancyBlocks;
 
   public ECBlockGroupStats(long lowRedundancyBlockGroups,
       long corruptBlockGroups, long missingBlockGroups,
       long bytesInFutureBlockGroups, long pendingDeletionBlocks) {
+    this(lowRedundancyBlockGroups, corruptBlockGroups, missingBlockGroups,
+        bytesInFutureBlockGroups, pendingDeletionBlocks, null);
+  }
+
+  public ECBlockGroupStats(long lowRedundancyBlockGroups,
+      long corruptBlockGroups, long missingBlockGroups,
+      long bytesInFutureBlockGroups, long pendingDeletionBlocks,
+      Long highestPriorityLowRedundancyBlocks) {
     this.lowRedundancyBlockGroups = lowRedundancyBlockGroups;
     this.corruptBlockGroups = corruptBlockGroups;
     this.missingBlockGroups = missingBlockGroups;
     this.bytesInFutureBlockGroups = bytesInFutureBlockGroups;
     this.pendingDeletionBlocks = pendingDeletionBlocks;
+    this.highestPriorityLowRedundancyBlocks
+        = highestPriorityLowRedundancyBlocks;
   }
 
   public long getBytesInFutureBlockGroups() {
@@ -65,6 +76,14 @@ public final class ECBlockGroupStats {
     return pendingDeletionBlocks;
   }
 
+  public boolean hasHighestPriorityLowRedundancyBlocks() {
+    return getHighestPriorityLowRedundancyBlocks() != null;
+  }
+
+  public Long getHighestPriorityLowRedundancyBlocks() {
+    return highestPriorityLowRedundancyBlocks;
+  }
+
   @Override
   public String toString() {
     StringBuilder statsBuilder = new StringBuilder();
@@ -76,8 +95,12 @@ public final class ECBlockGroupStats {
         .append(", BytesInFutureBlockGroups=").append(
             getBytesInFutureBlockGroups())
         .append(", PendingDeletionBlocks=").append(
-            getPendingDeletionBlocks())
-        .append("]");
+            getPendingDeletionBlocks());
+    if (hasHighestPriorityLowRedundancyBlocks()) {
+      statsBuilder.append(", HighestPriorityLowRedundancyBlocks=")
+          .append(getHighestPriorityLowRedundancyBlocks());
+    }
+    statsBuilder.append("]");
     return statsBuilder.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
index 49aaded..c210003 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
@@ -35,17 +35,29 @@ public final class ReplicatedBlockStats {
   private final long missingReplicationOneBlocks;
   private final long bytesInFutureBlocks;
   private final long pendingDeletionBlocks;
+  private final Long highestPriorityLowRedundancyBlocks;
 
   public ReplicatedBlockStats(long lowRedundancyBlocks,
       long corruptBlocks, long missingBlocks,
       long missingReplicationOneBlocks, long bytesInFutureBlocks,
       long pendingDeletionBlocks) {
+    this(lowRedundancyBlocks, corruptBlocks, missingBlocks,
+        missingReplicationOneBlocks, bytesInFutureBlocks, pendingDeletionBlocks,
+        null);
+  }
+
+  public ReplicatedBlockStats(long lowRedundancyBlocks,
+      long corruptBlocks, long missingBlocks,
+      long missingReplicationOneBlocks, long bytesInFutureBlocks,
+      long pendingDeletionBlocks, Long highestPriorityLowRedundancyBlocks) {
     this.lowRedundancyBlocks = lowRedundancyBlocks;
     this.corruptBlocks = corruptBlocks;
     this.missingBlocks = missingBlocks;
     this.missingReplicationOneBlocks = missingReplicationOneBlocks;
     this.bytesInFutureBlocks = bytesInFutureBlocks;
     this.pendingDeletionBlocks = pendingDeletionBlocks;
+    this.highestPriorityLowRedundancyBlocks
+        = highestPriorityLowRedundancyBlocks;
   }
 
   public long getLowRedundancyBlocks() {
@@ -72,6 +84,14 @@ public final class ReplicatedBlockStats {
     return pendingDeletionBlocks;
   }
 
+  public boolean hasHighestPriorityLowRedundancyBlocks() {
+    return getHighestPriorityLowRedundancyBlocks() != null;
+  }
+
+  public Long getHighestPriorityLowRedundancyBlocks(){
+    return highestPriorityLowRedundancyBlocks;
+  }
+
   @Override
   public String toString() {
     StringBuilder statsBuilder = new StringBuilder();
@@ -83,8 +103,12 @@ public final class ReplicatedBlockStats {
             getMissingReplicationOneBlocks())
         .append(", BytesInFutureBlocks=").append(getBytesInFutureBlocks())
         .append(", PendingDeletionBlocks=").append(
-            getPendingDeletionBlocks())
-        .append("]");
+            getPendingDeletionBlocks());
+    if (hasHighestPriorityLowRedundancyBlocks()) {
+        statsBuilder.append(", HighestPriorityLowRedundancyBlocks=").append(
+            getHighestPriorityLowRedundancyBlocks());
+    }
+    statsBuilder.append("]");
     return statsBuilder.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 490ccb4..4a5a493 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -1990,6 +1990,13 @@ public class PBHelperClient {
 
   public static ReplicatedBlockStats convert(
       GetFsReplicatedBlockStatsResponseProto res) {
+    if (res.hasHighestPrioLowRedundancyBlocks()) {
+      return new ReplicatedBlockStats(res.getLowRedundancy(),
+          res.getCorruptBlocks(), res.getMissingBlocks(),
+          res.getMissingReplOneBlocks(), res.getBlocksInFuture(),
+          res.getPendingDeletionBlocks(),
+          res.getHighestPrioLowRedundancyBlocks());
+    }
     return new ReplicatedBlockStats(res.getLowRedundancy(),
         res.getCorruptBlocks(), res.getMissingBlocks(),
         res.getMissingReplOneBlocks(), res.getBlocksInFuture(),
@@ -1998,6 +2005,12 @@ public class PBHelperClient {
 
   public static ECBlockGroupStats convert(
       GetFsECBlockGroupStatsResponseProto res) {
+    if (res.hasHighestPrioLowRedundancyBlocks()) {
+      return new ECBlockGroupStats(res.getLowRedundancy(),
+          res.getCorruptBlocks(), res.getMissingBlocks(),
+          res.getBlocksInFuture(), res.getPendingDeletionBlocks(),
+          res.getHighestPrioLowRedundancyBlocks());
+    }
     return new ECBlockGroupStats(res.getLowRedundancy(),
         res.getCorruptBlocks(), res.getMissingBlocks(),
         res.getBlocksInFuture(), res.getPendingDeletionBlocks());
@@ -2432,6 +2445,10 @@ public class PBHelperClient {
         replicatedBlockStats.getBytesInFutureBlocks());
     result.setPendingDeletionBlocks(
         replicatedBlockStats.getPendingDeletionBlocks());
+    if (replicatedBlockStats.hasHighestPriorityLowRedundancyBlocks()) {
+      result.setHighestPrioLowRedundancyBlocks(
+          replicatedBlockStats.getHighestPriorityLowRedundancyBlocks());
+    }
     return result.build();
   }
 
@@ -2447,6 +2464,10 @@ public class PBHelperClient {
         ecBlockGroupStats.getBytesInFutureBlockGroups());
     result.setPendingDeletionBlocks(
         ecBlockGroupStats.getPendingDeletionBlocks());
+    if (ecBlockGroupStats.hasHighestPriorityLowRedundancyBlocks()) {
+      result.setHighestPrioLowRedundancyBlocks(
+          ecBlockGroupStats.getHighestPriorityLowRedundancyBlocks());
+    }
     return result.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index e51aeda..ae4c93e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -348,6 +348,8 @@ message GetFsReplicatedBlockStatsResponseProto {
   required uint64 missing_repl_one_blocks = 4;
   required uint64 blocks_in_future = 5;
   required uint64 pending_deletion_blocks = 6;
+  optional uint64 highest_prio_low_redundancy_blocks = 7;
+
 }
 
 message GetFsECBlockGroupStatsRequestProto { // no input paramters
@@ -359,6 +361,7 @@ message GetFsECBlockGroupStatsResponseProto {
   required uint64 missing_blocks = 3;
   required uint64 blocks_in_future = 4;
   required uint64 pending_deletion_blocks = 5;
+  optional uint64 highest_prio_low_redundancy_blocks = 6;
 }
 
 enum DatanodeReportTypeProto {  // type of the datanode report

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
index 4d22ae7..e8ebf0d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
@@ -321,6 +321,16 @@ public class NamenodeBeanMetrics
   }
 
   @Override
+  public long getHighestPriorityLowRedundancyReplicatedBlocks() {
+    return 0;
+  }
+
+  @Override
+  public long getHighestPriorityLowRedundancyECBlocks() {
+    return 0;
+  }
+
+  @Override
   public String getCorruptFiles() {
     return "N/A";
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
index d2b2d50..480b232 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
@@ -185,12 +185,25 @@ public class ErasureCoding {
     long missingBlockGroups = 0;
     long bytesInFutureBlockGroups = 0;
     long pendingDeletionBlocks = 0;
+    long highestPriorityLowRedundancyBlocks = 0;
+    boolean hasHighestPriorityLowRedundancyBlocks = false;
+
     for (ECBlockGroupStats stats : allStats.values()) {
       lowRedundancyBlockGroups += stats.getLowRedundancyBlockGroups();
       corruptBlockGroups += stats.getCorruptBlockGroups();
       missingBlockGroups += stats.getMissingBlockGroups();
       bytesInFutureBlockGroups += stats.getBytesInFutureBlockGroups();
       pendingDeletionBlocks += stats.getPendingDeletionBlocks();
+      if (stats.hasHighestPriorityLowRedundancyBlocks()) {
+        hasHighestPriorityLowRedundancyBlocks = true;
+        highestPriorityLowRedundancyBlocks +=
+            stats.getHighestPriorityLowRedundancyBlocks();
+      }
+    }
+    if (hasHighestPriorityLowRedundancyBlocks) {
+      return new ECBlockGroupStats(lowRedundancyBlockGroups, corruptBlockGroups,
+          missingBlockGroups, bytesInFutureBlockGroups, pendingDeletionBlocks,
+          highestPriorityLowRedundancyBlocks);
     }
     return new ECBlockGroupStats(lowRedundancyBlockGroups, corruptBlockGroups,
         missingBlockGroups, bytesInFutureBlockGroups, pendingDeletionBlocks);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/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 72ea1c0..bac89bf 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
@@ -4428,6 +4428,14 @@ public class BlockManager implements BlockStatsMXBean {
     return this.neededReconstruction.getCorruptReplicationOneBlockSize();
   }
 
+  public long getHighestPriorityReplicatedBlockCount(){
+    return this.neededReconstruction.getHighestPriorityReplicatedBlockCount();
+  }
+
+  public long getHighestPriorityECBlockCount(){
+    return this.neededReconstruction.getHighestPriorityECBlockCount();
+  }
+
   public BlockInfo addBlockCollection(BlockInfo block,
       BlockCollection bc) {
     return blocksMap.addBlockCollection(block, bc);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
index e3f228d..40ea980 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
@@ -92,6 +92,10 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
   private final LongAdder corruptReplicationOneBlocks = new LongAdder();
   private final LongAdder lowRedundancyECBlockGroups = new LongAdder();
   private final LongAdder corruptECBlockGroups = new LongAdder();
+  private final LongAdder highestPriorityLowRedundancyReplicatedBlocks
+      = new LongAdder();
+  private final LongAdder highestPriorityLowRedundancyECBlocks
+      = new LongAdder();
 
   /** Create an object. */
   LowRedundancyBlocks() {
@@ -162,6 +166,18 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
     return corruptReplicationOneBlocks.longValue();
   }
 
+  /** Return the number of under replicated blocks
+   * with the highest priority to recover */
+  long getHighestPriorityReplicatedBlockCount() {
+    return highestPriorityLowRedundancyReplicatedBlocks.longValue();
+  }
+
+  /** Return the number of under replicated EC blocks
+   * with the highest priority to recover */
+  long getHighestPriorityECBlockCount() {
+    return highestPriorityLowRedundancyECBlocks.longValue();
+  }
+
   /**
    *  Return low redundancy striped blocks excluding corrupt blocks.
    */
@@ -300,6 +316,9 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS) {
         corruptECBlockGroups.increment();
       }
+      if (priLevel == QUEUE_HIGHEST_PRIORITY) {
+        highestPriorityLowRedundancyECBlocks.increment();
+      }
     } else {
       lowRedundancyBlocks.increment();
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS) {
@@ -308,6 +327,9 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
           corruptReplicationOneBlocks.increment();
         }
       }
+      if (priLevel == QUEUE_HIGHEST_PRIORITY) {
+        highestPriorityLowRedundancyReplicatedBlocks.increment();
+      }
     }
   }
 
@@ -380,6 +402,9 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS) {
         corruptECBlockGroups.decrement();
       }
+      if (priLevel == QUEUE_HIGHEST_PRIORITY) {
+        highestPriorityLowRedundancyECBlocks.decrement();
+      }
     } else {
       lowRedundancyBlocks.decrement();
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS) {
@@ -391,6 +416,9 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
                   "should be non-negative";
         }
       }
+      if (priLevel == QUEUE_HIGHEST_PRIORITY) {
+        highestPriorityLowRedundancyReplicatedBlocks.decrement();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/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 8c95f7d..5ef07b7 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
@@ -4218,7 +4218,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new ReplicatedBlockStats(getLowRedundancyReplicatedBlocks(),
         getCorruptReplicatedBlocks(), getMissingReplicatedBlocks(),
         getMissingReplicationOneBlocks(), getBytesInFutureReplicatedBlocks(),
-        getPendingDeletionReplicatedBlocks());
+        getPendingDeletionReplicatedBlocks(),
+        getHighestPriorityLowRedundancyReplicatedBlocks());
   }
 
   /**
@@ -4230,7 +4231,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   ECBlockGroupStats getECBlockGroupStats() {
     return new ECBlockGroupStats(getLowRedundancyECBlockGroups(),
         getCorruptECBlockGroups(), getMissingECBlockGroups(),
-        getBytesInFutureECBlockGroups(), getPendingDeletionECBlocks());
+        getBytesInFutureECBlockGroups(), getPendingDeletionECBlocks(),
+        getHighestPriorityLowRedundancyECBlocks());
   }
 
   @Override // FSNamesystemMBean
@@ -4838,6 +4840,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // ReplicatedBlocksMBean
+  @Metric({"HighestPriorityLowRedundancyReplicatedBlocks", "Number of " +
+      "replicated blocks which have the highest risk of loss."})
+  public long getHighestPriorityLowRedundancyReplicatedBlocks() {
+    return blockManager.getHighestPriorityReplicatedBlockCount();
+  }
+
+  @Override // ReplicatedBlocksMBean
+  @Metric({"HighestPriorityLowRedundancyECBlocks", "Number of erasure coded " +
+      "blocks which have the highest risk of loss."})
+  public long getHighestPriorityLowRedundancyECBlocks() {
+    return blockManager.getHighestPriorityECBlockCount();
+  }
+
+  @Override // ReplicatedBlocksMBean
   @Metric({"BytesInFutureReplicatedBlocks", "Total bytes in replicated " +
       "blocks with future generation stamp"})
   public long getBytesInFutureReplicatedBlocks() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
index e4ed3a9..5c7bbbb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
@@ -163,6 +163,24 @@ public interface NameNodeMXBean {
   public long getNumberOfMissingBlocksWithReplicationFactorOne();
 
   /**
+   * Gets the total number of replicated low redundancy blocks on the cluster
+   * with the highest risk of loss.
+   *
+   * @return the total number of low redundancy blocks on the cluster
+   * with the highest risk of loss.
+   */
+  public long getHighestPriorityLowRedundancyReplicatedBlocks();
+
+  /**
+   * Gets the total number of erasure coded low redundancy blocks on the cluster
+   * with the highest risk of loss
+   *
+   * @return the total number of low redundancy blocks on the cluster
+   * with the highest risk of loss
+   */
+  public long getHighestPriorityLowRedundancyECBlocks();
+
+  /**
    * Gets the total number of snapshottable dirs in the system.
    *
    * @return the total number of snapshottable dirs in the system

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 99a8e3e..c5571be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -549,6 +549,11 @@ public class DFSAdmin extends FsShell {
         replicatedBlockStats.getMissingReplicaBlocks());
     System.out.println("\tMissing blocks (with replication factor 1): " +
         replicatedBlockStats.getMissingReplicationOneBlocks());
+    if (replicatedBlockStats.hasHighestPriorityLowRedundancyBlocks()) {
+      System.out.println("\tLow redundancy blocks with highest priority " +
+          "to recover: " +
+          replicatedBlockStats.getHighestPriorityLowRedundancyBlocks());
+    }
     System.out.println("\tPending deletion blocks: " +
         replicatedBlockStats.getPendingDeletionBlocks());
 
@@ -561,6 +566,11 @@ public class DFSAdmin extends FsShell {
         ecBlockGroupStats.getCorruptBlockGroups());
     System.out.println("\tMissing block groups: " +
         ecBlockGroupStats.getMissingBlockGroups());
+    if (ecBlockGroupStats.hasHighestPriorityLowRedundancyBlocks()) {
+      System.out.println("\tLow redundancy blocks with highest priority " +
+          "to recover: " +
+          ecBlockGroupStats.getHighestPriorityLowRedundancyBlocks());
+    }
     System.out.println("\tPending deletion blocks: " +
         ecBlockGroupStats.getPendingDeletionBlocks());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
index 97a5a6e..cf40c39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
@@ -63,7 +63,8 @@ public class TestLowRedundancyBlockQueues {
   private void verifyBlockStats(LowRedundancyBlocks queues,
       int lowRedundancyReplicaCount, int corruptReplicaCount,
       int corruptReplicationOneCount, int lowRedundancyStripedCount,
-      int corruptStripedCount) {
+      int corruptStripedCount, int highestPriorityReplicatedBlockCount,
+      int highestPriorityECBlockCount) {
     assertEquals("Low redundancy replica count incorrect!",
         lowRedundancyReplicaCount, queues.getLowRedundancyBlocks());
     assertEquals("Corrupt replica count incorrect!",
@@ -81,6 +82,14 @@ public class TestLowRedundancyBlockQueues {
     assertEquals("LowRedundancyBlocks queue size incorrect!",
         (lowRedundancyReplicaCount + corruptReplicaCount +
         lowRedundancyStripedCount + corruptStripedCount), queues.size());
+    assertEquals("Highest priority replicated low redundancy " +
+            "blocks count is incorrect!",
+        highestPriorityReplicatedBlockCount,
+        queues.getHighestPriorityReplicatedBlockCount());
+    assertEquals("Highest priority erasure coded low redundancy " +
+            "blocks count is incorrect!",
+        highestPriorityECBlockCount,
+        queues.getHighestPriorityECBlockCount());
   }
 
   /**
@@ -100,42 +109,46 @@ public class TestLowRedundancyBlockQueues {
     // Add a block with a single entry
     assertAdded(queues, block1, 1, 0, 3);
     assertInLevel(queues, block1, LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY);
-    verifyBlockStats(queues, 1, 0, 0, 0, 0);
+    verifyBlockStats(queues, 1, 0, 0, 0, 0, 1, 0);
 
     // Repeated additions fail
     assertFalse(queues.add(block1, 1, 0, 0, 3));
-    verifyBlockStats(queues, 1, 0, 0, 0, 0);
+    verifyBlockStats(queues, 1, 0, 0, 0, 0, 1, 0);
 
     // Add a second block with two replicas
     assertAdded(queues, block2, 2, 0, 3);
     assertInLevel(queues, block2, LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY);
-    verifyBlockStats(queues, 2, 0, 0, 0, 0);
+    verifyBlockStats(queues, 2, 0, 0, 0, 0, 1, 0);
 
     // Now try to add a block that is corrupt
     assertAdded(queues, block_corrupt, 0, 0, 3);
     assertInLevel(queues, block_corrupt,
                   LowRedundancyBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
-    verifyBlockStats(queues, 2, 1, 0, 0, 0);
+    verifyBlockStats(queues, 2, 1, 0, 0, 0, 1, 0);
 
     // Insert a very insufficiently redundancy block
     assertAdded(queues, block_very_low_redundancy, 4, 0, 25);
     assertInLevel(queues, block_very_low_redundancy,
                   LowRedundancyBlocks.QUEUE_VERY_LOW_REDUNDANCY);
-    verifyBlockStats(queues, 3, 1, 0, 0, 0);
+    verifyBlockStats(queues, 3, 1, 0, 0, 0, 1, 0);
 
     // Insert a corrupt block with replication factor 1
     assertAdded(queues, block_corrupt_repl_one, 0, 0, 1);
-    verifyBlockStats(queues, 3, 2, 1, 0, 0);
+    verifyBlockStats(queues, 3, 2, 1, 0, 0, 1, 0);
 
     // Bump up the expected count for corrupt replica one block from 1 to 3
     queues.update(block_corrupt_repl_one, 0, 0, 0, 3, 0, 2);
-    verifyBlockStats(queues, 3, 2, 0, 0, 0);
+    verifyBlockStats(queues, 3, 2, 0, 0, 0, 1, 0);
 
     // Reduce the expected replicas to 1
     queues.update(block_corrupt, 0, 0, 0, 1, 0, -2);
-    verifyBlockStats(queues, 3, 2, 1, 0, 0);
+    verifyBlockStats(queues, 3, 2, 1, 0, 0, 1, 0);
     queues.update(block_very_low_redundancy, 0, 0, 0, 1, -4, -24);
-    verifyBlockStats(queues, 2, 3, 2, 0, 0);
+    verifyBlockStats(queues, 2, 3, 2, 0, 0, 1, 0);
+
+    // Reduce the expected replicas to 1 for block1
+    queues.update(block1, 1, 0, 0, 1, 0, 0);
+    verifyBlockStats(queues, 2, 3, 2, 0, 0, 0, 0);
   }
 
   @Test
@@ -145,12 +158,12 @@ public class TestLowRedundancyBlockQueues {
     assertAdded(queues, corruptBlock, 0, 0, 3);
     assertInLevel(queues, corruptBlock,
         LowRedundancyBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
-    verifyBlockStats(queues, 0, 1, 0, 0, 0);
+    verifyBlockStats(queues, 0, 1, 0, 0, 0, 0, 0);
 
     // Remove with wrong priority
     queues.remove(corruptBlock, LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY);
     // Verify the number of corrupt block is decremented
-    verifyBlockStats(queues, 0, 0, 0, 0, 0);
+    verifyBlockStats(queues, 0, 0, 0, 0, 0, 0, 0);
   }
 
   @Test
@@ -186,17 +199,17 @@ public class TestLowRedundancyBlockQueues {
         assertInLevel(queues, block,
             LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY);
       }
-      verifyBlockStats(queues, 0, 0, 0, numUR, 0);
+      verifyBlockStats(queues, 0, 0, 0, numUR, 0, 0, 1);
     }
 
     // add a corrupted block
     BlockInfo block_corrupt = genStripedBlockInfo(-10, numBytes);
     assertEquals(numCorrupt, queues.getCorruptBlockSize());
-    verifyBlockStats(queues, 0, 0, 0, numUR, numCorrupt);
+    verifyBlockStats(queues, 0, 0, 0, numUR, numCorrupt, 0, 1);
 
     assertAdded(queues, block_corrupt, dataBlkNum - 1, 0, groupSize);
     numCorrupt++;
-    verifyBlockStats(queues, 0, 0, 0, numUR, numCorrupt);
+    verifyBlockStats(queues, 0, 0, 0, numUR, numCorrupt, 0, 1);
 
     assertInLevel(queues, block_corrupt,
         LowRedundancyBlocks.QUEUE_WITH_CORRUPT_BLOCKS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/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 05cf2ea..57a1b01 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
@@ -412,10 +412,12 @@ public class TestNameNodeMetrics {
     // Verify replica metrics
     assertGauge("LowRedundancyReplicatedBlocks", 0L, rb);
     assertGauge("CorruptReplicatedBlocks", 0L, rb);
+    assertGauge("HighestPriorityLowRedundancyReplicatedBlocks", 0L, rb);
 
     // Verify striped block groups metrics
     assertGauge("LowRedundancyECBlockGroups", 0L, rb);
     assertGauge("CorruptECBlockGroups", 0L, rb);
+    assertGauge("HighestPriorityLowRedundancyECBlocks", 0L, rb);
   }
 
   /**
@@ -492,9 +494,11 @@ public class TestNameNodeMetrics {
     // Verify replicated blocks metrics
     assertGauge("LowRedundancyReplicatedBlocks", 1L, rb);
     assertGauge("CorruptReplicatedBlocks", 1L, rb);
+    assertGauge("HighestPriorityLowRedundancyReplicatedBlocks", 1L, rb);
     // Verify striped blocks metrics
     assertGauge("LowRedundancyECBlockGroups", 0L, rb);
     assertGauge("CorruptECBlockGroups", 0L, rb);
+    assertGauge("HighestPriorityLowRedundancyECBlocks", 0L, rb);
 
     verifyAggregatedMetricsTally();
 
@@ -517,9 +521,11 @@ public class TestNameNodeMetrics {
     // Verify replicated blocks metrics
     assertGauge("LowRedundancyReplicatedBlocks", 0L, rb);
     assertGauge("CorruptReplicatedBlocks", 0L, rb);
+    assertGauge("HighestPriorityLowRedundancyReplicatedBlocks", 0L, rb);
     // Verify striped blocks metrics
     assertGauge("LowRedundancyECBlockGroups", 0L, rb);
     assertGauge("CorruptECBlockGroups", 0L, rb);
+    assertGauge("HighestPriorityLowRedundancyECBlocks", 0L, rb);
 
     verifyAggregatedMetricsTally();
 
@@ -580,9 +586,11 @@ public class TestNameNodeMetrics {
     // Verify replica metrics
     assertGauge("LowRedundancyReplicatedBlocks", 0L, rb);
     assertGauge("CorruptReplicatedBlocks", 0L, rb);
+    assertGauge("HighestPriorityLowRedundancyReplicatedBlocks", 0L, rb);
     // Verify striped block groups metrics
     assertGauge("LowRedundancyECBlockGroups", 1L, rb);
     assertGauge("CorruptECBlockGroups", 1L, rb);
+    assertGauge("HighestPriorityLowRedundancyECBlocks", 1L, rb);
 
     verifyAggregatedMetricsTally();
 
@@ -602,9 +610,11 @@ public class TestNameNodeMetrics {
     // Verify replicated blocks metrics
     assertGauge("LowRedundancyReplicatedBlocks", 0L, rb);
     assertGauge("CorruptReplicatedBlocks", 0L, rb);
+    assertGauge("HighestPriorityLowRedundancyReplicatedBlocks", 0L, rb);
     // Verify striped blocks metrics
     assertGauge("LowRedundancyECBlockGroups", 0L, rb);
     assertGauge("CorruptECBlockGroups", 0L, rb);
+    assertGauge("HighestPriorityLowRedundancyECBlocks", 0L, rb);
 
     verifyAggregatedMetricsTally();
 
@@ -666,6 +676,8 @@ public class TestNameNodeMetrics {
     assertGauge("UnderReplicatedBlocks", 1L, rb);
     assertGauge("MissingBlocks", 1L, rb);
     assertGauge("MissingReplOneBlocks", 1L, rb);
+    assertGauge("HighestPriorityLowRedundancyReplicatedBlocks", 0L, rb);
+    assertGauge("HighestPriorityLowRedundancyECBlocks", 0L, rb);
     fs.delete(file, true);
     waitForDnMetricValue(NS_METRICS, "UnderReplicatedBlocks", 0L);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9499df7b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
index badb81b..af15c4c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
@@ -579,7 +579,7 @@ public class TestDFSAdmin {
       // Verify report command for all counts to be zero
       resetStream();
       assertEquals(0, ToolRunner.run(dfsAdmin, new String[] {"-report"}));
-      verifyNodesAndCorruptBlocks(numDn, numDn, 0, 0, client);
+      verifyNodesAndCorruptBlocks(numDn, numDn, 0, 0, client, 0L, 0L);
 
       final short replFactor = 1;
       final long fileLength = 512L;
@@ -614,7 +614,7 @@ public class TestDFSAdmin {
       // Verify report command for all counts to be zero
       resetStream();
       assertEquals(0, ToolRunner.run(dfsAdmin, new String[] {"-report"}));
-      verifyNodesAndCorruptBlocks(numDn, numDn, 0, 0, client);
+      verifyNodesAndCorruptBlocks(numDn, numDn, 0, 0, client, 0L, 0L);
 
       // Choose a DataNode to shutdown
       final List<DataNode> datanodes = miniCluster.getDataNodes();
@@ -636,7 +636,7 @@ public class TestDFSAdmin {
 
       // Verify report command to show dead DataNode
       assertEquals(0, ToolRunner.run(dfsAdmin, new String[] {"-report"}));
-      verifyNodesAndCorruptBlocks(numDn, numDn - 1, 0, 0, client);
+      verifyNodesAndCorruptBlocks(numDn, numDn - 1, 0, 0, client, 0L, 1L);
 
       // Corrupt the replicated block
       final int blockFilesCorrupted = miniCluster
@@ -664,7 +664,7 @@ public class TestDFSAdmin {
       // verify report command for corrupt replicated block
       resetStream();
       assertEquals(0, ToolRunner.run(dfsAdmin, new String[] {"-report"}));
-      verifyNodesAndCorruptBlocks(numDn, numDn - 1, 1, 0, client);
+      verifyNodesAndCorruptBlocks(numDn, numDn - 1, 1, 0, client, 0L, 1L);
 
       lbs = miniCluster.getFileSystem().getClient().
           getNamenode().getBlockLocations(
@@ -689,7 +689,7 @@ public class TestDFSAdmin {
       // and EC block group
       resetStream();
       assertEquals(0, ToolRunner.run(dfsAdmin, new String[] {"-report"}));
-      verifyNodesAndCorruptBlocks(numDn, numDn - 1, 1, 1, client);
+      verifyNodesAndCorruptBlocks(numDn, numDn - 1, 1, 1, client, 0L, 0L);
     }
   }
 
@@ -834,7 +834,10 @@ public class TestDFSAdmin {
       final int numLiveDn,
       final int numCorruptBlocks,
       final int numCorruptECBlockGroups,
-      final DFSClient client) throws IOException {
+      final DFSClient client,
+      final Long highestPriorityLowRedundancyReplicatedBlocks,
+      final Long highestPriorityLowRedundancyECBlocks)
+      throws IOException {
 
     /* init vars */
     final String outStr = scanIntoString(out);
@@ -847,12 +850,23 @@ public class TestDFSAdmin {
     final String expectedCorruptedECBlockGroupsStr = String.format(
         "Block groups with corrupt internal blocks: %d",
         numCorruptECBlockGroups);
+    final String highestPriorityLowRedundancyReplicatedBlocksStr
+        = String.format(
+        "\tLow redundancy blocks with highest priority " +
+            "to recover: %d",
+        highestPriorityLowRedundancyReplicatedBlocks);
+    final String highestPriorityLowRedundancyECBlocksStr = String.format(
+        "\tLow redundancy blocks with highest priority " +
+            "to recover: %d",
+        highestPriorityLowRedundancyReplicatedBlocks);
 
     // verify nodes and corrupt blocks
     assertThat(outStr, is(allOf(
         containsString(expectedLiveNodesStr),
         containsString(expectedCorruptedBlocksStr),
-        containsString(expectedCorruptedECBlockGroupsStr))));
+        containsString(expectedCorruptedECBlockGroupsStr),
+        containsString(highestPriorityLowRedundancyReplicatedBlocksStr),
+        containsString(highestPriorityLowRedundancyECBlocksStr))));
 
     assertEquals(
         numDn,
@@ -867,8 +881,12 @@ public class TestDFSAdmin {
         client.getCorruptBlocksCount());
     assertEquals(numCorruptBlocks, client.getNamenode()
         .getReplicatedBlockStats().getCorruptBlocks());
+    assertEquals(highestPriorityLowRedundancyReplicatedBlocks, client.getNamenode()
+        .getReplicatedBlockStats().getHighestPriorityLowRedundancyBlocks());
     assertEquals(numCorruptECBlockGroups, client.getNamenode()
         .getECBlockGroupStats().getCorruptBlockGroups());
+    assertEquals(highestPriorityLowRedundancyECBlocks, client.getNamenode()
+        .getECBlockGroupStats().getHighestPriorityLowRedundancyBlocks());
   }
 
   @Test


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


[50/50] [abbrv] hadoop git commit: Fixing issue due to commit 2b2399d6 after rebase onto trunk.

Posted by tm...@apache.org.
Fixing issue due to commit 2b2399d6 after rebase onto trunk.


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

Branch: refs/heads/HADOOP-15407
Commit: 7262485aef633521024b06cf93185f2380082320
Parents: ee6866d
Author: Thomas Marquardt <tm...@microsoft.com>
Authored: Sat Aug 11 05:32:30 2018 +0000
Committer: Thomas Marquardt <tm...@microsoft.com>
Committed: Sat Aug 11 05:32:30 2018 +0000

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7262485a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
index b0a30a0..2cb517b 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;


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


[16/50] [abbrv] hadoop git commit: YARN-8633. Update DataTables version in yarn-common in line with JQuery 3 upgrade. Contributed by Akhil PB.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/js/jquery.dataTables.min.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/js/jquery.dataTables.min.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/js/jquery.dataTables.min.js
new file mode 100644
index 0000000..85dd817
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/js/jquery.dataTables.min.js
@@ -0,0 +1,160 @@
+/*! DataTables 1.10.7
+ * ©2008-2015 SpryMedia Ltd - datatables.net/license
+ */
+(function(Ea,Q,k){var P=function(h){function W(a){var b,c,e={};h.each(a,function(d){if((b=d.match(/^([^A-Z]+?)([A-Z])/))&&-1!=="a aa ai ao as b fn i m o s ".indexOf(b[1]+" "))c=d.replace(b[0],b[2].toLowerCase()),e[c]=d,"o"===b[1]&&W(a[d])});a._hungarianMap=e}function H(a,b,c){a._hungarianMap||W(a);var e;h.each(b,function(d){e=a._hungarianMap[d];if(e!==k&&(c||b[e]===k))"o"===e.charAt(0)?(b[e]||(b[e]={}),h.extend(!0,b[e],b[d]),H(a[e],b[e],c)):b[e]=b[d]})}function P(a){var b=m.defaults.oLanguage,c=a.sZeroRecords;
+!a.sEmptyTable&&(c&&"No data available in table"===b.sEmptyTable)&&E(a,a,"sZeroRecords","sEmptyTable");!a.sLoadingRecords&&(c&&"Loading..."===b.sLoadingRecords)&&E(a,a,"sZeroRecords","sLoadingRecords");a.sInfoThousands&&(a.sThousands=a.sInfoThousands);(a=a.sDecimal)&&db(a)}function eb(a){A(a,"ordering","bSort");A(a,"orderMulti","bSortMulti");A(a,"orderClasses","bSortClasses");A(a,"orderCellsTop","bSortCellsTop");A(a,"order","aaSorting");A(a,"orderFixed","aaSortingFixed");A(a,"paging","bPaginate");
+A(a,"pagingType","sPaginationType");A(a,"pageLength","iDisplayLength");A(a,"searching","bFilter");if(a=a.aoSearchCols)for(var b=0,c=a.length;b<c;b++)a[b]&&H(m.models.oSearch,a[b])}function fb(a){A(a,"orderable","bSortable");A(a,"orderData","aDataSort");A(a,"orderSequence","asSorting");A(a,"orderDataType","sortDataType");var b=a.aDataSort;b&&!h.isArray(b)&&(a.aDataSort=[b])}function gb(a){var a=a.oBrowser,b=h("<div/>").css({position:"absolute",top:0,left:0,height:1,width:1,overflow:"hidden"}).append(h("<div/>").css({position:"absolute",
+top:1,left:1,width:100,overflow:"scroll"}).append(h('<div class="test"/>').css({width:"100%",height:10}))).appendTo("body"),c=b.find(".test");a.bScrollOversize=100===c[0].offsetWidth;a.bScrollbarLeft=1!==Math.round(c.offset().left);b.remove()}function hb(a,b,c,e,d,f){var g,j=!1;c!==k&&(g=c,j=!0);for(;e!==d;)a.hasOwnProperty(e)&&(g=j?b(g,a[e],e,a):a[e],j=!0,e+=f);return g}function Fa(a,b){var c=m.defaults.column,e=a.aoColumns.length,c=h.extend({},m.models.oColumn,c,{nTh:b?b:Q.createElement("th"),sTitle:c.sTitle?
+c.sTitle:b?b.innerHTML:"",aDataSort:c.aDataSort?c.aDataSort:[e],mData:c.mData?c.mData:e,idx:e});a.aoColumns.push(c);c=a.aoPreSearchCols;c[e]=h.extend({},m.models.oSearch,c[e]);ka(a,e,h(b).data())}function ka(a,b,c){var b=a.aoColumns[b],e=a.oClasses,d=h(b.nTh);if(!b.sWidthOrig){b.sWidthOrig=d.attr("width")||null;var f=(d.attr("style")||"").match(/width:\s*(\d+[pxem%]+)/);f&&(b.sWidthOrig=f[1])}c!==k&&null!==c&&(fb(c),H(m.defaults.column,c),c.mDataProp!==k&&!c.mData&&(c.mData=c.mDataProp),c.sType&&
+(b._sManualType=c.sType),c.className&&!c.sClass&&(c.sClass=c.className),h.extend(b,c),E(b,c,"sWidth","sWidthOrig"),c.iDataSort!==k&&(b.aDataSort=[c.iDataSort]),E(b,c,"aDataSort"));var g=b.mData,j=R(g),i=b.mRender?R(b.mRender):null,c=function(a){return"string"===typeof a&&-1!==a.indexOf("@")};b._bAttrSrc=h.isPlainObject(g)&&(c(g.sort)||c(g.type)||c(g.filter));b.fnGetData=function(a,b,c){var e=j(a,b,k,c);return i&&b?i(e,b,a,c):e};b.fnSetData=function(a,b,c){return S(g)(a,b,c)};"number"!==typeof g&&
+(a._rowReadObject=!0);a.oFeatures.bSort||(b.bSortable=!1,d.addClass(e.sSortableNone));a=-1!==h.inArray("asc",b.asSorting);c=-1!==h.inArray("desc",b.asSorting);!b.bSortable||!a&&!c?(b.sSortingClass=e.sSortableNone,b.sSortingClassJUI=""):a&&!c?(b.sSortingClass=e.sSortableAsc,b.sSortingClassJUI=e.sSortJUIAscAllowed):!a&&c?(b.sSortingClass=e.sSortableDesc,b.sSortingClassJUI=e.sSortJUIDescAllowed):(b.sSortingClass=e.sSortable,b.sSortingClassJUI=e.sSortJUI)}function X(a){if(!1!==a.oFeatures.bAutoWidth){var b=
+a.aoColumns;Ga(a);for(var c=0,e=b.length;c<e;c++)b[c].nTh.style.width=b[c].sWidth}b=a.oScroll;(""!==b.sY||""!==b.sX)&&Y(a);w(a,null,"column-sizing",[a])}function la(a,b){var c=Z(a,"bVisible");return"number"===typeof c[b]?c[b]:null}function $(a,b){var c=Z(a,"bVisible"),c=h.inArray(b,c);return-1!==c?c:null}function aa(a){return Z(a,"bVisible").length}function Z(a,b){var c=[];h.map(a.aoColumns,function(a,d){a[b]&&c.push(d)});return c}function Ha(a){var b=a.aoColumns,c=a.aoData,e=m.ext.type.detect,d,
+f,g,j,i,h,l,q,n;d=0;for(f=b.length;d<f;d++)if(l=b[d],n=[],!l.sType&&l._sManualType)l.sType=l._sManualType;else if(!l.sType){g=0;for(j=e.length;g<j;g++){i=0;for(h=c.length;i<h;i++){n[i]===k&&(n[i]=x(a,i,d,"type"));q=e[g](n[i],a);if(!q&&g!==e.length-1)break;if("html"===q)break}if(q){l.sType=q;break}}l.sType||(l.sType="string")}}function ib(a,b,c,e){var d,f,g,j,i,o,l=a.aoColumns;if(b)for(d=b.length-1;0<=d;d--){o=b[d];var q=o.targets!==k?o.targets:o.aTargets;h.isArray(q)||(q=[q]);f=0;for(g=q.length;f<
+g;f++)if("number"===typeof q[f]&&0<=q[f]){for(;l.length<=q[f];)Fa(a);e(q[f],o)}else if("number"===typeof q[f]&&0>q[f])e(l.length+q[f],o);else if("string"===typeof q[f]){j=0;for(i=l.length;j<i;j++)("_all"==q[f]||h(l[j].nTh).hasClass(q[f]))&&e(j,o)}}if(c){d=0;for(a=c.length;d<a;d++)e(d,c[d])}}function K(a,b,c,e){var d=a.aoData.length,f=h.extend(!0,{},m.models.oRow,{src:c?"dom":"data"});f._aData=b;a.aoData.push(f);for(var b=a.aoColumns,f=0,g=b.length;f<g;f++)c&&Ia(a,d,f,x(a,d,f)),b[f].sType=null;a.aiDisplayMaster.push(d);
+(c||!a.oFeatures.bDeferRender)&&Ja(a,d,c,e);return d}function ma(a,b){var c;b instanceof h||(b=h(b));return b.map(function(b,d){c=na(a,d);return K(a,c.data,d,c.cells)})}function x(a,b,c,e){var d=a.iDraw,f=a.aoColumns[c],g=a.aoData[b]._aData,j=f.sDefaultContent,c=f.fnGetData(g,e,{settings:a,row:b,col:c});if(c===k)return a.iDrawError!=d&&null===j&&(I(a,0,"Requested unknown parameter "+("function"==typeof f.mData?"{function}":"'"+f.mData+"'")+" for row "+b,4),a.iDrawError=d),j;if((c===g||null===c)&&
+null!==j)c=j;else if("function"===typeof c)return c.call(g);return null===c&&"display"==e?"":c}function Ia(a,b,c,e){a.aoColumns[c].fnSetData(a.aoData[b]._aData,e,{settings:a,row:b,col:c})}function Ka(a){return h.map(a.match(/(\\.|[^\.])+/g),function(a){return a.replace(/\\./g,".")})}function R(a){if(h.isPlainObject(a)){var b={};h.each(a,function(a,c){c&&(b[a]=R(c))});return function(a,c,f,g){var j=b[c]||b._;return j!==k?j(a,c,f,g):a}}if(null===a)return function(a){return a};if("function"===typeof a)return function(b,
+c,f,g){return a(b,c,f,g)};if("string"===typeof a&&(-1!==a.indexOf(".")||-1!==a.indexOf("[")||-1!==a.indexOf("("))){var c=function(a,b,f){var g,j;if(""!==f){j=Ka(f);for(var i=0,h=j.length;i<h;i++){f=j[i].match(ba);g=j[i].match(T);if(f){j[i]=j[i].replace(ba,"");""!==j[i]&&(a=a[j[i]]);g=[];j.splice(0,i+1);j=j.join(".");i=0;for(h=a.length;i<h;i++)g.push(c(a[i],b,j));a=f[0].substring(1,f[0].length-1);a=""===a?g:g.join(a);break}else if(g){j[i]=j[i].replace(T,"");a=a[j[i]]();continue}if(null===a||a[j[i]]===
+k)return k;a=a[j[i]]}}return a};return function(b,d){return c(b,d,a)}}return function(b){return b[a]}}function S(a){if(h.isPlainObject(a))return S(a._);if(null===a)return function(){};if("function"===typeof a)return function(b,e,d){a(b,"set",e,d)};if("string"===typeof a&&(-1!==a.indexOf(".")||-1!==a.indexOf("[")||-1!==a.indexOf("("))){var b=function(a,e,d){var d=Ka(d),f;f=d[d.length-1];for(var g,j,i=0,h=d.length-1;i<h;i++){g=d[i].match(ba);j=d[i].match(T);if(g){d[i]=d[i].replace(ba,"");a[d[i]]=[];
+f=d.slice();f.splice(0,i+1);g=f.join(".");j=0;for(h=e.length;j<h;j++)f={},b(f,e[j],g),a[d[i]].push(f);return}j&&(d[i]=d[i].replace(T,""),a=a[d[i]](e));if(null===a[d[i]]||a[d[i]]===k)a[d[i]]={};a=a[d[i]]}if(f.match(T))a[f.replace(T,"")](e);else a[f.replace(ba,"")]=e};return function(c,e){return b(c,e,a)}}return function(b,e){b[a]=e}}function La(a){return D(a.aoData,"_aData")}function oa(a){a.aoData.length=0;a.aiDisplayMaster.length=0;a.aiDisplay.length=0}function pa(a,b,c){for(var e=-1,d=0,f=a.length;d<
+f;d++)a[d]==b?e=d:a[d]>b&&a[d]--; -1!=e&&c===k&&a.splice(e,1)}function ca(a,b,c,e){var d=a.aoData[b],f,g=function(c,f){for(;c.childNodes.length;)c.removeChild(c.firstChild);c.innerHTML=x(a,b,f,"display")};if("dom"===c||(!c||"auto"===c)&&"dom"===d.src)d._aData=na(a,d,e,e===k?k:d._aData).data;else{var j=d.anCells;if(j)if(e!==k)g(j[e],e);else{c=0;for(f=j.length;c<f;c++)g(j[c],c)}}d._aSortData=null;d._aFilterData=null;g=a.aoColumns;if(e!==k)g[e].sType=null;else{c=0;for(f=g.length;c<f;c++)g[c].sType=null;
+Ma(d)}}function na(a,b,c,e){var d=[],f=b.firstChild,g,j=0,i,o=a.aoColumns,l=a._rowReadObject,e=e||l?{}:[],q=function(a,b){if("string"===typeof a){var c=a.indexOf("@");-1!==c&&(c=a.substring(c+1),S(a)(e,b.getAttribute(c)))}},a=function(a){if(c===k||c===j)g=o[j],i=h.trim(a.innerHTML),g&&g._bAttrSrc?(S(g.mData._)(e,i),q(g.mData.sort,a),q(g.mData.type,a),q(g.mData.filter,a)):l?(g._setter||(g._setter=S(g.mData)),g._setter(e,i)):e[j]=i;j++};if(f)for(;f;){b=f.nodeName.toUpperCase();if("TD"==b||"TH"==b)a(f),
+d.push(f);f=f.nextSibling}else{d=b.anCells;f=0;for(b=d.length;f<b;f++)a(d[f])}return{data:e,cells:d}}function Ja(a,b,c,e){var d=a.aoData[b],f=d._aData,g=[],j,i,h,l,q;if(null===d.nTr){j=c||Q.createElement("tr");d.nTr=j;d.anCells=g;j._DT_RowIndex=b;Ma(d);l=0;for(q=a.aoColumns.length;l<q;l++){h=a.aoColumns[l];i=c?e[l]:Q.createElement(h.sCellType);g.push(i);if(!c||h.mRender||h.mData!==l)i.innerHTML=x(a,b,l,"display");h.sClass&&(i.className+=" "+h.sClass);h.bVisible&&!c?j.appendChild(i):!h.bVisible&&c&&
+i.parentNode.removeChild(i);h.fnCreatedCell&&h.fnCreatedCell.call(a.oInstance,i,x(a,b,l),f,b,l)}w(a,"aoRowCreatedCallback",null,[j,f,b])}d.nTr.setAttribute("role","row")}function Ma(a){var b=a.nTr,c=a._aData;if(b){c.DT_RowId&&(b.id=c.DT_RowId);if(c.DT_RowClass){var e=c.DT_RowClass.split(" ");a.__rowc=a.__rowc?Na(a.__rowc.concat(e)):e;h(b).removeClass(a.__rowc.join(" ")).addClass(c.DT_RowClass)}c.DT_RowAttr&&h(b).attr(c.DT_RowAttr);c.DT_RowData&&h(b).data(c.DT_RowData)}}function jb(a){var b,c,e,d,
+f,g=a.nTHead,j=a.nTFoot,i=0===h("th, td",g).length,o=a.oClasses,l=a.aoColumns;i&&(d=h("<tr/>").appendTo(g));b=0;for(c=l.length;b<c;b++)f=l[b],e=h(f.nTh).addClass(f.sClass),i&&e.appendTo(d),a.oFeatures.bSort&&(e.addClass(f.sSortingClass),!1!==f.bSortable&&(e.attr("tabindex",a.iTabIndex).attr("aria-controls",a.sTableId),Oa(a,f.nTh,b))),f.sTitle!=e.html()&&e.html(f.sTitle),Pa(a,"header")(a,e,f,o);i&&da(a.aoHeader,g);h(g).find(">tr").attr("role","row");h(g).find(">tr>th, >tr>td").addClass(o.sHeaderTH);
+h(j).find(">tr>th, >tr>td").addClass(o.sFooterTH);if(null!==j){a=a.aoFooter[0];b=0;for(c=a.length;b<c;b++)f=l[b],f.nTf=a[b].cell,f.sClass&&h(f.nTf).addClass(f.sClass)}}function ea(a,b,c){var e,d,f,g=[],j=[],i=a.aoColumns.length,o;if(b){c===k&&(c=!1);e=0;for(d=b.length;e<d;e++){g[e]=b[e].slice();g[e].nTr=b[e].nTr;for(f=i-1;0<=f;f--)!a.aoColumns[f].bVisible&&!c&&g[e].splice(f,1);j.push([])}e=0;for(d=g.length;e<d;e++){if(a=g[e].nTr)for(;f=a.firstChild;)a.removeChild(f);f=0;for(b=g[e].length;f<b;f++)if(o=
+i=1,j[e][f]===k){a.appendChild(g[e][f].cell);for(j[e][f]=1;g[e+i]!==k&&g[e][f].cell==g[e+i][f].cell;)j[e+i][f]=1,i++;for(;g[e][f+o]!==k&&g[e][f].cell==g[e][f+o].cell;){for(c=0;c<i;c++)j[e+c][f+o]=1;o++}h(g[e][f].cell).attr("rowspan",i).attr("colspan",o)}}}}function M(a){var b=w(a,"aoPreDrawCallback","preDraw",[a]);if(-1!==h.inArray(!1,b))C(a,!1);else{var b=[],c=0,e=a.asStripeClasses,d=e.length,f=a.oLanguage,g=a.iInitDisplayStart,j="ssp"==B(a),i=a.aiDisplay;a.bDrawing=!0;g!==k&&-1!==g&&(a._iDisplayStart=
+j?g:g>=a.fnRecordsDisplay()?0:g,a.iInitDisplayStart=-1);var g=a._iDisplayStart,o=a.fnDisplayEnd();if(a.bDeferLoading)a.bDeferLoading=!1,a.iDraw++,C(a,!1);else if(j){if(!a.bDestroying&&!kb(a))return}else a.iDraw++;if(0!==i.length){f=j?a.aoData.length:o;for(j=j?0:g;j<f;j++){var l=i[j],q=a.aoData[l];null===q.nTr&&Ja(a,l);l=q.nTr;if(0!==d){var n=e[c%d];q._sRowStripe!=n&&(h(l).removeClass(q._sRowStripe).addClass(n),q._sRowStripe=n)}w(a,"aoRowCallback",null,[l,q._aData,c,j]);b.push(l);c++}}else c=f.sZeroRecords,
+1==a.iDraw&&"ajax"==B(a)?c=f.sLoadingRecords:f.sEmptyTable&&0===a.fnRecordsTotal()&&(c=f.sEmptyTable),b[0]=h("<tr/>",{"class":d?e[0]:""}).append(h("<td />",{valign:"top",colSpan:aa(a),"class":a.oClasses.sRowEmpty}).html(c))[0];w(a,"aoHeaderCallback","header",[h(a.nTHead).children("tr")[0],La(a),g,o,i]);w(a,"aoFooterCallback","footer",[h(a.nTFoot).children("tr")[0],La(a),g,o,i]);e=h(a.nTBody);e.children().detach();e.append(h(b));w(a,"aoDrawCallback","draw",[a]);a.bSorted=!1;a.bFiltered=!1;a.bDrawing=
+!1}}function N(a,b){var c=a.oFeatures,e=c.bFilter;c.bSort&&lb(a);e?fa(a,a.oPreviousSearch):a.aiDisplay=a.aiDisplayMaster.slice();!0!==b&&(a._iDisplayStart=0);a._drawHold=b;M(a);a._drawHold=!1}function mb(a){var b=a.oClasses,c=h(a.nTable),c=h("<div/>").insertBefore(c),e=a.oFeatures,d=h("<div/>",{id:a.sTableId+"_wrapper","class":b.sWrapper+(a.nTFoot?"":" "+b.sNoFooter)});a.nHolding=c[0];a.nTableWrapper=d[0];a.nTableReinsertBefore=a.nTable.nextSibling;for(var f=a.sDom.split(""),g,j,i,o,l,q,n=0;n<f.length;n++){g=
+null;j=f[n];if("<"==j){i=h("<div/>")[0];o=f[n+1];if("'"==o||'"'==o){l="";for(q=2;f[n+q]!=o;)l+=f[n+q],q++;"H"==l?l=b.sJUIHeader:"F"==l&&(l=b.sJUIFooter);-1!=l.indexOf(".")?(o=l.split("."),i.id=o[0].substr(1,o[0].length-1),i.className=o[1]):"#"==l.charAt(0)?i.id=l.substr(1,l.length-1):i.className=l;n+=q}d.append(i);d=h(i)}else if(">"==j)d=d.parent();else if("l"==j&&e.bPaginate&&e.bLengthChange)g=nb(a);else if("f"==j&&e.bFilter)g=ob(a);else if("r"==j&&e.bProcessing)g=pb(a);else if("t"==j)g=qb(a);else if("i"==
+j&&e.bInfo)g=rb(a);else if("p"==j&&e.bPaginate)g=sb(a);else if(0!==m.ext.feature.length){i=m.ext.feature;q=0;for(o=i.length;q<o;q++)if(j==i[q].cFeature){g=i[q].fnInit(a);break}}g&&(i=a.aanFeatures,i[j]||(i[j]=[]),i[j].push(g),d.append(g))}c.replaceWith(d)}function da(a,b){var c=h(b).children("tr"),e,d,f,g,j,i,o,l,q,n;a.splice(0,a.length);f=0;for(i=c.length;f<i;f++)a.push([]);f=0;for(i=c.length;f<i;f++){e=c[f];for(d=e.firstChild;d;){if("TD"==d.nodeName.toUpperCase()||"TH"==d.nodeName.toUpperCase()){l=
+1*d.getAttribute("colspan");q=1*d.getAttribute("rowspan");l=!l||0===l||1===l?1:l;q=!q||0===q||1===q?1:q;g=0;for(j=a[f];j[g];)g++;o=g;n=1===l?!0:!1;for(j=0;j<l;j++)for(g=0;g<q;g++)a[f+g][o+j]={cell:d,unique:n},a[f+g].nTr=e}d=d.nextSibling}}}function qa(a,b,c){var e=[];c||(c=a.aoHeader,b&&(c=[],da(c,b)));for(var b=0,d=c.length;b<d;b++)for(var f=0,g=c[b].length;f<g;f++)if(c[b][f].unique&&(!e[f]||!a.bSortCellsTop))e[f]=c[b][f].cell;return e}function ra(a,b,c){w(a,"aoServerParams","serverParams",[b]);
+if(b&&h.isArray(b)){var e={},d=/(.*?)\[\]$/;h.each(b,function(a,b){var c=b.name.match(d);c?(c=c[0],e[c]||(e[c]=[]),e[c].push(b.value)):e[b.name]=b.value});b=e}var f,g=a.ajax,j=a.oInstance,i=function(b){w(a,null,"xhr",[a,b,a.jqXHR]);c(b)};if(h.isPlainObject(g)&&g.data){f=g.data;var o=h.isFunction(f)?f(b,a):f,b=h.isFunction(f)&&o?o:h.extend(!0,b,o);delete g.data}o={data:b,success:function(b){var c=b.error||b.sError;c&&I(a,0,c);a.json=b;i(b)},dataType:"json",cache:!1,type:a.sServerMethod,error:function(b,
+c){var f=w(a,null,"xhr",[a,null,a.jqXHR]);-1===h.inArray(!0,f)&&("parsererror"==c?I(a,0,"Invalid JSON response",1):4===b.readyState&&I(a,0,"Ajax error",7));C(a,!1)}};a.oAjaxData=b;w(a,null,"preXhr",[a,b]);a.fnServerData?a.fnServerData.call(j,a.sAjaxSource,h.map(b,function(a,b){return{name:b,value:a}}),i,a):a.sAjaxSource||"string"===typeof g?a.jqXHR=h.ajax(h.extend(o,{url:g||a.sAjaxSource})):h.isFunction(g)?a.jqXHR=g.call(j,b,i,a):(a.jqXHR=h.ajax(h.extend(o,g)),g.data=f)}function kb(a){return a.bAjaxDataGet?
+(a.iDraw++,C(a,!0),ra(a,tb(a),function(b){ub(a,b)}),!1):!0}function tb(a){var b=a.aoColumns,c=b.length,e=a.oFeatures,d=a.oPreviousSearch,f=a.aoPreSearchCols,g,j=[],i,o,l,q=U(a);g=a._iDisplayStart;i=!1!==e.bPaginate?a._iDisplayLength:-1;var n=function(a,b){j.push({name:a,value:b})};n("sEcho",a.iDraw);n("iColumns",c);n("sColumns",D(b,"sName").join(","));n("iDisplayStart",g);n("iDisplayLength",i);var k={draw:a.iDraw,columns:[],order:[],start:g,length:i,search:{value:d.sSearch,regex:d.bRegex}};for(g=
+0;g<c;g++)o=b[g],l=f[g],i="function"==typeof o.mData?"function":o.mData,k.columns.push({data:i,name:o.sName,searchable:o.bSearchable,orderable:o.bSortable,search:{value:l.sSearch,regex:l.bRegex}}),n("mDataProp_"+g,i),e.bFilter&&(n("sSearch_"+g,l.sSearch),n("bRegex_"+g,l.bRegex),n("bSearchable_"+g,o.bSearchable)),e.bSort&&n("bSortable_"+g,o.bSortable);e.bFilter&&(n("sSearch",d.sSearch),n("bRegex",d.bRegex));e.bSort&&(h.each(q,function(a,b){k.order.push({column:b.col,dir:b.dir});n("iSortCol_"+a,b.col);
+n("sSortDir_"+a,b.dir)}),n("iSortingCols",q.length));b=m.ext.legacy.ajax;return null===b?a.sAjaxSource?j:k:b?j:k}function ub(a,b){var c=sa(a,b),e=b.sEcho!==k?b.sEcho:b.draw,d=b.iTotalRecords!==k?b.iTotalRecords:b.recordsTotal,f=b.iTotalDisplayRecords!==k?b.iTotalDisplayRecords:b.recordsFiltered;if(e){if(1*e<a.iDraw)return;a.iDraw=1*e}oa(a);a._iRecordsTotal=parseInt(d,10);a._iRecordsDisplay=parseInt(f,10);e=0;for(d=c.length;e<d;e++)K(a,c[e]);a.aiDisplay=a.aiDisplayMaster.slice();a.bAjaxDataGet=!1;
+M(a);a._bInitComplete||ta(a,b);a.bAjaxDataGet=!0;C(a,!1)}function sa(a,b){var c=h.isPlainObject(a.ajax)&&a.ajax.dataSrc!==k?a.ajax.dataSrc:a.sAjaxDataProp;return"data"===c?b.aaData||b[c]:""!==c?R(c)(b):b}function ob(a){var b=a.oClasses,c=a.sTableId,e=a.oLanguage,d=a.oPreviousSearch,f=a.aanFeatures,g='<input type="search" class="'+b.sFilterInput+'"/>',j=e.sSearch,j=j.match(/_INPUT_/)?j.replace("_INPUT_",g):j+g,b=h("<div/>",{id:!f.f?c+"_filter":null,"class":b.sFilter}).append(h("<label/>").append(j)),
+f=function(){var b=!this.value?"":this.value;b!=d.sSearch&&(fa(a,{sSearch:b,bRegex:d.bRegex,bSmart:d.bSmart,bCaseInsensitive:d.bCaseInsensitive}),a._iDisplayStart=0,M(a))},g=null!==a.searchDelay?a.searchDelay:"ssp"===B(a)?400:0,i=h("input",b).val(d.sSearch).attr("placeholder",e.sSearchPlaceholder).bind("keyup.DT search.DT input.DT paste.DT cut.DT",g?ua(f,g):f).bind("keypress.DT",function(a){if(13==a.keyCode)return!1}).attr("aria-controls",c);h(a.nTable).on("search.dt.DT",function(b,c){if(a===c)try{i[0]!==
+Q.activeElement&&i.val(d.sSearch)}catch(f){}});return b[0]}function fa(a,b,c){var e=a.oPreviousSearch,d=a.aoPreSearchCols,f=function(a){e.sSearch=a.sSearch;e.bRegex=a.bRegex;e.bSmart=a.bSmart;e.bCaseInsensitive=a.bCaseInsensitive};Ha(a);if("ssp"!=B(a)){vb(a,b.sSearch,c,b.bEscapeRegex!==k?!b.bEscapeRegex:b.bRegex,b.bSmart,b.bCaseInsensitive);f(b);for(b=0;b<d.length;b++)wb(a,d[b].sSearch,b,d[b].bEscapeRegex!==k?!d[b].bEscapeRegex:d[b].bRegex,d[b].bSmart,d[b].bCaseInsensitive);xb(a)}else f(b);a.bFiltered=
+!0;w(a,null,"search",[a])}function xb(a){for(var b=m.ext.search,c=a.aiDisplay,e,d,f=0,g=b.length;f<g;f++){for(var j=[],i=0,h=c.length;i<h;i++)d=c[i],e=a.aoData[d],b[f](a,e._aFilterData,d,e._aData,i)&&j.push(d);c.length=0;c.push.apply(c,j)}}function wb(a,b,c,e,d,f){if(""!==b)for(var g=a.aiDisplay,e=Qa(b,e,d,f),d=g.length-1;0<=d;d--)b=a.aoData[g[d]]._aFilterData[c],e.test(b)||g.splice(d,1)}function vb(a,b,c,e,d,f){var e=Qa(b,e,d,f),d=a.oPreviousSearch.sSearch,f=a.aiDisplayMaster,g;0!==m.ext.search.length&&
+(c=!0);g=yb(a);if(0>=b.length)a.aiDisplay=f.slice();else{if(g||c||d.length>b.length||0!==b.indexOf(d)||a.bSorted)a.aiDisplay=f.slice();b=a.aiDisplay;for(c=b.length-1;0<=c;c--)e.test(a.aoData[b[c]]._sFilterRow)||b.splice(c,1)}}function Qa(a,b,c,e){a=b?a:va(a);c&&(a="^(?=.*?"+h.map(a.match(/"[^"]+"|[^ ]+/g)||[""],function(a){if('"'===a.charAt(0))var b=a.match(/^"(.*)"$/),a=b?b[1]:a;return a.replace('"',"")}).join(")(?=.*?")+").*$");return RegExp(a,e?"i":"")}function va(a){return a.replace(Yb,"\\$1")}
+function yb(a){var b=a.aoColumns,c,e,d,f,g,j,i,h,l=m.ext.type.search;c=!1;e=0;for(f=a.aoData.length;e<f;e++)if(h=a.aoData[e],!h._aFilterData){j=[];d=0;for(g=b.length;d<g;d++)c=b[d],c.bSearchable?(i=x(a,e,d,"filter"),l[c.sType]&&(i=l[c.sType](i)),null===i&&(i=""),"string"!==typeof i&&i.toString&&(i=i.toString())):i="",i.indexOf&&-1!==i.indexOf("&")&&(wa.innerHTML=i,i=Zb?wa.textContent:wa.innerText),i.replace&&(i=i.replace(/[\r\n]/g,"")),j.push(i);h._aFilterData=j;h._sFilterRow=j.join("  ");c=!0}return c}
+function zb(a){return{search:a.sSearch,smart:a.bSmart,regex:a.bRegex,caseInsensitive:a.bCaseInsensitive}}function Ab(a){return{sSearch:a.search,bSmart:a.smart,bRegex:a.regex,bCaseInsensitive:a.caseInsensitive}}function rb(a){var b=a.sTableId,c=a.aanFeatures.i,e=h("<div/>",{"class":a.oClasses.sInfo,id:!c?b+"_info":null});c||(a.aoDrawCallback.push({fn:Bb,sName:"information"}),e.attr("role","status").attr("aria-live","polite"),h(a.nTable).attr("aria-describedby",b+"_info"));return e[0]}function Bb(a){var b=
+a.aanFeatures.i;if(0!==b.length){var c=a.oLanguage,e=a._iDisplayStart+1,d=a.fnDisplayEnd(),f=a.fnRecordsTotal(),g=a.fnRecordsDisplay(),j=g?c.sInfo:c.sInfoEmpty;g!==f&&(j+=" "+c.sInfoFiltered);j+=c.sInfoPostFix;j=Cb(a,j);c=c.fnInfoCallback;null!==c&&(j=c.call(a.oInstance,a,e,d,f,g,j));h(b).html(j)}}function Cb(a,b){var c=a.fnFormatNumber,e=a._iDisplayStart+1,d=a._iDisplayLength,f=a.fnRecordsDisplay(),g=-1===d;return b.replace(/_START_/g,c.call(a,e)).replace(/_END_/g,c.call(a,a.fnDisplayEnd())).replace(/_MAX_/g,
+c.call(a,a.fnRecordsTotal())).replace(/_TOTAL_/g,c.call(a,f)).replace(/_PAGE_/g,c.call(a,g?1:Math.ceil(e/d))).replace(/_PAGES_/g,c.call(a,g?1:Math.ceil(f/d)))}function ga(a){var b,c,e=a.iInitDisplayStart,d=a.aoColumns,f;c=a.oFeatures;if(a.bInitialised){mb(a);jb(a);ea(a,a.aoHeader);ea(a,a.aoFooter);C(a,!0);c.bAutoWidth&&Ga(a);b=0;for(c=d.length;b<c;b++)f=d[b],f.sWidth&&(f.nTh.style.width=s(f.sWidth));N(a);d=B(a);"ssp"!=d&&("ajax"==d?ra(a,[],function(c){var f=sa(a,c);for(b=0;b<f.length;b++)K(a,f[b]);
+a.iInitDisplayStart=e;N(a);C(a,!1);ta(a,c)},a):(C(a,!1),ta(a)))}else setTimeout(function(){ga(a)},200)}function ta(a,b){a._bInitComplete=!0;b&&X(a);w(a,"aoInitComplete","init",[a,b])}function Ra(a,b){var c=parseInt(b,10);a._iDisplayLength=c;Sa(a);w(a,null,"length",[a,c])}function nb(a){for(var b=a.oClasses,c=a.sTableId,e=a.aLengthMenu,d=h.isArray(e[0]),f=d?e[0]:e,e=d?e[1]:e,d=h("<select/>",{name:c+"_length","aria-controls":c,"class":b.sLengthSelect}),g=0,j=f.length;g<j;g++)d[0][g]=new Option(e[g],
+f[g]);var i=h("<div><label/></div>").addClass(b.sLength);a.aanFeatures.l||(i[0].id=c+"_length");i.children().append(a.oLanguage.sLengthMenu.replace("_MENU_",d[0].outerHTML));h("select",i).val(a._iDisplayLength).bind("change.DT",function(){Ra(a,h(this).val());M(a)});h(a.nTable).bind("length.dt.DT",function(b,c,f){a===c&&h("select",i).val(f)});return i[0]}function sb(a){var b=a.sPaginationType,c=m.ext.pager[b],e="function"===typeof c,d=function(a){M(a)},b=h("<div/>").addClass(a.oClasses.sPaging+b)[0],
+f=a.aanFeatures;e||c.fnInit(a,b,d);f.p||(b.id=a.sTableId+"_paginate",a.aoDrawCallback.push({fn:function(a){if(e){var b=a._iDisplayStart,i=a._iDisplayLength,h=a.fnRecordsDisplay(),l=-1===i,b=l?0:Math.ceil(b/i),i=l?1:Math.ceil(h/i),h=c(b,i),q,l=0;for(q=f.p.length;l<q;l++)Pa(a,"pageButton")(a,f.p[l],l,h,b,i)}else c.fnUpdate(a,d)},sName:"pagination"}));return b}function Ta(a,b,c){var e=a._iDisplayStart,d=a._iDisplayLength,f=a.fnRecordsDisplay();0===f||-1===d?e=0:"number"===typeof b?(e=b*d,e>f&&(e=0)):
+"first"==b?e=0:"previous"==b?(e=0<=d?e-d:0,0>e&&(e=0)):"next"==b?e+d<f&&(e+=d):"last"==b?e=Math.floor((f-1)/d)*d:I(a,0,"Unknown paging action: "+b,5);b=a._iDisplayStart!==e;a._iDisplayStart=e;b&&(w(a,null,"page",[a]),c&&M(a));return b}function pb(a){return h("<div/>",{id:!a.aanFeatures.r?a.sTableId+"_processing":null,"class":a.oClasses.sProcessing}).html(a.oLanguage.sProcessing).insertBefore(a.nTable)[0]}function C(a,b){a.oFeatures.bProcessing&&h(a.aanFeatures.r).css("display",b?"block":"none");w(a,
+null,"processing",[a,b])}function qb(a){var b=h(a.nTable);b.attr("role","grid");var c=a.oScroll;if(""===c.sX&&""===c.sY)return a.nTable;var e=c.sX,d=c.sY,f=a.oClasses,g=b.children("caption"),j=g.length?g[0]._captionSide:null,i=h(b[0].cloneNode(!1)),o=h(b[0].cloneNode(!1)),l=b.children("tfoot");c.sX&&"100%"===b.attr("width")&&b.removeAttr("width");l.length||(l=null);c=h("<div/>",{"class":f.sScrollWrapper}).append(h("<div/>",{"class":f.sScrollHead}).css({overflow:"hidden",position:"relative",border:0,
+width:e?!e?null:s(e):"100%"}).append(h("<div/>",{"class":f.sScrollHeadInner}).css({"box-sizing":"content-box",width:c.sXInner||"100%"}).append(i.removeAttr("id").css("margin-left",0).append("top"===j?g:null).append(b.children("thead"))))).append(h("<div/>",{"class":f.sScrollBody}).css({overflow:"auto",height:!d?null:s(d),width:!e?null:s(e)}).append(b));l&&c.append(h("<div/>",{"class":f.sScrollFoot}).css({overflow:"hidden",border:0,width:e?!e?null:s(e):"100%"}).append(h("<div/>",{"class":f.sScrollFootInner}).append(o.removeAttr("id").css("margin-left",
+0).append("bottom"===j?g:null).append(b.children("tfoot")))));var b=c.children(),q=b[0],f=b[1],n=l?b[2]:null;if(e)h(f).on("scroll.DT",function(){var a=this.scrollLeft;q.scrollLeft=a;l&&(n.scrollLeft=a)});a.nScrollHead=q;a.nScrollBody=f;a.nScrollFoot=n;a.aoDrawCallback.push({fn:Y,sName:"scrolling"});return c[0]}function Y(a){var b=a.oScroll,c=b.sX,e=b.sXInner,d=b.sY,f=b.iBarWidth,g=h(a.nScrollHead),j=g[0].style,i=g.children("div"),o=i[0].style,l=i.children("table"),i=a.nScrollBody,q=h(i),n=i.style,
+k=h(a.nScrollFoot).children("div"),p=k.children("table"),m=h(a.nTHead),r=h(a.nTable),t=r[0],O=t.style,L=a.nTFoot?h(a.nTFoot):null,ha=a.oBrowser,w=ha.bScrollOversize,v,u,y,x,z,A=[],B=[],C=[],D,E=function(a){a=a.style;a.paddingTop="0";a.paddingBottom="0";a.borderTopWidth="0";a.borderBottomWidth="0";a.height=0};r.children("thead, tfoot").remove();z=m.clone().prependTo(r);v=m.find("tr");y=z.find("tr");z.find("th, td").removeAttr("tabindex");L&&(x=L.clone().prependTo(r),u=L.find("tr"),x=x.find("tr"));
+c||(n.width="100%",g[0].style.width="100%");h.each(qa(a,z),function(b,c){D=la(a,b);c.style.width=a.aoColumns[D].sWidth});L&&G(function(a){a.style.width=""},x);b.bCollapse&&""!==d&&(n.height=q[0].offsetHeight+m[0].offsetHeight+"px");g=r.outerWidth();if(""===c){if(O.width="100%",w&&(r.find("tbody").height()>i.offsetHeight||"scroll"==q.css("overflow-y")))O.width=s(r.outerWidth()-f)}else""!==e?O.width=s(e):g==q.width()&&q.height()<r.height()?(O.width=s(g-f),r.outerWidth()>g-f&&(O.width=s(g))):O.width=
+s(g);g=r.outerWidth();G(E,y);G(function(a){C.push(a.innerHTML);A.push(s(h(a).css("width")))},y);G(function(a,b){a.style.width=A[b]},v);h(y).height(0);L&&(G(E,x),G(function(a){B.push(s(h(a).css("width")))},x),G(function(a,b){a.style.width=B[b]},u),h(x).height(0));G(function(a,b){a.innerHTML='<div class="dataTables_sizing" style="height:0;overflow:hidden;">'+C[b]+"</div>";a.style.width=A[b]},y);L&&G(function(a,b){a.innerHTML="";a.style.width=B[b]},x);if(r.outerWidth()<g){u=i.scrollHeight>i.offsetHeight||
+"scroll"==q.css("overflow-y")?g+f:g;if(w&&(i.scrollHeight>i.offsetHeight||"scroll"==q.css("overflow-y")))O.width=s(u-f);(""===c||""!==e)&&I(a,1,"Possible column misalignment",6)}else u="100%";n.width=s(u);j.width=s(u);L&&(a.nScrollFoot.style.width=s(u));!d&&w&&(n.height=s(t.offsetHeight+f));d&&b.bCollapse&&(n.height=s(d),b=c&&t.offsetWidth>i.offsetWidth?f:0,t.offsetHeight<i.offsetHeight&&(n.height=s(t.offsetHeight+b)));b=r.outerWidth();l[0].style.width=s(b);o.width=s(b);l=r.height()>i.clientHeight||
+"scroll"==q.css("overflow-y");ha="padding"+(ha.bScrollbarLeft?"Left":"Right");o[ha]=l?f+"px":"0px";L&&(p[0].style.width=s(b),k[0].style.width=s(b),k[0].style[ha]=l?f+"px":"0px");q.scroll();if((a.bSorted||a.bFiltered)&&!a._drawHold)i.scrollTop=0}function G(a,b,c){for(var e=0,d=0,f=b.length,g,j;d<f;){g=b[d].firstChild;for(j=c?c[d].firstChild:null;g;)1===g.nodeType&&(c?a(g,j,e):a(g,e),e++),g=g.nextSibling,j=c?j.nextSibling:null;d++}}function Ga(a){var b=a.nTable,c=a.aoColumns,e=a.oScroll,d=e.sY,f=e.sX,
+g=e.sXInner,j=c.length,e=Z(a,"bVisible"),i=h("th",a.nTHead),o=b.getAttribute("width"),l=b.parentNode,k=!1,n,m;(n=b.style.width)&&-1!==n.indexOf("%")&&(o=n);for(n=0;n<e.length;n++)m=c[e[n]],null!==m.sWidth&&(m.sWidth=Db(m.sWidthOrig,l),k=!0);if(!k&&!f&&!d&&j==aa(a)&&j==i.length)for(n=0;n<j;n++)c[n].sWidth=s(i.eq(n).width());else{j=h(b).clone().css("visibility","hidden").removeAttr("id");j.find("tbody tr").remove();var p=h("<tr/>").appendTo(j.find("tbody"));j.find("tfoot th, tfoot td").css("width",
+"");i=qa(a,j.find("thead")[0]);for(n=0;n<e.length;n++)m=c[e[n]],i[n].style.width=null!==m.sWidthOrig&&""!==m.sWidthOrig?s(m.sWidthOrig):"";if(a.aoData.length)for(n=0;n<e.length;n++)k=e[n],m=c[k],h(Eb(a,k)).clone(!1).append(m.sContentPadding).appendTo(p);j.appendTo(l);f&&g?j.width(g):f?(j.css("width","auto"),j.width()<l.offsetWidth&&j.width(l.offsetWidth)):d?j.width(l.offsetWidth):o&&j.width(o);Fb(a,j[0]);if(f){for(n=g=0;n<e.length;n++)m=c[e[n]],d=h(i[n]).outerWidth(),g+=null===m.sWidthOrig?d:parseInt(m.sWidth,
+10)+d-h(i[n]).width();j.width(s(g));b.style.width=s(g)}for(n=0;n<e.length;n++)if(m=c[e[n]],d=h(i[n]).width())m.sWidth=s(d);b.style.width=s(j.css("width"));j.remove()}o&&(b.style.width=s(o));if((o||f)&&!a._reszEvt)b=function(){h(Ea).bind("resize.DT-"+a.sInstance,ua(function(){X(a)}))},a.oBrowser.bScrollOversize?setTimeout(b,1E3):b(),a._reszEvt=!0}function ua(a,b){var c=b!==k?b:200,e,d;return function(){var b=this,g=+new Date,j=arguments;e&&g<e+c?(clearTimeout(d),d=setTimeout(function(){e=k;a.apply(b,
+j)},c)):(e=g,a.apply(b,j))}}function Db(a,b){if(!a)return 0;var c=h("<div/>").css("width",s(a)).appendTo(b||Q.body),e=c[0].offsetWidth;c.remove();return e}function Fb(a,b){var c=a.oScroll;if(c.sX||c.sY)c=!c.sX?c.iBarWidth:0,b.style.width=s(h(b).outerWidth()-c)}function Eb(a,b){var c=Gb(a,b);if(0>c)return null;var e=a.aoData[c];return!e.nTr?h("<td/>").html(x(a,c,b,"display"))[0]:e.anCells[b]}function Gb(a,b){for(var c,e=-1,d=-1,f=0,g=a.aoData.length;f<g;f++)c=x(a,f,b,"display")+"",c=c.replace($b,""),
+c.length>e&&(e=c.length,d=f);return d}function s(a){return null===a?"0px":"number"==typeof a?0>a?"0px":a+"px":a.match(/\d$/)?a+"px":a}function Hb(){var a=m.__scrollbarWidth;if(a===k){var b=h("<p/>").css({position:"absolute",top:0,left:0,width:"100%",height:150,padding:0,overflow:"scroll",visibility:"hidden"}).appendTo("body"),a=b[0].offsetWidth-b[0].clientWidth;m.__scrollbarWidth=a;b.remove()}return a}function U(a){var b,c,e=[],d=a.aoColumns,f,g,j,i;b=a.aaSortingFixed;c=h.isPlainObject(b);var o=[];
+f=function(a){a.length&&!h.isArray(a[0])?o.push(a):o.push.apply(o,a)};h.isArray(b)&&f(b);c&&b.pre&&f(b.pre);f(a.aaSorting);c&&b.post&&f(b.post);for(a=0;a<o.length;a++){i=o[a][0];f=d[i].aDataSort;b=0;for(c=f.length;b<c;b++)g=f[b],j=d[g].sType||"string",o[a]._idx===k&&(o[a]._idx=h.inArray(o[a][1],d[g].asSorting)),e.push({src:i,col:g,dir:o[a][1],index:o[a]._idx,type:j,formatter:m.ext.type.order[j+"-pre"]})}return e}function lb(a){var b,c,e=[],d=m.ext.type.order,f=a.aoData,g=0,j,i=a.aiDisplayMaster,h;
+Ha(a);h=U(a);b=0;for(c=h.length;b<c;b++)j=h[b],j.formatter&&g++,Ib(a,j.col);if("ssp"!=B(a)&&0!==h.length){b=0;for(c=i.length;b<c;b++)e[i[b]]=b;g===h.length?i.sort(function(a,b){var c,d,g,j,i=h.length,k=f[a]._aSortData,m=f[b]._aSortData;for(g=0;g<i;g++)if(j=h[g],c=k[j.col],d=m[j.col],c=c<d?-1:c>d?1:0,0!==c)return"asc"===j.dir?c:-c;c=e[a];d=e[b];return c<d?-1:c>d?1:0}):i.sort(function(a,b){var c,g,j,i,k=h.length,m=f[a]._aSortData,r=f[b]._aSortData;for(j=0;j<k;j++)if(i=h[j],c=m[i.col],g=r[i.col],i=d[i.type+
+"-"+i.dir]||d["string-"+i.dir],c=i(c,g),0!==c)return c;c=e[a];g=e[b];return c<g?-1:c>g?1:0})}a.bSorted=!0}function Jb(a){for(var b,c,e=a.aoColumns,d=U(a),a=a.oLanguage.oAria,f=0,g=e.length;f<g;f++){c=e[f];var j=c.asSorting;b=c.sTitle.replace(/<.*?>/g,"");var i=c.nTh;i.removeAttribute("aria-sort");c.bSortable&&(0<d.length&&d[0].col==f?(i.setAttribute("aria-sort","asc"==d[0].dir?"ascending":"descending"),c=j[d[0].index+1]||j[0]):c=j[0],b+="asc"===c?a.sSortAscending:a.sSortDescending);i.setAttribute("aria-label",
+b)}}function Ua(a,b,c,e){var d=a.aaSorting,f=a.aoColumns[b].asSorting,g=function(a,b){var c=a._idx;c===k&&(c=h.inArray(a[1],f));return c+1<f.length?c+1:b?null:0};"number"===typeof d[0]&&(d=a.aaSorting=[d]);c&&a.oFeatures.bSortMulti?(c=h.inArray(b,D(d,"0")),-1!==c?(b=g(d[c],!0),null===b&&1===d.length&&(b=0),null===b?d.splice(c,1):(d[c][1]=f[b],d[c]._idx=b)):(d.push([b,f[0],0]),d[d.length-1]._idx=0)):d.length&&d[0][0]==b?(b=g(d[0]),d.length=1,d[0][1]=f[b],d[0]._idx=b):(d.length=0,d.push([b,f[0]]),d[0]._idx=
+0);N(a);"function"==typeof e&&e(a)}function Oa(a,b,c,e){var d=a.aoColumns[c];Va(b,{},function(b){!1!==d.bSortable&&(a.oFeatures.bProcessing?(C(a,!0),setTimeout(function(){Ua(a,c,b.shiftKey,e);"ssp"!==B(a)&&C(a,!1)},0)):Ua(a,c,b.shiftKey,e))})}function xa(a){var b=a.aLastSort,c=a.oClasses.sSortColumn,e=U(a),d=a.oFeatures,f,g;if(d.bSort&&d.bSortClasses){d=0;for(f=b.length;d<f;d++)g=b[d].src,h(D(a.aoData,"anCells",g)).removeClass(c+(2>d?d+1:3));d=0;for(f=e.length;d<f;d++)g=e[d].src,h(D(a.aoData,"anCells",
+g)).addClass(c+(2>d?d+1:3))}a.aLastSort=e}function Ib(a,b){var c=a.aoColumns[b],e=m.ext.order[c.sSortDataType],d;e&&(d=e.call(a.oInstance,a,b,$(a,b)));for(var f,g=m.ext.type.order[c.sType+"-pre"],j=0,i=a.aoData.length;j<i;j++)if(c=a.aoData[j],c._aSortData||(c._aSortData=[]),!c._aSortData[b]||e)f=e?d[j]:x(a,j,b,"sort"),c._aSortData[b]=g?g(f):f}function ya(a){if(a.oFeatures.bStateSave&&!a.bDestroying){var b={time:+new Date,start:a._iDisplayStart,length:a._iDisplayLength,order:h.extend(!0,[],a.aaSorting),
+search:zb(a.oPreviousSearch),columns:h.map(a.aoColumns,function(b,e){return{visible:b.bVisible,search:zb(a.aoPreSearchCols[e])}})};w(a,"aoStateSaveParams","stateSaveParams",[a,b]);a.oSavedState=b;a.fnStateSaveCallback.call(a.oInstance,a,b)}}function Kb(a){var b,c,e=a.aoColumns;if(a.oFeatures.bStateSave){var d=a.fnStateLoadCallback.call(a.oInstance,a);if(d&&d.time&&(b=w(a,"aoStateLoadParams","stateLoadParams",[a,d]),-1===h.inArray(!1,b)&&(b=a.iStateDuration,!(0<b&&d.time<+new Date-1E3*b)&&e.length===
+d.columns.length))){a.oLoadedState=h.extend(!0,{},d);d.start!==k&&(a._iDisplayStart=d.start,a.iInitDisplayStart=d.start);d.length!==k&&(a._iDisplayLength=d.length);d.order!==k&&(a.aaSorting=[],h.each(d.order,function(b,c){a.aaSorting.push(c[0]>=e.length?[0,c[1]]:c)}));d.search!==k&&h.extend(a.oPreviousSearch,Ab(d.search));b=0;for(c=d.columns.length;b<c;b++){var f=d.columns[b];f.visible!==k&&(e[b].bVisible=f.visible);f.search!==k&&h.extend(a.aoPreSearchCols[b],Ab(f.search))}w(a,"aoStateLoaded","stateLoaded",
+[a,d])}}}function za(a){var b=m.settings,a=h.inArray(a,D(b,"nTable"));return-1!==a?b[a]:null}function I(a,b,c,e){c="DataTables warning: "+(null!==a?"table id="+a.sTableId+" - ":"")+c;e&&(c+=". For more information about this error, please see http://datatables.net/tn/"+e);if(b)Ea.console&&console.log&&console.log(c);else if(b=m.ext,b=b.sErrMode||b.errMode,w(a,null,"error",[a,e,c]),"alert"==b)alert(c);else{if("throw"==b)throw Error(c);"function"==typeof b&&b(a,e,c)}}function E(a,b,c,e){h.isArray(c)?
+h.each(c,function(c,f){h.isArray(f)?E(a,b,f[0],f[1]):E(a,b,f)}):(e===k&&(e=c),b[c]!==k&&(a[e]=b[c]))}function Lb(a,b,c){var e,d;for(d in b)b.hasOwnProperty(d)&&(e=b[d],h.isPlainObject(e)?(h.isPlainObject(a[d])||(a[d]={}),h.extend(!0,a[d],e)):a[d]=c&&"data"!==d&&"aaData"!==d&&h.isArray(e)?e.slice():e);return a}function Va(a,b,c){h(a).bind("click.DT",b,function(b){a.blur();c(b)}).bind("keypress.DT",b,function(a){13===a.which&&(a.preventDefault(),c(a))}).bind("selectstart.DT",function(){return!1})}function z(a,
+b,c,e){c&&a[b].push({fn:c,sName:e})}function w(a,b,c,e){var d=[];b&&(d=h.map(a[b].slice().reverse(),function(b){return b.fn.apply(a.oInstance,e)}));null!==c&&(b=h.Event(c+".dt"),h(a.nTable).trigger(b,e),d.push(b.result));return d}function Sa(a){var b=a._iDisplayStart,c=a.fnDisplayEnd(),e=a._iDisplayLength;b>=c&&(b=c-e);b-=b%e;if(-1===e||0>b)b=0;a._iDisplayStart=b}function Pa(a,b){var c=a.renderer,e=m.ext.renderer[b];return h.isPlainObject(c)&&c[b]?e[c[b]]||e._:"string"===typeof c?e[c]||e._:e._}function B(a){return a.oFeatures.bServerSide?
+"ssp":a.ajax||a.sAjaxSource?"ajax":"dom"}function Wa(a,b){var c=[],c=Mb.numbers_length,e=Math.floor(c/2);b<=c?c=V(0,b):a<=e?(c=V(0,c-2),c.push("ellipsis"),c.push(b-1)):(a>=b-1-e?c=V(b-(c-2),b):(c=V(a-e+2,a+e-1),c.push("ellipsis"),c.push(b-1)),c.splice(0,0,"ellipsis"),c.splice(0,0,0));c.DT_el="span";return c}function db(a){h.each({num:function(b){return Aa(b,a)},"num-fmt":function(b){return Aa(b,a,Xa)},"html-num":function(b){return Aa(b,a,Ba)},"html-num-fmt":function(b){return Aa(b,a,Ba,Xa)}},function(b,
+c){u.type.order[b+a+"-pre"]=c;b.match(/^html\-/)&&(u.type.search[b+a]=u.type.search.html)})}function Nb(a){return function(){var b=[za(this[m.ext.iApiIndex])].concat(Array.prototype.slice.call(arguments));return m.ext.internal[a].apply(this,b)}}var m,u,t,r,v,Ya={},Ob=/[\r\n]/g,Ba=/<.*?>/g,ac=/^[\w\+\-]/,bc=/[\w\+\-]$/,Yb=RegExp("(\\/|\\.|\\*|\\+|\\?|\\||\\(|\\)|\\[|\\]|\\{|\\}|\\\\|\\$|\\^|\\-)","g"),Xa=/[',$\u00a3\u20ac\u00a5%\u2009\u202F\u20BD\u20a9\u20BArfk]/gi,J=function(a){return!a||!0===a||
+"-"===a?!0:!1},Pb=function(a){var b=parseInt(a,10);return!isNaN(b)&&isFinite(a)?b:null},Qb=function(a,b){Ya[b]||(Ya[b]=RegExp(va(b),"g"));return"string"===typeof a&&"."!==b?a.replace(/\./g,"").replace(Ya[b],"."):a},Za=function(a,b,c){var e="string"===typeof a;if(J(a))return!0;b&&e&&(a=Qb(a,b));c&&e&&(a=a.replace(Xa,""));return!isNaN(parseFloat(a))&&isFinite(a)},Rb=function(a,b,c){return J(a)?!0:!(J(a)||"string"===typeof a)?null:Za(a.replace(Ba,""),b,c)?!0:null},D=function(a,b,c){var e=[],d=0,f=a.length;
+if(c!==k)for(;d<f;d++)a[d]&&a[d][b]&&e.push(a[d][b][c]);else for(;d<f;d++)a[d]&&e.push(a[d][b]);return e},ia=function(a,b,c,e){var d=[],f=0,g=b.length;if(e!==k)for(;f<g;f++)a[b[f]][c]&&d.push(a[b[f]][c][e]);else for(;f<g;f++)d.push(a[b[f]][c]);return d},V=function(a,b){var c=[],e;b===k?(b=0,e=a):(e=b,b=a);for(var d=b;d<e;d++)c.push(d);return c},Sb=function(a){for(var b=[],c=0,e=a.length;c<e;c++)a[c]&&b.push(a[c]);return b},Na=function(a){var b=[],c,e,d=a.length,f,g=0;e=0;a:for(;e<d;e++){c=a[e];for(f=
+0;f<g;f++)if(b[f]===c)continue a;b.push(c);g++}return b},A=function(a,b,c){a[b]!==k&&(a[c]=a[b])},ba=/\[.*?\]$/,T=/\(\)$/,wa=h("<div>")[0],Zb=wa.textContent!==k,$b=/<.*?>/g;m=function(a){this.$=function(a,b){return this.api(!0).$(a,b)};this._=function(a,b){return this.api(!0).rows(a,b).data()};this.api=function(a){return a?new t(za(this[u.iApiIndex])):new t(this)};this.fnAddData=function(a,b){var c=this.api(!0),e=h.isArray(a)&&(h.isArray(a[0])||h.isPlainObject(a[0]))?c.rows.add(a):c.row.add(a);(b===
+k||b)&&c.draw();return e.flatten().toArray()};this.fnAdjustColumnSizing=function(a){var b=this.api(!0).columns.adjust(),c=b.settings()[0],e=c.oScroll;a===k||a?b.draw(!1):(""!==e.sX||""!==e.sY)&&Y(c)};this.fnClearTable=function(a){var b=this.api(!0).clear();(a===k||a)&&b.draw()};this.fnClose=function(a){this.api(!0).row(a).child.hide()};this.fnDeleteRow=function(a,b,c){var e=this.api(!0),a=e.rows(a),d=a.settings()[0],h=d.aoData[a[0][0]];a.remove();b&&b.call(this,d,h);(c===k||c)&&e.draw();return h};
+this.fnDestroy=function(a){this.api(!0).destroy(a)};this.fnDraw=function(a){this.api(!0).draw(a)};this.fnFilter=function(a,b,c,e,d,h){d=this.api(!0);null===b||b===k?d.search(a,c,e,h):d.column(b).search(a,c,e,h);d.draw()};this.fnGetData=function(a,b){var c=this.api(!0);if(a!==k){var e=a.nodeName?a.nodeName.toLowerCase():"";return b!==k||"td"==e||"th"==e?c.cell(a,b).data():c.row(a).data()||null}return c.data().toArray()};this.fnGetNodes=function(a){var b=this.api(!0);return a!==k?b.row(a).node():b.rows().nodes().flatten().toArray()};
+this.fnGetPosition=function(a){var b=this.api(!0),c=a.nodeName.toUpperCase();return"TR"==c?b.row(a).index():"TD"==c||"TH"==c?(a=b.cell(a).index(),[a.row,a.columnVisible,a.column]):null};this.fnIsOpen=function(a){return this.api(!0).row(a).child.isShown()};this.fnOpen=function(a,b,c){return this.api(!0).row(a).child(b,c).show().child()[0]};this.fnPageChange=function(a,b){var c=this.api(!0).page(a);(b===k||b)&&c.draw(!1)};this.fnSetColumnVis=function(a,b,c){a=this.api(!0).column(a).visible(b);(c===
+k||c)&&a.columns.adjust().draw()};this.fnSettings=function(){return za(this[u.iApiIndex])};this.fnSort=function(a){this.api(!0).order(a).draw()};this.fnSortListener=function(a,b,c){this.api(!0).order.listener(a,b,c)};this.fnUpdate=function(a,b,c,e,d){var h=this.api(!0);c===k||null===c?h.row(b).data(a):h.cell(b,c).data(a);(d===k||d)&&h.columns.adjust();(e===k||e)&&h.draw();return 0};this.fnVersionCheck=u.fnVersionCheck;var b=this,c=a===k,e=this.length;c&&(a={});this.oApi=this.internal=u.internal;for(var d in m.ext.internal)d&&
+(this[d]=Nb(d));this.each(function(){var d={},d=1<e?Lb(d,a,!0):a,g=0,j,i=this.getAttribute("id"),o=!1,l=m.defaults,q=h(this);if("table"!=this.nodeName.toLowerCase())I(null,0,"Non-table node initialisation ("+this.nodeName+")",2);else{eb(l);fb(l.column);H(l,l,!0);H(l.column,l.column,!0);H(l,h.extend(d,q.data()));var n=m.settings,g=0;for(j=n.length;g<j;g++){var r=n[g];if(r.nTable==this||r.nTHead.parentNode==this||r.nTFoot&&r.nTFoot.parentNode==this){g=d.bRetrieve!==k?d.bRetrieve:l.bRetrieve;if(c||g)return r.oInstance;
+if(d.bDestroy!==k?d.bDestroy:l.bDestroy){r.oInstance.fnDestroy();break}else{I(r,0,"Cannot reinitialise DataTable",3);return}}if(r.sTableId==this.id){n.splice(g,1);break}}if(null===i||""===i)this.id=i="DataTables_Table_"+m.ext._unique++;var p=h.extend(!0,{},m.models.oSettings,{sDestroyWidth:q[0].style.width,sInstance:i,sTableId:i});p.nTable=this;p.oApi=b.internal;p.oInit=d;n.push(p);p.oInstance=1===b.length?b:q.dataTable();eb(d);d.oLanguage&&P(d.oLanguage);d.aLengthMenu&&!d.iDisplayLength&&(d.iDisplayLength=
+h.isArray(d.aLengthMenu[0])?d.aLengthMenu[0][0]:d.aLengthMenu[0]);d=Lb(h.extend(!0,{},l),d);E(p.oFeatures,d,"bPaginate bLengthChange bFilter bSort bSortMulti bInfo bProcessing bAutoWidth bSortClasses bServerSide bDeferRender".split(" "));E(p,d,["asStripeClasses","ajax","fnServerData","fnFormatNumber","sServerMethod","aaSorting","aaSortingFixed","aLengthMenu","sPaginationType","sAjaxSource","sAjaxDataProp","iStateDuration","sDom","bSortCellsTop","iTabIndex","fnStateLoadCallback","fnStateSaveCallback",
+"renderer","searchDelay",["iCookieDuration","iStateDuration"],["oSearch","oPreviousSearch"],["aoSearchCols","aoPreSearchCols"],["iDisplayLength","_iDisplayLength"],["bJQueryUI","bJUI"]]);E(p.oScroll,d,[["sScrollX","sX"],["sScrollXInner","sXInner"],["sScrollY","sY"],["bScrollCollapse","bCollapse"]]);E(p.oLanguage,d,"fnInfoCallback");z(p,"aoDrawCallback",d.fnDrawCallback,"user");z(p,"aoServerParams",d.fnServerParams,"user");z(p,"aoStateSaveParams",d.fnStateSaveParams,"user");z(p,"aoStateLoadParams",
+d.fnStateLoadParams,"user");z(p,"aoStateLoaded",d.fnStateLoaded,"user");z(p,"aoRowCallback",d.fnRowCallback,"user");z(p,"aoRowCreatedCallback",d.fnCreatedRow,"user");z(p,"aoHeaderCallback",d.fnHeaderCallback,"user");z(p,"aoFooterCallback",d.fnFooterCallback,"user");z(p,"aoInitComplete",d.fnInitComplete,"user");z(p,"aoPreDrawCallback",d.fnPreDrawCallback,"user");i=p.oClasses;d.bJQueryUI?(h.extend(i,m.ext.oJUIClasses,d.oClasses),d.sDom===l.sDom&&"lfrtip"===l.sDom&&(p.sDom='<"H"lfr>t<"F"ip>'),p.renderer)?
+h.isPlainObject(p.renderer)&&!p.renderer.header&&(p.renderer.header="jqueryui"):p.renderer="jqueryui":h.extend(i,m.ext.classes,d.oClasses);q.addClass(i.sTable);if(""!==p.oScroll.sX||""!==p.oScroll.sY)p.oScroll.iBarWidth=Hb();!0===p.oScroll.sX&&(p.oScroll.sX="100%");p.iInitDisplayStart===k&&(p.iInitDisplayStart=d.iDisplayStart,p._iDisplayStart=d.iDisplayStart);null!==d.iDeferLoading&&(p.bDeferLoading=!0,g=h.isArray(d.iDeferLoading),p._iRecordsDisplay=g?d.iDeferLoading[0]:d.iDeferLoading,p._iRecordsTotal=
+g?d.iDeferLoading[1]:d.iDeferLoading);var t=p.oLanguage;h.extend(!0,t,d.oLanguage);""!==t.sUrl&&(h.ajax({dataType:"json",url:t.sUrl,success:function(a){P(a);H(l.oLanguage,a);h.extend(true,t,a);ga(p)},error:function(){ga(p)}}),o=!0);null===d.asStripeClasses&&(p.asStripeClasses=[i.sStripeOdd,i.sStripeEven]);var g=p.asStripeClasses,s=q.children("tbody").find("tr").eq(0);-1!==h.inArray(!0,h.map(g,function(a){return s.hasClass(a)}))&&(h("tbody tr",this).removeClass(g.join(" ")),p.asDestroyStripes=g.slice());
+n=[];g=this.getElementsByTagName("thead");0!==g.length&&(da(p.aoHeader,g[0]),n=qa(p));if(null===d.aoColumns){r=[];g=0;for(j=n.length;g<j;g++)r.push(null)}else r=d.aoColumns;g=0;for(j=r.length;g<j;g++)Fa(p,n?n[g]:null);ib(p,d.aoColumnDefs,r,function(a,b){ka(p,a,b)});if(s.length){var u=function(a,b){return a.getAttribute("data-"+b)!==null?b:null};h.each(na(p,s[0]).cells,function(a,b){var c=p.aoColumns[a];if(c.mData===a){var d=u(b,"sort")||u(b,"order"),e=u(b,"filter")||u(b,"search");if(d!==null||e!==
+null){c.mData={_:a+".display",sort:d!==null?a+".@data-"+d:k,type:d!==null?a+".@data-"+d:k,filter:e!==null?a+".@data-"+e:k};ka(p,a)}}})}var v=p.oFeatures;d.bStateSave&&(v.bStateSave=!0,Kb(p,d),z(p,"aoDrawCallback",ya,"state_save"));if(d.aaSorting===k){n=p.aaSorting;g=0;for(j=n.length;g<j;g++)n[g][1]=p.aoColumns[g].asSorting[0]}xa(p);v.bSort&&z(p,"aoDrawCallback",function(){if(p.bSorted){var a=U(p),b={};h.each(a,function(a,c){b[c.src]=c.dir});w(p,null,"order",[p,a,b]);Jb(p)}});z(p,"aoDrawCallback",
+function(){(p.bSorted||B(p)==="ssp"||v.bDeferRender)&&xa(p)},"sc");gb(p);g=q.children("caption").each(function(){this._captionSide=q.css("caption-side")});j=q.children("thead");0===j.length&&(j=h("<thead/>").appendTo(this));p.nTHead=j[0];j=q.children("tbody");0===j.length&&(j=h("<tbody/>").appendTo(this));p.nTBody=j[0];j=q.children("tfoot");if(0===j.length&&0<g.length&&(""!==p.oScroll.sX||""!==p.oScroll.sY))j=h("<tfoot/>").appendTo(this);0===j.length||0===j.children().length?q.addClass(i.sNoFooter):
+0<j.length&&(p.nTFoot=j[0],da(p.aoFooter,p.nTFoot));if(d.aaData)for(g=0;g<d.aaData.length;g++)K(p,d.aaData[g]);else(p.bDeferLoading||"dom"==B(p))&&ma(p,h(p.nTBody).children("tr"));p.aiDisplay=p.aiDisplayMaster.slice();p.bInitialised=!0;!1===o&&ga(p)}});b=null;return this};var Tb=[],y=Array.prototype,cc=function(a){var b,c,e=m.settings,d=h.map(e,function(a){return a.nTable});if(a){if(a.nTable&&a.oApi)return[a];if(a.nodeName&&"table"===a.nodeName.toLowerCase())return b=h.inArray(a,d),-1!==b?[e[b]]:
+null;if(a&&"function"===typeof a.settings)return a.settings().toArray();"string"===typeof a?c=h(a):a instanceof h&&(c=a)}else return[];if(c)return c.map(function(){b=h.inArray(this,d);return-1!==b?e[b]:null}).toArray()};t=function(a,b){if(!(this instanceof t))return new t(a,b);var c=[],e=function(a){(a=cc(a))&&c.push.apply(c,a)};if(h.isArray(a))for(var d=0,f=a.length;d<f;d++)e(a[d]);else e(a);this.context=Na(c);b&&this.push.apply(this,b.toArray?b.toArray():b);this.selector={rows:null,cols:null,opts:null};
+t.extend(this,this,Tb)};m.Api=t;t.prototype={any:function(){return 0!==this.flatten().length},concat:y.concat,context:[],each:function(a){for(var b=0,c=this.length;b<c;b++)a.call(this,this[b],b,this);return this},eq:function(a){var b=this.context;return b.length>a?new t(b[a],this[a]):null},filter:function(a){var b=[];if(y.filter)b=y.filter.call(this,a,this);else for(var c=0,e=this.length;c<e;c++)a.call(this,this[c],c,this)&&b.push(this[c]);return new t(this.context,b)},flatten:function(){var a=[];
+return new t(this.context,a.concat.apply(a,this.toArray()))},join:y.join,indexOf:y.indexOf||function(a,b){for(var c=b||0,e=this.length;c<e;c++)if(this[c]===a)return c;return-1},iterator:function(a,b,c,e){var d=[],f,g,h,i,o,l=this.context,q,n,m=this.selector;"string"===typeof a&&(e=c,c=b,b=a,a=!1);g=0;for(h=l.length;g<h;g++){var p=new t(l[g]);if("table"===b)f=c.call(p,l[g],g),f!==k&&d.push(f);else if("columns"===b||"rows"===b)f=c.call(p,l[g],this[g],g),f!==k&&d.push(f);else if("column"===b||"column-rows"===
+b||"row"===b||"cell"===b){n=this[g];"column-rows"===b&&(q=Ca(l[g],m.opts));i=0;for(o=n.length;i<o;i++)f=n[i],f="cell"===b?c.call(p,l[g],f.row,f.column,g,i):c.call(p,l[g],f,g,i,q),f!==k&&d.push(f)}}return d.length||e?(a=new t(l,a?d.concat.apply([],d):d),b=a.selector,b.rows=m.rows,b.cols=m.cols,b.opts=m.opts,a):this},lastIndexOf:y.lastIndexOf||function(a,b){return this.indexOf.apply(this.toArray.reverse(),arguments)},length:0,map:function(a){var b=[];if(y.map)b=y.map.call(this,a,this);else for(var c=
+0,e=this.length;c<e;c++)b.push(a.call(this,this[c],c));return new t(this.context,b)},pluck:function(a){return this.map(function(b){return b[a]})},pop:y.pop,push:y.push,reduce:y.reduce||function(a,b){return hb(this,a,b,0,this.length,1)},reduceRight:y.reduceRight||function(a,b){return hb(this,a,b,this.length-1,-1,-1)},reverse:y.reverse,selector:null,shift:y.shift,sort:y.sort,splice:y.splice,toArray:function(){return y.slice.call(this)},to$:function(){return h(this)},toJQuery:function(){return h(this)},
+unique:function(){return new t(this.context,Na(this))},unshift:y.unshift};t.extend=function(a,b,c){if(c.length&&b&&(b instanceof t||b.__dt_wrapper)){var e,d,f,g=function(a,b,c){return function(){var d=b.apply(a,arguments);t.extend(d,d,c.methodExt);return d}};e=0;for(d=c.length;e<d;e++)f=c[e],b[f.name]="function"===typeof f.val?g(a,f.val,f):h.isPlainObject(f.val)?{}:f.val,b[f.name].__dt_wrapper=!0,t.extend(a,b[f.name],f.propExt)}};t.register=r=function(a,b){if(h.isArray(a))for(var c=0,e=a.length;c<
+e;c++)t.register(a[c],b);else for(var d=a.split("."),f=Tb,g,j,c=0,e=d.length;c<e;c++){g=(j=-1!==d[c].indexOf("()"))?d[c].replace("()",""):d[c];var i;a:{i=0;for(var o=f.length;i<o;i++)if(f[i].name===g){i=f[i];break a}i=null}i||(i={name:g,val:{},methodExt:[],propExt:[]},f.push(i));c===e-1?i.val=b:f=j?i.methodExt:i.propExt}};t.registerPlural=v=function(a,b,c){t.register(a,c);t.register(b,function(){var a=c.apply(this,arguments);return a===this?this:a instanceof t?a.length?h.isArray(a[0])?new t(a.context,
+a[0]):a[0]:k:a})};r("tables()",function(a){var b;if(a){b=t;var c=this.context;if("number"===typeof a)a=[c[a]];else var e=h.map(c,function(a){return a.nTable}),a=h(e).filter(a).map(function(){var a=h.inArray(this,e);return c[a]}).toArray();b=new b(a)}else b=this;return b});r("table()",function(a){var a=this.tables(a),b=a.context;return b.length?new t(b[0]):a});v("tables().nodes()","table().node()",function(){return this.iterator("table",function(a){return a.nTable},1)});v("tables().body()","table().body()",
+function(){return this.iterator("table",function(a){return a.nTBody},1)});v("tables().header()","table().header()",function(){return this.iterator("table",function(a){return a.nTHead},1)});v("tables().footer()","table().footer()",function(){return this.iterator("table",function(a){return a.nTFoot},1)});v("tables().containers()","table().container()",function(){return this.iterator("table",function(a){return a.nTableWrapper},1)});r("draw()",function(a){return this.iterator("table",function(b){N(b,
+!1===a)})});r("page()",function(a){return a===k?this.page.info().page:this.iterator("table",function(b){Ta(b,a)})});r("page.info()",function(){if(0===this.context.length)return k;var a=this.context[0],b=a._iDisplayStart,c=a._iDisplayLength,e=a.fnRecordsDisplay(),d=-1===c;return{page:d?0:Math.floor(b/c),pages:d?1:Math.ceil(e/c),start:b,end:a.fnDisplayEnd(),length:c,recordsTotal:a.fnRecordsTotal(),recordsDisplay:e}});r("page.len()",function(a){return a===k?0!==this.context.length?this.context[0]._iDisplayLength:
+k:this.iterator("table",function(b){Ra(b,a)})});var Ub=function(a,b,c){if(c){var e=new t(a);e.one("draw",function(){c(e.ajax.json())})}"ssp"==B(a)?N(a,b):(C(a,!0),ra(a,[],function(c){oa(a);for(var c=sa(a,c),e=0,g=c.length;e<g;e++)K(a,c[e]);N(a,b);C(a,!1)}))};r("ajax.json()",function(){var a=this.context;if(0<a.length)return a[0].json});r("ajax.params()",function(){var a=this.context;if(0<a.length)return a[0].oAjaxData});r("ajax.reload()",function(a,b){return this.iterator("table",function(c){Ub(c,
+!1===b,a)})});r("ajax.url()",function(a){var b=this.context;if(a===k){if(0===b.length)return k;b=b[0];return b.ajax?h.isPlainObject(b.ajax)?b.ajax.url:b.ajax:b.sAjaxSource}return this.iterator("table",function(b){h.isPlainObject(b.ajax)?b.ajax.url=a:b.ajax=a})});r("ajax.url().load()",function(a,b){return this.iterator("table",function(c){Ub(c,!1===b,a)})});var $a=function(a,b,c,e,d){var f=[],g,j,i,o,l,q;i=typeof b;if(!b||"string"===i||"function"===i||b.length===k)b=[b];i=0;for(o=b.length;i<o;i++){j=
+b[i]&&b[i].split?b[i].split(","):[b[i]];l=0;for(q=j.length;l<q;l++)(g=c("string"===typeof j[l]?h.trim(j[l]):j[l]))&&g.length&&f.push.apply(f,g)}a=u.selector[a];if(a.length){i=0;for(o=a.length;i<o;i++)f=a[i](e,d,f)}return f},ab=function(a){a||(a={});a.filter&&a.search===k&&(a.search=a.filter);return h.extend({search:"none",order:"current",page:"all"},a)},bb=function(a){for(var b=0,c=a.length;b<c;b++)if(0<a[b].length)return a[0]=a[b],a[0].length=1,a.length=1,a.context=[a.context[b]],a;a.length=0;return a},
+Ca=function(a,b){var c,e,d,f=[],g=a.aiDisplay;c=a.aiDisplayMaster;var j=b.search;e=b.order;d=b.page;if("ssp"==B(a))return"removed"===j?[]:V(0,c.length);if("current"==d){c=a._iDisplayStart;for(e=a.fnDisplayEnd();c<e;c++)f.push(g[c])}else if("current"==e||"applied"==e)f="none"==j?c.slice():"applied"==j?g.slice():h.map(c,function(a){return-1===h.inArray(a,g)?a:null});else if("index"==e||"original"==e){c=0;for(e=a.aoData.length;c<e;c++)"none"==j?f.push(c):(d=h.inArray(c,g),(-1===d&&"removed"==j||0<=d&&
+"applied"==j)&&f.push(c))}return f};r("rows()",function(a,b){a===k?a="":h.isPlainObject(a)&&(b=a,a="");var b=ab(b),c=this.iterator("table",function(c){var d=b;return $a("row",a,function(a){var b=Pb(a);if(b!==null&&!d)return[b];var j=Ca(c,d);if(b!==null&&h.inArray(b,j)!==-1)return[b];if(!a)return j;if(typeof a==="function")return h.map(j,function(b){var d=c.aoData[b];return a(b,d._aData,d.nTr)?b:null});b=Sb(ia(c.aoData,j,"nTr"));return a.nodeName&&h.inArray(a,b)!==-1?[a._DT_RowIndex]:h(b).filter(a).map(function(){return this._DT_RowIndex}).toArray()},
+c,d)},1);c.selector.rows=a;c.selector.opts=b;return c});r("rows().nodes()",function(){return this.iterator("row",function(a,b){return a.aoData[b].nTr||k},1)});r("rows().data()",function(){return this.iterator(!0,"rows",function(a,b){return ia(a.aoData,b,"_aData")},1)});v("rows().cache()","row().cache()",function(a){return this.iterator("row",function(b,c){var e=b.aoData[c];return"search"===a?e._aFilterData:e._aSortData},1)});v("rows().invalidate()","row().invalidate()",function(a){return this.iterator("row",
+function(b,c){ca(b,c,a)})});v("rows().indexes()","row().index()",function(){return this.iterator("row",function(a,b){return b},1)});v("rows().remove()","row().remove()",function(){var a=this;return this.iterator("row",function(b,c,e){var d=b.aoData;d.splice(c,1);for(var f=0,g=d.length;f<g;f++)null!==d[f].nTr&&(d[f].nTr._DT_RowIndex=f);h.inArray(c,b.aiDisplay);pa(b.aiDisplayMaster,c);pa(b.aiDisplay,c);pa(a[e],c,!1);Sa(b)})});r("rows.add()",function(a){var b=this.iterator("table",function(b){var c,
+f,g,h=[];f=0;for(g=a.length;f<g;f++)c=a[f],c.nodeName&&"TR"===c.nodeName.toUpperCase()?h.push(ma(b,c)[0]):h.push(K(b,c));return h},1),c=this.rows(-1);c.pop();c.push.apply(c,b.toArray());return c});r("row()",function(a,b){return bb(this.rows(a,b))});r("row().data()",function(a){var b=this.context;if(a===k)return b.length&&this.length?b[0].aoData[this[0]]._aData:k;b[0].aoData[this[0]]._aData=a;ca(b[0],this[0],"data");return this});r("row().node()",function(){var a=this.context;return a.length&&this.length?
+a[0].aoData[this[0]].nTr||null:null});r("row.add()",function(a){a instanceof h&&a.length&&(a=a[0]);var b=this.iterator("table",function(b){return a.nodeName&&"TR"===a.nodeName.toUpperCase()?ma(b,a)[0]:K(b,a)});return this.row(b[0])});var cb=function(a,b){var c=a.context;c.length&&(c=c[0].aoData[b!==k?b:a[0]],c._details&&(c._details.remove(),c._detailsShow=k,c._details=k))},Vb=function(a,b){var c=a.context;if(c.length&&a.length){var e=c[0].aoData[a[0]];if(e._details){(e._detailsShow=b)?e._details.insertAfter(e.nTr):
+e._details.detach();var d=c[0],f=new t(d),g=d.aoData;f.off("draw.dt.DT_details column-visibility.dt.DT_details destroy.dt.DT_details");0<D(g,"_details").length&&(f.on("draw.dt.DT_details",function(a,b){d===b&&f.rows({page:"current"}).eq(0).each(function(a){a=g[a];a._detailsShow&&a._details.insertAfter(a.nTr)})}),f.on("column-visibility.dt.DT_details",function(a,b){if(d===b)for(var c,e=aa(b),f=0,h=g.length;f<h;f++)c=g[f],c._details&&c._details.children("td[colspan]").attr("colspan",e)}),f.on("destroy.dt.DT_details",
+function(a,b){if(d===b)for(var c=0,e=g.length;c<e;c++)g[c]._details&&cb(f,c)}))}}};r("row().child()",function(a,b){var c=this.context;if(a===k)return c.length&&this.length?c[0].aoData[this[0]]._details:k;if(!0===a)this.child.show();else if(!1===a)cb(this);else if(c.length&&this.length){var e=c[0],c=c[0].aoData[this[0]],d=[],f=function(a,b){if(h.isArray(a)||a instanceof h)for(var c=0,k=a.length;c<k;c++)f(a[c],b);else a.nodeName&&"tr"===a.nodeName.toLowerCase()?d.push(a):(c=h("<tr><td/></tr>").addClass(b),
+h("td",c).addClass(b).html(a)[0].colSpan=aa(e),d.push(c[0]))};f(a,b);c._details&&c._details.remove();c._details=h(d);c._detailsShow&&c._details.insertAfter(c.nTr)}return this});r(["row().child.show()","row().child().show()"],function(){Vb(this,!0);return this});r(["row().child.hide()","row().child().hide()"],function(){Vb(this,!1);return this});r(["row().child.remove()","row().child().remove()"],function(){cb(this);return this});r("row().child.isShown()",function(){var a=this.context;return a.length&&
+this.length?a[0].aoData[this[0]]._detailsShow||!1:!1});var dc=/^(.+):(name|visIdx|visible)$/,Wb=function(a,b,c,e,d){for(var c=[],e=0,f=d.length;e<f;e++)c.push(x(a,d[e],b));return c};r("columns()",function(a,b){a===k?a="":h.isPlainObject(a)&&(b=a,a="");var b=ab(b),c=this.iterator("table",function(c){var d=a,f=b,g=c.aoColumns,j=D(g,"sName"),i=D(g,"nTh");return $a("column",d,function(a){var b=Pb(a);if(a==="")return V(g.length);if(b!==null)return[b>=0?b:g.length+b];if(typeof a==="function"){var d=Ca(c,
+f);return h.map(g,function(b,f){return a(f,Wb(c,f,0,0,d),i[f])?f:null})}var k=typeof a==="string"?a.match(dc):"";if(k)switch(k[2]){case "visIdx":case "visible":b=parseInt(k[1],10);if(b<0){var m=h.map(g,function(a,b){return a.bVisible?b:null});return[m[m.length+b]]}return[la(c,b)];case "name":return h.map(j,function(a,b){return a===k[1]?b:null})}else return h(i).filter(a).map(function(){return h.inArray(this,i)}).toArray()},c,f)},1);c.selector.cols=a;c.selector.opts=b;return c});v("columns().header()",
+"column().header()",function(){return this.iterator("column",function(a,b){return a.aoColumns[b].nTh},1)});v("columns().footer()","column().footer()",function(){return this.iterator("column",function(a,b){return a.aoColumns[b].nTf},1)});v("columns().data()","column().data()",function(){return this.iterator("column-rows",Wb,1)});v("columns().dataSrc()","column().dataSrc()",function(){return this.iterator("column",function(a,b){return a.aoColumns[b].mData},1)});v("columns().cache()","column().cache()",
+function(a){return this.iterator("column-rows",function(b,c,e,d,f){return ia(b.aoData,f,"search"===a?"_aFilterData":"_aSortData",c)},1)});v("columns().nodes()","column().nodes()",function(){return this.iterator("column-rows",function(a,b,c,e,d){return ia(a.aoData,d,"anCells",b)},1)});v("columns().visible()","column().visible()",function(a,b){return this.iterator("column",function(c,e){if(a===k)return c.aoColumns[e].bVisible;var d=c.aoColumns,f=d[e],g=c.aoData,j,i,m;if(a!==k&&f.bVisible!==a){if(a){var l=
+h.inArray(!0,D(d,"bVisible"),e+1);j=0;for(i=g.length;j<i;j++)m=g[j].nTr,d=g[j].anCells,m&&m.insertBefore(d[e],d[l]||null)}else h(D(c.aoData,"anCells",e)).detach();f.bVisible=a;ea(c,c.aoHeader);ea(c,c.aoFooter);if(b===k||b)X(c),(c.oScroll.sX||c.oScroll.sY)&&Y(c);w(c,null,"column-visibility",[c,e,a]);ya(c)}})});v("columns().indexes()","column().index()",function(a){return this.iterator("column",function(b,c){return"visible"===a?$(b,c):c},1)});r("columns.adjust()",function(){return this.iterator("table",
+function(a){X(a)},1)});r("column.index()",function(a,b){if(0!==this.context.length){var c=this.context[0];if("fromVisible"===a||"toData"===a)return la(c,b);if("fromData"===a||"toVisible"===a)return $(c,b)}});r("column()",function(a,b){return bb(this.columns(a,b))});r("cells()",function(a,b,c){h.isPlainObject(a)&&(a.row===k?(c=a,a=null):(c=b,b=null));h.isPlainObject(b)&&(c=b,b=null);if(null===b||b===k)return this.iterator("table",function(b){var d=a,e=ab(c),f=b.aoData,g=Ca(b,e),i=Sb(ia(f,g,"anCells")),
+j=h([].concat.apply([],i)),l,m=b.aoColumns.length,o,r,t,s,u,v;return $a("cell",d,function(a){var c=typeof a==="function";if(a===null||a===k||c){o=[];r=0;for(t=g.length;r<t;r++){l=g[r];for(s=0;s<m;s++){u={row:l,column:s};if(c){v=b.aoData[l];a(u,x(b,l,s),v.anCells?v.anCells[s]:null)&&o.push(u)}else o.push(u)}}return o}return h.isPlainObject(a)?[a]:j.filter(a).map(function(a,b){l=b.parentNode._DT_RowIndex;return{row:l,column:h.inArray(b,f[l].anCells)}}).toArray()},b,e)});var e=this.columns(b,c),d=this.rows(a,
+c),f,g,j,i,m,l=this.iterator("table",function(a,b){f=[];g=0;for(j=d[b].length;g<j;g++){i=0;for(m=e[b].length;i<m;i++)f.push({row:d[b][g],column:e[b][i]})}return f},1);h.extend(l.selector,{cols:b,rows:a,opts:c});return l});v("cells().nodes()","cell().node()",function(){return this.iterator("cell",function(a,b,c){return(a=a.aoData[b].anCells)?a[c]:k},1)});r("cells().data()",function(){return this.iterator("cell",function(a,b,c){return x(a,b,c)},1)});v("cells().cache()","cell().cache()",function(a){a=
+"search"===a?"_aFilterData":"_aSortData";return this.iterator("cell",function(b,c,e){return b.aoData[c][a][e]},1)});v("cells().render()","cell().render()",function(a){return this.iterator("cell",function(b,c,e){return x(b,c,e,a)},1)});v("cells().indexes()","cell().index()",function(){return this.iterator("cell",function(a,b,c){return{row:b,column:c,columnVisible:$(a,c)}},1)});v("cells().invalidate()","cell().invalidate()",function(a){return this.iterator("cell",function(b,c,e){ca(b,c,a,e)})});r("cell()",
+function(a,b,c){return bb(this.cells(a,b,c))});r("cell().data()",function(a){var b=this.context,c=this[0];if(a===k)return b.length&&c.length?x(b[0],c[0].row,c[0].column):k;Ia(b[0],c[0].row,c[0].column,a);ca(b[0],c[0].row,"data",c[0].column);return this});r("order()",function(a,b){var c=this.context;if(a===k)return 0!==c.length?c[0].aaSorting:k;"number"===typeof a?a=[[a,b]]:h.isArray(a[0])||(a=Array.prototype.slice.call(arguments));return this.iterator("table",function(b){b.aaSorting=a.slice()})});
+r("order.listener()",function(a,b,c){return this.iterator("table",function(e){Oa(e,a,b,c)})});r(["columns().order()","column().order()"],function(a){var b=this;return this.iterator("table",function(c,e){var d=[];h.each(b[e],function(b,c){d.push([c,a])});c.aaSorting=d})});r("search()",function(a,b,c,e){var d=this.context;return a===k?0!==d.length?d[0].oPreviousSearch.sSearch:k:this.iterator("table",function(d){d.oFeatures.bFilter&&fa(d,h.extend({},d.oPreviousSearch,{sSearch:a+"",bRegex:null===b?!1:
+b,bSmart:null===c?!0:c,bCaseInsensitive:null===e?!0:e}),1)})});v("columns().search()","column().search()",function(a,b,c,e){return this.iterator("column",function(d,f){var g=d.aoPreSearchCols;if(a===k)return g[f].sSearch;d.oFeatures.bFilter&&(h.extend(g[f],{sSearch:a+"",bRegex:null===b?!1:b,bSmart:null===c?!0:c,bCaseInsensitive:null===e?!0:e}),fa(d,d.oPreviousSearch,1))})});r("state()",function(){return this.context.length?this.context[0].oSavedState:null});r("state.clear()",function(){return this.iterator("table",
+function(a){a.fnStateSaveCallback.call(a.oInstance,a,{})})});r("state.loaded()",function(){return this.context.length?this.context[0].oLoadedState:null});r("state.save()",function(){return this.iterator("table",function(a){ya(a)})});m.versionCheck=m.fnVersionCheck=function(a){for(var b=m.version.split("."),a=a.split("."),c,e,d=0,f=a.length;d<f;d++)if(c=parseInt(b[d],10)||0,e=parseInt(a[d],10)||0,c!==e)return c>e;return!0};m.isDataTable=m.fnIsDataTable=function(a){var b=h(a).get(0),c=!1;h.each(m.settings,
+function(a,d){var f=d.nScrollHead?h("table",d.nScrollHead)[0]:null,g=d.nScrollFoot?h("table",d.nScrollFoot)[0]:null;if(d.nTable===b||f===b||g===b)c=!0});return c};m.tables=m.fnTables=function(a){return h.map(m.settings,function(b){if(!a||a&&h(b.nTable).is(":visible"))return b.nTable})};m.util={throttle:ua,escapeRegex:va};m.camelToHungarian=H;r("$()",function(a,b){var c=this.rows(b).nodes(),c=h(c);return h([].concat(c.filter(a).toArray(),c.find(a).toArray()))});h.each(["on","one","off"],function(a,
+b){r(b+"()",function(){var a=Array.prototype.slice.call(arguments);a[0].match(/\.dt\b/)||(a[0]+=".dt");var e=h(this.tables().nodes());e[b].apply(e,a);return this})});r("clear()",function(){return this.iterator("table",function(a){oa(a)})});r("settings()",function(){return new t(this.context,this.context)});r("init()",function(){var a=this.context;return a.length?a[0].oInit:null});r("data()",function(){return this.iterator("table",function(a){return D(a.aoData,"_aData")}).flatten()});r("destroy()",
+function(a){a=a||!1;return this.iterator("table",function(b){var c=b.nTableWrapper.parentNode,e=b.oClasses,d=b.nTable,f=b.nTBody,g=b.nTHead,j=b.nTFoot,i=h(d),f=h(f),k=h(b.nTableWrapper),l=h.map(b.aoData,function(a){return a.nTr}),q;b.bDestroying=!0;w(b,"aoDestroyCallback","destroy",[b]);a||(new t(b)).columns().visible(!0);k.unbind(".DT").find(":not(tbody *)").unbind(".DT");h(Ea).unbind(".DT-"+b.sInstance);d!=g.parentNode&&(i.children("thead").detach(),i.append(g));j&&d!=j.parentNode&&(i.children("tfoot").detach(),
+i.append(j));i.detach();k.detach();b.aaSorting=[];b.aaSortingFixed=[];xa(b);h(l).removeClass(b.asStripeClasses.join(" "));h("th, td",g).removeClass(e.sSortable+" "+e.sSortableAsc+" "+e.sSortableDesc+" "+e.sSortableNone);b.bJUI&&(h("th span."+e.sSortIcon+", td span."+e.sSortIcon,g).detach(),h("th, td",g).each(function(){var a=h("div."+e.sSortJUIWrapper,this);h(this).append(a.contents());a.detach()}));!a&&c&&c.insertBefore(d,b.nTableReinsertBefore);f.children().detach();f.append(l);i.css("width",b.sDestroyWidth).removeClass(e.sTable);
+(q=b.asDestroyStripes.length)&&f.children().each(function(a){h(this).addClass(b.asDestroyStripes[a%q])});c=h.inArray(b,m.settings);-1!==c&&m.settings.splice(c,1)})});h.each(["column","row","cell"],function(a,b){r(b+"s().every()",function(a){return this.iterator(b,function(e,d,f){a.call((new t(e))[b](d,f))})})});r("i18n()",function(a,b,c){var e=this.context[0],a=R(a)(e.oLanguage);a===k&&(a=b);c!==k&&h.isPlainObject(a)&&(a=a[c]!==k?a[c]:a._);return a.replace("%d",c)});m.version="1.10.7";m.settings=
+[];m.models={};m.models.oSearch={bCaseInsensitive:!0,sSearch:"",bRegex:!1,bSmart:!0};m.models.oRow={nTr:null,anCells:null,_aData:[],_aSortData:null,_aFilterData:null,_sFilterRow:null,_sRowStripe:"",src:null};m.models.oColumn={idx:null,aDataSort:null,asSorting:null,bSearchable:null,bSortable:null,bVisible:null,_sManualType:null,_bAttrSrc:!1,fnCreatedCell:null,fnGetData:null,fnSetData:null,mData:null,mRender:null,nTh:null,nTf:null,sClass:null,sContentPadding:null,sDefaultContent:null,sName:null,sSortDataType:"std",
+sSortingClass:null,sSortingClassJUI:null,sTitle:null,sType:null,sWidth:null,sWidthOrig:null};m.defaults={aaData:null,aaSorting:[[0,"asc"]],aaSortingFixed:[],ajax:null,aLengthMenu:[10,25,50,100],aoColumns:null,aoColumnDefs:null,aoSearchCols:[],asStripeClasses:null,bAutoWidth:!0,bDeferRender:!1,bDestroy:!1,bFilter:!0,bInfo:!0,bJQueryUI:!1,bLengthChange:!0,bPaginate:!0,bProcessing:!1,bRetrieve:!1,bScrollCollapse:!1,bServerSide:!1,bSort:!0,bSortMulti:!0,bSortCellsTop:!1,bSortClasses:!0,bStateSave:!1,
+fnCreatedRow:null,fnDrawCallback:null,fnFooterCallback:null,fnFormatNumber:function(a){return a.toString().replace(/\B(?=(\d{3})+(?!\d))/g,this.oLanguage.sThousands)},fnHeaderCallback:null,fnInfoCallback:null,fnInitComplete:null,fnPreDrawCallback:null,fnRowCallback:null,fnServerData:null,fnServerParams:null,fnStateLoadCallback:function(a){try{return JSON.parse((-1===a.iStateDuration?sessionStorage:localStorage).getItem("DataTables_"+a.sInstance+"_"+location.pathname))}catch(b){}},fnStateLoadParams:null,
+fnStateLoaded:null,fnStateSaveCallback:function(a,b){try{(-1===a.iStateDuration?sessionStorage:localStorage).setItem("DataTables_"+a.sInstance+"_"+location.pathname,JSON.stringify(b))}catch(c){}},fnStateSaveParams:null,iStateDuration:7200,iDeferLoading:null,iDisplayLength:10,iDisplayStart:0,iTabIndex:0,oClasses:{},oLanguage:{oAria:{sSortAscending:": activate to sort column ascending",sSortDescending:": activate to sort column descending"},oPaginate:{sFirst:"First",sLast:"Last",sNext:"Next",sPrevious:"Previous"},
+sEmptyTable:"No data available in table",sInfo:"Showing _START_ to _END_ of _TOTAL_ entries",sInfoEmpty:"Showing 0 to 0 of 0 entries",sInfoFiltered:"(filtered from _MAX_ total entries)",sInfoPostFix:"",sDecimal:"",sThousands:",",sLengthMenu:"Show _MENU_ entries",sLoadingRecords:"Loading...",sProcessing:"Processing...",sSearch:"Search:",sSearchPlaceholder:"",sUrl:"",sZeroRecords:"No matching records found"},oSearch:h.extend({},m.models.oSearch),sAjaxDataProp:"data",sAjaxSource:null,sDom:"lfrtip",searchDelay:null,
+sPaginationType:"simple_numbers",sScrollX:"",sScrollXInner:"",sScrollY:"",sServerMethod:"GET",renderer:null};W(m.defaults);m.defaults.column={aDataSort:null,iDataSort:-1,asSorting:["asc","desc"],bSearchable:!0,bSortable:!0,bVisible:!0,fnCreatedCell:null,mData:null,mRender:null,sCellType:"td",sClass:"",sContentPadding:"",sDefaultContent:null,sName:"",sSortDataType:"std",sTitle:null,sType:null,sWidth:null};W(m.defaults.column);m.models.oSettings={oFeatures:{bAutoWidth:null,bDeferRender:null,bFilter:null,
+bInfo:null,bLengthChange:null,bPaginate:null,bProcessing:null,bServerSide:null,bSort:null,bSortMulti:null,bSortClasses:null,bStateSave:null},oScroll:{bCollapse:null,iBarWidth:0,sX:null,sXInner:null,sY:null},oLanguage:{fnInfoCallback:null},oBrowser:{bScrollOversize:!1,bScrollbarLeft:!1},ajax:null,aanFeatures:[],aoData:[],aiDisplay:[],aiDisplayMaster:[],aoColumns:[],aoHeader:[],aoFooter:[],oPreviousSearch:{},aoPreSearchCols:[],aaSorting:null,aaSortingFixed:[],asStripeClasses:null,asDestroyStripes:[],
+sDestroyWidth:0,aoRowCallback:[],aoHeaderCallback:[],aoFooterCallback:[],aoDrawCallback:[],aoRowCreatedCallback:[],aoPreDrawCallback:[],aoInitComplete:[],aoStateSaveParams:[],aoStateLoadParams:[],aoStateLoaded:[],sTableId:"",nTable:null,nTHead:null,nTFoot:null,nTBody:null,nTableWrapper:null,bDeferLoading:!1,bInitialised:!1,aoOpenRows:[],sDom:null,searchDelay:null,sPaginationType:"two_button",iStateDuration:0,aoStateSave:[],aoStateLoad:[],oSavedState:null,oLoadedState:null,sAjaxSource:null,sAjaxDataProp:null,
+bAjaxDataGet:!0,jqXHR:null,json:k,oAjaxData:k,fnServerData:null,aoServerParams:[],sServerMethod:null,fnFormatNumber:null,aLengthMenu:null,iDraw:0,bDrawing:!1,iDrawError:-1,_iDisplayLength:10,_iDisplayStart:0,_iRecordsTotal:0,_iRecordsDisplay:0,bJUI:null,oClasses:{},bFiltered:!1,bSorted:!1,bSortCellsTop:null,oInit:null,aoDestroyCallback:[],fnRecordsTotal:function(){return"ssp"==B(this)?1*this._iRecordsTotal:this.aiDisplayMaster.length},fnRecordsDisplay:function(){return"ssp"==B(this)?1*this._iRecordsDisplay:
+this.aiDisplay.length},fnDisplayEnd:function(){var a=this._iDisplayLength,b=this._iDisplayStart,c=b+a,e=this.aiDisplay.length,d=this.oFeatures,f=d.bPaginate;return d.bServerSide?!1===f||-1===a?b+e:Math.min(b+a,this._iRecordsDisplay):!f||c>e||-1===a?e:c},oInstance:null,sInstance:null,iTabIndex:0,nScrollHead:null,nScrollFoot:null,aLastSort:[],oPlugins:{}};m.ext=u={buttons:{},classes:{},errMode:"alert",feature:[],search:[],selector:{cell:[],column:[],row:[]},internal:{},legacy:{ajax:null},pager:{},renderer:{pageButton:{},
+header:{}},order:{},type:{detect:[],search:{},order:{}},_unique:0,fnVersionCheck:m.fnVersionCheck,iApiIndex:0,oJUIClasses:{},sVersion:m.version};h.extend(u,{afnFiltering:u.search,aTypes:u.type.detect,ofnSearch:u.type.search,oSort:u.type.order,afnSortData:u.order,aoFeatures:u.feature,oApi:u.internal,oStdClasses:u.classes,oPagination:u.pager});h.extend(m.ext.classes,{sTable:"dataTable",sNoFooter:"no-footer",sPageButton:"paginate_button",sPageButtonActive:"current",sPageButtonDisabled:"disabled",sStripeOdd:"odd",
+sStripeEven:"even",sRowEmpty:"dataTables_empty",sWrapper:"dataTables_wrapper",sFilter:"dataTables_filter",sInfo:"dataTables_info",sPaging:"dataTables_paginate paging_",sLength:"dataTables_length",sProcessing:"dataTables_processing",sSortAsc:"sorting_asc",sSortDesc:"sorting_desc",sSortable:"sorting",sSortableAsc:"sorting_asc_disabled",sSortableDesc:"sorting_desc_disabled",sSortableNone:"sorting_disabled",sSortColumn:"sorting_",sFilterInput:"",sLengthSelect:"",sScrollWrapper:"dataTables_scroll",sScrollHead:"dataTables_scrollHead",
+sScrollHeadInner:"dataTables_scrollHeadInner",sScrollBody:"dataTables_scrollBody",sScrollFoot:"dataTables_scrollFoot",sScrollFootInner:"dataTables_scrollFootInner",sHeaderTH:"",sFooterTH:"",sSortJUIAsc:"",sSortJUIDesc:"",sSortJUI:"",sSortJUIAscAllowed:"",sSortJUIDescAllowed:"",sSortJUIWrapper:"",sSortIcon:"",sJUIHeader:"",sJUIFooter:""});var Da="",Da="",F=Da+"ui-state-default",ja=Da+"css_right ui-icon ui-icon-",Xb=Da+"fg-toolbar ui-toolbar ui-widget-header ui-helper-clearfix";h.extend(m.ext.oJUIClasses,
+m.ext.classes,{sPageButton:"fg-button ui-button "+F,sPageButtonActive:"ui-state-disabled",sPageButtonDisabled:"ui-state-disabled",sPaging:"dataTables_paginate fg-buttonset ui-buttonset fg-buttonset-multi ui-buttonset-multi paging_",sSortAsc:F+" sorting_asc",sSortDesc:F+" sorting_desc",sSortable:F+" sorting",sSortableAsc:F+" sorting_asc_disabled",sSortableDesc:F+" sorting_desc_disabled",sSortableNone:F+" sorting_disabled",sSortJUIAsc:ja+"triangle-1-n",sSortJUIDesc:ja+"triangle-1-s",sSortJUI:ja+"carat-2-n-s",
+sSortJUIAscAllowed:ja+"carat-1-n",sSortJUIDescAllowed:ja+"carat-1-s",sSortJUIWrapper:"DataTables_sort_wrapper",sSortIcon:"DataTables_sort_icon",sScrollHead:"dataTables_scrollHead "+F,sScrollFoot:"dataTables_scrollFoot "+F,sHeaderTH:F,sFooterTH:F,sJUIHeader:Xb+" ui-corner-tl ui-corner-tr",sJUIFooter:Xb+" ui-corner-bl ui-corner-br"});var Mb=m.ext.pager;h.extend(Mb,{simple:function(){return["previous","next"]},full:function(){return["first","previous","next","last"]},simple_numbers:function(a,b){return["previous",
+Wa(a,b),"next"]},full_numbers:function(a,b){return["first","previous",Wa(a,b),"next","last"]},_numbers:Wa,numbers_length:7});h.extend(!0,m.ext.renderer,{pageButton:{_:function(a,b,c,e,d,f){var g=a.oClasses,j=a.oLanguage.oPaginate,i,k,l=0,m=function(b,e){var n,r,t,s,u=function(b){Ta(a,b.data.action,true)};n=0;for(r=e.length;n<r;n++){s=e[n];if(h.isArray(s)){t=h("<"+(s.DT_el||"div")+"/>").appendTo(b);m(t,s)}else{k=i="";switch(s){case "ellipsis":b.append('<span class="ellipsis">&#x2026;</span>');break;
+case "first":i=j.sFirst;k=s+(d>0?"":" "+g.sPageButtonDisabled);break;case "previous":i=j.sPrevious;k=s+(d>0?"":" "+g.sPageButtonDisabled);break;case "next":i=j.sNext;k=s+(d<f-1?"":" "+g.sPageButtonDisabled);break;case "last":i=j.sLast;k=s+(d<f-1?"":" "+g.sPageButtonDisabled);break;default:i=s+1;k=d===s?g.sPageButtonActive:""}if(i){t=h("<a>",{"class":g.sPageButton+" "+k,"aria-controls":a.sTableId,"data-dt-idx":l,tabindex:a.iTabIndex,id:c===0&&typeof s==="string"?a.sTableId+"_"+s:null}).html(i).appendTo(b);
+Va(t,{action:s},u);l++}}}},n;try{n=h(Q.activeElement).data("dt-idx")}catch(r){}m(h(b).empty(),e);n&&h(b).find("[data-dt-idx="+n+"]").focus()}}});h.extend(m.ext.type.detect,[function(a,b){var c=b.oLanguage.sDecimal;return Za(a,c)?"num"+c:null},function(a){if(a&&!(a instanceof Date)&&(!ac.test(a)||!bc.test(a)))return null;var b=Date.parse(a);return null!==b&&!isNaN(b)||J(a)?"date":null},function(a,b){var c=b.oLanguage.sDecimal;return Za(a,c,!0)?"num-fmt"+c:null},function(a,b){var c=b.oLanguage.sDecimal;
+return Rb(a,c)?"html-num"+c:null},function(a,b){var c=b.oLanguage.sDecimal;return Rb(a,c,!0)?"html-num-fmt"+c:null},function(a){return J(a)||"string"===typeof a&&-1!==a.indexOf("<")?"html":null}]);h.extend(m.ext.type.search,{html:function(a){return J(a)?a:"string"===typeof a?a.replace(Ob," ").replace(Ba,""):""},string:function(a){return J(a)?a:"string"===typeof a?a.replace(Ob," "):a}});var Aa=function(a,b,c,e){if(0!==a&&(!a||"-"===a))return-Infinity;b&&(a=Qb(a,b));a.replace&&(c&&(a=a.replace(c,"")),
+e&&(a=a.replace(e,"")));return 1*a};h.extend(u.type.order,{"date-pre":function(a){return Date.parse(a)||0},"html-pre":function(a){return J(a)?"":a.replace?a.replace(/<.*?>/g,"").toLowerCase():a+""},"string-pre":function(a){return J(a)?"":"string"===typeof a?a.toLowerCase():!a.toString?"":a.toString()},"string-asc":function(a,b){return a<b?-1:a>b?1:0},"string-desc":function(a,b){return a<b?1:a>b?-1:0}});db("");h.extend(!0,m.ext.renderer,{header:{_:function(a,b,c,e){h(a.nTable).on("order.dt.DT",function(d,
+f,g,h){if(a===f){d=c.idx;b.removeClass(c.sSortingClass+" "+e.sSortAsc+" "+e.sSortDesc).addClass(h[d]=="asc"?e.sSortAsc:h[d]=="desc"?e.sSortDesc:c.sSortingClass)}})},jqueryui:function(a,b,c,e){h("<div/>").addClass(e.sSortJUIWrapper).append(b.contents()).append(h("<span/>").addClass(e.sSortIcon+" "+c.sSortingClassJUI)).appendTo(b);h(a.nTable).on("order.dt.DT",function(d,f,g,h){if(a===f){d=c.idx;b.removeClass(e.sSortAsc+" "+e.sSortDesc).addClass(h[d]=="asc"?e.sSortAsc:h[d]=="desc"?e.sSortDesc:c.sSortingClass);
+b.find("span."+e.sSortIcon).removeClass(e.sSortJUIAsc+" "+e.sSortJUIDesc+" "+e.sSortJUI+" "+e.sSortJUIAscAllowed+" "+e.sSortJUIDescAllowed).addClass(h[d]=="asc"?e.sSortJUIAsc:h[d]=="desc"?e.sSortJUIDesc:c.sSortingClassJUI)}})}}});m.render={number:function(a,b,c,e){return{display:function(d){if("number"!==typeof d&&"string"!==typeof d)return d;var f=0>d?"-":"",d=Math.abs(parseFloat(d)),g=parseInt(d,10),d=c?b+(d-g).toFixed(c).substring(2):"";return f+(e||"")+g.toString().replace(/\B(?=(\d{3})+(?!\d))/g,
+a)+d}}}};h.extend(m.ext.internal,{_fnExternApiFunc:Nb,_fnBuildAjax:ra,_fnAjaxUpdate:kb,_fnAjaxParameters:tb,_fnAjaxUpdateDraw:ub,_fnAjaxDataSrc:sa,_fnAddColumn:Fa,_fnColumnOptions:ka,_fnAdjustColumnSizing:X,_fnVisibleToColumnIndex:la,_fnColumnIndexToVisible:$,_fnVisbleColumns:aa,_fnGetColumns:Z,_fnColumnTypes:Ha,_fnApplyColumnDefs:ib,_fnHungarianMap:W,_fnCamelToHungarian:H,_fnLanguageCompat:P,_fnBrowserDetect:gb,_fnAddData:K,_fnAddTr:ma,_fnNodeToDataIndex:function(a,b){return b._DT_RowIndex!==k?b._DT_RowIndex:
+null},_fnNodeToColumnIndex:function(a,b,c){return h.inArray(c,a.aoData[b].anCells)},_fnGetCellData:x,_fnSetCellData:Ia,_fnSplitObjNotation:Ka,_fnGetObjectDataFn:R,_fnSetObjectDataFn:S,_fnGetDataMaster:La,_fnClearTable:oa,_fnDeleteIndex:pa,_fnInvalidate:ca,_fnGetRowElements:na,_fnCreateTr:Ja,_fnBuildHead:jb,_fnDrawHead:ea,_fnDraw:M,_fnReDraw:N,_fnAddOptionsHtml:mb,_fnDetectHeader:da,_fnGetUniqueThs:qa,_fnFeatureHtmlFilter:ob,_fnFilterComplete:fa,_fnFilterCustom:xb,_fnFilterColumn:wb,_fnFilter:vb,_fnFilterCreateSearch:Qa,
+_fnEscapeRegex:va,_fnFilterData:yb,_fnFeatureHtmlInfo:rb,_fnUpdateInfo:Bb,_fnInfoMacros:Cb,_fnInitialise:ga,_fnInitComplete:ta,_fnLengthChange:Ra,_fnFeatureHtmlLength:nb,_fnFeatureHtmlPaginate:sb,_fnPageChange:Ta,_fnFeatureHtmlProcessing:pb,_fnProcessingDisplay:C,_fnFeatureHtmlTable:qb,_fnScrollDraw:Y,_fnApplyToChildren:G,_fnCalculateColumnWidths:Ga,_fnThrottle:ua,_fnConvertToWidth:Db,_fnScrollingWidthAdjust:Fb,_fnGetWidestNode:Eb,_fnGetMaxLenString:Gb,_fnStringToCss:s,_fnScrollBarWidth:Hb,_fnSortFlatten:U,
+_fnSort:lb,_fnSortAria:Jb,_fnSortListener:Ua,_fnSortAttachListener:Oa,_fnSortingClasses:xa,_fnSortData:Ib,_fnSaveState:ya,_fnLoadState:Kb,_fnSettingsFromNode:za,_fnLog:I,_fnMap:E,_fnBindAction:Va,_fnCallbackReg:z,_fnCallbackFire:w,_fnLengthOverflow:Sa,_fnRenderer:Pa,_fnDataSource:B,_fnRowAttributes:Ma,_fnCalculateEnd:function(){}});h.fn.dataTable=m;h.fn.dataTableSettings=m.settings;h.fn.dataTableExt=m.ext;h.fn.DataTable=function(a){return h(this).dataTable(a).api()};h.each(m,function(a,b){h.fn.DataTable[a]=
+b});return h.fn.dataTable};"function"===typeof define&&define.amd?define("datatables",["jquery"],P):"object"===typeof exports?module.exports=P(require("jquery")):jQuery&&!jQuery.fn.dataTable&&P(jQuery)})(window,document);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/demo_page.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/demo_page.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/demo_page.css
deleted file mode 100644
index b60ee7d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/demo_page.css
+++ /dev/null
@@ -1,110 +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.
-*/
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * General page setup
- */
-#dt_example {
-	font: 80%/1.45em "Lucida Grande", Verdana, Arial, Helvetica, sans-serif;
-	margin: 0;
-	padding: 0;
-	color: #333;
-	background-color: #fff;
-}
-
-
-#dt_example #container {
-	width: 800px;
-	margin: 30px auto;
-	padding: 0;
-}
-
-
-#dt_example #footer {
-	margin: 50px auto 0 auto;
-	padding: 0;
-}
-
-#dt_example #demo {
-	margin: 30px auto 0 auto;
-}
-
-#dt_example .demo_jui {
-	margin: 30px auto 0 auto;
-}
-
-#dt_example .big {
-	font-size: 1.3em;
-	font-weight: bold;
-	line-height: 1.6em;
-	color: #4E6CA3;
-}
-
-#dt_example .spacer {
-	height: 20px;
-	clear: both;
-}
-
-#dt_example .clear {
-	clear: both;
-}
-
-#dt_example pre {
-	padding: 15px;
-	background-color: #F5F5F5;
-	border: 1px solid #CCCCCC;
-}
-
-#dt_example h1 {
-	margin-top: 2em;
-	font-size: 1.3em;
-	font-weight: normal;
-	line-height: 1.6em;
-	color: #4E6CA3;
-	border-bottom: 1px solid #B0BED9;
-	clear: both;
-}
-
-#dt_example h2 {
-	font-size: 1.2em;
-	font-weight: normal;
-	line-height: 1.6em;
-	color: #4E6CA3;
-	clear: both;
-}
-
-#dt_example a {
-	color: #0063DC;
-	text-decoration: none;
-}
-
-#dt_example a:hover {
-	text-decoration: underline;
-}
-
-#dt_example ul {
-	color: #4E6CA3;
-}
-
-.css_right {
-	float: right;
-}
-
-.css_left {
-	float: left;
-}


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


[19/50] [abbrv] hadoop git commit: HDDS-344. Remove multibyte characters from OzoneAcl. Contributed by Takanobu Asanuma.

Posted by tm...@apache.org.
HDDS-344. Remove multibyte characters from OzoneAcl. Contributed by Takanobu Asanuma.


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

Branch: refs/heads/HADOOP-15407
Commit: 778369ea0204e75ce86fc7da3321b046f8139d9a
Parents: 3d96bc6
Author: Márton Elek <el...@apache.org>
Authored: Thu Aug 9 14:23:41 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Thu Aug 9 14:26:37 2018 +0200

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/ozone/OzoneAcl.java      | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/778369ea/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
index ff0ac4e..1827b23 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
@@ -25,9 +25,11 @@ import java.util.Objects;
  * OzoneACL classes define bucket ACLs used in OZONE.
  *
  * ACLs in Ozone follow this pattern.
- * • user:name:rw
- * • group:name:rw
- * • world::rw
+ * <ul>
+ * <li>user:name:rw
+ * <li>group:name:rw
+ * <li>world::rw
+ * </ul>
  */
 public class OzoneAcl {
   private OzoneACLType type;


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


[28/50] [abbrv] hadoop git commit: YARN-8575. Avoid committing allocation proposal to unavailable nodes in async scheduling. Contributed by Tao Yang.

Posted by tm...@apache.org.
YARN-8575. Avoid committing allocation proposal to unavailable nodes in async scheduling. Contributed by Tao Yang.


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

Branch: refs/heads/HADOOP-15407
Commit: 0a71bf145293adbd3728525ab4c36c08d51377d3
Parents: 08d5060
Author: Weiwei Yang <ww...@apache.org>
Authored: Fri Aug 10 14:37:45 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Fri Aug 10 14:37:45 2018 +0800

----------------------------------------------------------------------
 .../scheduler/common/fica/FiCaSchedulerApp.java | 12 ++++
 .../yarn/server/resourcemanager/MockNodes.java  |  6 +-
 .../resourcemanager/TestResourceManager.java    | 16 ++++-
 .../TestCapacitySchedulerAsyncScheduling.java   | 69 ++++++++++++++++++++
 .../scheduler/capacity/TestUtils.java           |  2 +
 5 files changed, 100 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a71bf14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.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/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 9810e98..6a5af81 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -429,6 +430,17 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
         SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>
             schedulerContainer = allocation.getAllocatedOrReservedContainer();
 
+        // Make sure node is in RUNNING state
+        if (schedulerContainer.getSchedulerNode().getRMNode().getState()
+            != NodeState.RUNNING) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Failed to accept this proposal because node "
+                + schedulerContainer.getSchedulerNode().getNodeID() + " is in "
+                + schedulerContainer.getSchedulerNode().getRMNode().getState()
+                + " state (not RUNNING)");
+          }
+          return false;
+        }
         if (schedulerContainer.isAllocated()) {
           // When allocate a new container
           containerRequest =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a71bf14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.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/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index 9041132..c444b6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -347,17 +347,17 @@ public class MockNodes {
   }
 
   public static RMNode newNodeInfo(int rack, final Resource perNode, int hostnum) {
-    return buildRMNode(rack, perNode, null, "localhost:0", hostnum, null, 123);
+    return buildRMNode(rack, perNode, NodeState.RUNNING, "localhost:0", hostnum, null, 123);
   }
   
   public static RMNode newNodeInfo(int rack, final Resource perNode,
       int hostnum, String hostName) {
-    return buildRMNode(rack, perNode, null, "localhost:0", hostnum, hostName, 123);
+    return buildRMNode(rack, perNode, NodeState.RUNNING, "localhost:0", hostnum, hostName, 123);
   }
 
   public static RMNode newNodeInfo(int rack, final Resource perNode,
       int hostnum, String hostName, int port) {
-    return buildRMNode(rack, perNode, null, "localhost:0", hostnum, hostName, port);
+    return buildRMNode(rack, perNode, NodeState.RUNNING, "localhost:0", hostnum, hostName, port);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a71bf14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.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/TestResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
index 941e477..a66c583 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -39,6 +40,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStartedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
@@ -86,8 +89,9 @@ public class TestResourceManager {
   }
 
   @Test
-  public void testResourceAllocation() throws IOException,
-      YarnException, InterruptedException {
+  public void testResourceAllocation()
+      throws IOException, YarnException, InterruptedException,
+      TimeoutException {
     LOG.info("--- START: testResourceAllocation ---");
         
     final int memory = 4 * 1024;
@@ -105,6 +109,14 @@ public class TestResourceManager {
       registerNode(host2, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
           Resources.createResource(memory/2, vcores/2));
 
+    // nodes should be in RUNNING state
+    RMNodeImpl node1 = (RMNodeImpl) resourceManager.getRMContext().getRMNodes().get(
+        nm1.getNodeId());
+    RMNodeImpl node2 = (RMNodeImpl) resourceManager.getRMContext().getRMNodes().get(
+        nm2.getNodeId());
+    node1.handle(new RMNodeStartedEvent(nm1.getNodeId(), null, null));
+    node2.handle(new RMNodeStartedEvent(nm2.getNodeId(), null, null));
+
     // Submit an application
     Application application = new Application("user1", resourceManager);
     application.submit();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a71bf14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.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/TestCapacitySchedulerAsyncScheduling.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/TestCapacitySchedulerAsyncScheduling.java
index c2c1519..840d30d 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/TestCapacitySchedulerAsyncScheduling.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/TestCapacitySchedulerAsyncScheduling.java
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -43,6 +45,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -745,6 +749,71 @@ public class TestCapacitySchedulerAsyncScheduling {
     rm1.close();
   }
 
+  @Test(timeout = 30000)
+  public void testCommitProposalsForUnusableNode() throws Exception {
+    // disable async-scheduling for simulating complex scene
+    Configuration disableAsyncConf = new Configuration(conf);
+    disableAsyncConf.setBoolean(
+        CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, false);
+
+    // init RM & NMs
+    final MockRM rm = new MockRM(disableAsyncConf);
+    rm.start();
+    final MockNM nm1 = rm.registerNode("192.168.0.1:1234", 8 * GB);
+    final MockNM nm2 = rm.registerNode("192.168.0.2:2234", 8 * GB);
+    final MockNM nm3 = rm.registerNode("192.168.0.3:2234", 8 * GB);
+    rm.drainEvents();
+    CapacityScheduler cs =
+        (CapacityScheduler) rm.getRMContext().getScheduler();
+    SchedulerNode sn1 = cs.getSchedulerNode(nm1.getNodeId());
+
+    // launch app1-am on nm1
+    RMApp app1 = rm.submitApp(1 * GB, "app1", "user", null, false, "default",
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS, null, null, true, true);
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+
+    // launch app2-am on nm2
+    RMApp app2 = rm.submitApp(1 * GB, "app2", "user", null, false, "default",
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS, null, null, true, true);
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+
+    // app2 asks 1 * 8G container
+    am2.allocate(ImmutableList.of(ResourceRequest
+        .newInstance(Priority.newInstance(0), "*",
+            Resources.createResource(8 * GB), 1)), null);
+
+    List<Object> reservedProposalParts = new ArrayList<>();
+    final CapacityScheduler spyCs = Mockito.spy(cs);
+    // handle CapacityScheduler#tryCommit
+    Mockito.doAnswer(new Answer<Object>() {
+      public Boolean answer(InvocationOnMock invocation) throws Exception {
+        for (Object argument : invocation.getArguments()) {
+          reservedProposalParts.add(argument);
+        }
+        return false;
+      }
+    }).when(spyCs).tryCommit(Mockito.any(Resource.class),
+        Mockito.any(ResourceCommitRequest.class), Mockito.anyBoolean());
+
+    spyCs.handle(new NodeUpdateSchedulerEvent(sn1.getRMNode()));
+
+    // decommission nm1
+    RMNode rmNode1 = cs.getNode(nm1.getNodeId()).getRMNode();
+    cs.getRMContext().getDispatcher().getEventHandler().handle(
+        new RMNodeEvent(nm1.getNodeId(), RMNodeEventType.DECOMMISSION));
+    rm.drainEvents();
+    Assert.assertEquals(NodeState.DECOMMISSIONED, rmNode1.getState());
+    Assert.assertNull(cs.getNode(nm1.getNodeId()));
+
+    // try commit after nm1 decommissioned
+    boolean isSuccess =
+        cs.tryCommit((Resource) reservedProposalParts.get(0),
+            (ResourceCommitRequest) reservedProposalParts.get(1),
+            (Boolean) reservedProposalParts.get(2));
+    Assert.assertFalse(isSuccess);
+    rm.stop();
+  }
+
   private ResourceCommitRequest createAllocateFromReservedProposal(
       int containerId, Resource allocateResource, FiCaSchedulerApp schedulerApp,
       SchedulerNode allocateNode, SchedulerNode reservedNode,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a71bf14/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 fae63be..b13790d 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
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
@@ -220,6 +221,7 @@ public class TestUtils {
     when(rmNode.getNodeAddress()).thenReturn(host+":"+port);
     when(rmNode.getHostName()).thenReturn(host);
     when(rmNode.getRackName()).thenReturn(rack);
+    when(rmNode.getState()).thenReturn(NodeState.RUNNING);
     
     FiCaSchedulerNode node = spy(new FiCaSchedulerNode(rmNode, false));
     LOG.info("node = " + host + " avail=" + node.getUnallocatedResource());


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


[22/50] [abbrv] hadoop git commit: HDFS-13735. Make QJM HTTP URL connection timeout configurable. Contributed by Chao Sun.

Posted by tm...@apache.org.
HDFS-13735. Make QJM HTTP URL connection timeout configurable. Contributed by Chao Sun.


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

Branch: refs/heads/HADOOP-15407
Commit: 5326a7906de7c86a236d948012cabf3a9ba82310
Parents: d352f16
Author: Chen Liang <cl...@apache.org>
Authored: Thu Aug 9 10:11:47 2018 -0700
Committer: Chen Liang <cl...@apache.org>
Committed: Thu Aug 9 10:11:47 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java     |  5 +++++
 .../qjournal/client/QuorumJournalManager.java     | 11 +++++++++--
 .../src/main/resources/hdfs-default.xml           | 18 ++++++++++++++++++
 3 files changed, 32 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5326a790/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 4f21ee1..55085eb 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
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReservedSpaceCalculator;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.apache.hadoop.http.HttpConfig;
 
 /** 
@@ -1033,6 +1034,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_QJOURNAL_GET_JOURNAL_STATE_TIMEOUT_KEY = "dfs.qjournal.get-journal-state.timeout.ms";
   public static final String  DFS_QJOURNAL_NEW_EPOCH_TIMEOUT_KEY = "dfs.qjournal.new-epoch.timeout.ms";
   public static final String  DFS_QJOURNAL_WRITE_TXNS_TIMEOUT_KEY = "dfs.qjournal.write-txns.timeout.ms";
+  public static final String  DFS_QJOURNAL_HTTP_OPEN_TIMEOUT_KEY = "dfs.qjournal.http.open.timeout.ms";
+  public static final String  DFS_QJOURNAL_HTTP_READ_TIMEOUT_KEY = "dfs.qjournal.http.read.timeout.ms";
   public static final int     DFS_QJOURNAL_START_SEGMENT_TIMEOUT_DEFAULT = 20000;
   public static final int     DFS_QJOURNAL_PREPARE_RECOVERY_TIMEOUT_DEFAULT = 120000;
   public static final int     DFS_QJOURNAL_ACCEPT_RECOVERY_TIMEOUT_DEFAULT = 120000;
@@ -1041,6 +1044,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_QJOURNAL_GET_JOURNAL_STATE_TIMEOUT_DEFAULT = 120000;
   public static final int     DFS_QJOURNAL_NEW_EPOCH_TIMEOUT_DEFAULT = 120000;
   public static final int     DFS_QJOURNAL_WRITE_TXNS_TIMEOUT_DEFAULT = 20000;
+  public static final int     DFS_QJOURNAL_HTTP_OPEN_TIMEOUT_DEFAULT = URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT;
+  public static final int     DFS_QJOURNAL_HTTP_READ_TIMEOUT_DEFAULT = URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT;
   
   public static final String DFS_MAX_NUM_BLOCKS_TO_LOG_KEY = "dfs.namenode.max-num-blocks-to-log";
   public static final long   DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT = 1000l;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5326a790/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
index 7a70a3d..4faaa98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
@@ -124,8 +124,6 @@ public class QuorumJournalManager implements JournalManager {
     this.nsInfo = nsInfo;
     this.nameServiceId = nameServiceId;
     this.loggers = new AsyncLoggerSet(createLoggers(loggerFactory));
-    this.connectionFactory = URLConnectionFactory
-        .newDefaultURLConnectionFactory(conf);
 
     // Configure timeouts.
     this.startSegmentTimeoutMs = conf.getInt(
@@ -156,6 +154,15 @@ public class QuorumJournalManager implements JournalManager {
             .DFS_QJM_OPERATIONS_TIMEOUT,
         DFSConfigKeys.DFS_QJM_OPERATIONS_TIMEOUT_DEFAULT, TimeUnit
             .MILLISECONDS);
+
+    int connectTimeoutMs = conf.getInt(
+        DFSConfigKeys.DFS_QJOURNAL_HTTP_OPEN_TIMEOUT_KEY,
+        DFSConfigKeys.DFS_QJOURNAL_HTTP_OPEN_TIMEOUT_DEFAULT);
+    int readTimeoutMs = conf.getInt(
+        DFSConfigKeys.DFS_QJOURNAL_HTTP_READ_TIMEOUT_KEY,
+        DFSConfigKeys.DFS_QJOURNAL_HTTP_READ_TIMEOUT_DEFAULT);
+    this.connectionFactory = URLConnectionFactory
+        .newDefaultURLConnectionFactory(connectTimeoutMs, readTimeoutMs, conf);
   }
   
   protected List<AsyncLogger> createLoggers(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5326a790/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 dea79f5..8eaf2a4 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
@@ -4583,6 +4583,24 @@
 </property>
 
 <property>
+  <name>dfs.qjournal.http.open.timeout.ms</name>
+  <value>60000</value>
+  <description>
+    Timeout in milliseconds when open a new HTTP connection to remote
+    journals.
+  </description>
+</property>
+
+<property>
+  <name>dfs.qjournal.http.read.timeout.ms</name>
+  <value>60000</value>
+  <description>
+    Timeout in milliseconds when reading from a HTTP connection from remote
+    journals.
+  </description>
+</property>
+
+<property>
   <name>dfs.quota.by.storage.type.enabled</name>
   <value>true</value>
   <description>


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


[24/50] [abbrv] hadoop git commit: YARN-8136. Add version attribute to site doc examples and quickstart. (Eric Yang via wangda)

Posted by tm...@apache.org.
YARN-8136. Add version attribute to site doc examples and quickstart. (Eric Yang via wangda)

Change-Id: I4541b239f490ca0a6edf9698e0d3deaf83669151


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

Branch: refs/heads/HADOOP-15407
Commit: 8244abb7aeb768b73682b8c9a26516a9cf06bca5
Parents: 344c335
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Aug 9 11:03:46 2018 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Aug 9 11:04:02 2018 -0700

----------------------------------------------------------------------
 .../hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md | 5 ++++-
 .../src/site/markdown/yarn-service/QuickStart.md                | 1 +
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8244abb7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md
index 03fec79..73e00b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md
@@ -30,6 +30,7 @@ Note this example requires registry DNS.
 ```
 {
   "name": "httpd-service",
+  "version": "1.0",
   "lifetime": "3600",
   "components": [
     {
@@ -169,9 +170,10 @@ Then visit port 8080 for each IP to view the pages.
 Docker images may have built with ENTRYPOINT to enable start up of docker image without any parameters.
 When passing parameters to ENTRYPOINT enabled image, `launch_command` is delimited by comma (,).
 
+```
 {
   "name": "sleeper-service",
-  "version": "1",
+  "version": "1.0",
   "components" :
   [
     {
@@ -198,3 +200,4 @@ When passing parameters to ENTRYPOINT enabled image, `launch_command` is delimit
     }
   ]
 }
+```

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8244abb7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md
index e144320..bba9bb8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md
@@ -40,6 +40,7 @@ Below is a simple service definition that launches sleep containers on YARN by w
 ```
 {
   "name": "sleeper-service",
+  "version": "1.0",
   "components" : 
     [
       {


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


[36/50] [abbrv] hadoop git commit: HADOOP-15446. ABFS: tune imports & javadocs; stabilise tests. Contributed by Steve Loughran and Da Zhou.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
index a78e7af..2b3ccc0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -26,14 +26,17 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
 
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
-import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
-import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
-import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*;
+
 /**
  * AbfsClient
  */
@@ -53,7 +56,7 @@ public class AbfsClient {
     this.baseUrl = baseUrl;
     this.sharedKeyCredentials = sharedKeyCredentials;
     String baseUrlString = baseUrl.toString();
-    this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(AbfsHttpConstants.FORWARD_SLASH) + 1);
+    this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1);
     this.abfsConfiguration = abfsConfiguration;
     this.retryPolicy = exponentialRetryPolicy;
     this.userAgent = initializeUserAgent();
@@ -73,19 +76,19 @@ public class AbfsClient {
 
   List<AbfsHttpHeader> createDefaultHeaders() {
     final List<AbfsHttpHeader> requestHeaders = new ArrayList<AbfsHttpHeader>();
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_VERSION, xMsVersion));
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT, AbfsHttpConstants.APPLICATION_JSON
-            + AbfsHttpConstants.COMMA + AbfsHttpConstants.SINGLE_WHITE_SPACE + AbfsHttpConstants.APPLICATION_OCTET_STREAM));
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT_CHARSET,
-            AbfsHttpConstants.UTF_8));
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.CONTENT_TYPE, AbfsHttpConstants.EMPTY_STRING));
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.USER_AGENT, userAgent));
+    requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion));
+    requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON
+            + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM));
+    requestHeaders.add(new AbfsHttpHeader(ACCEPT_CHARSET,
+            UTF_8));
+    requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, EMPTY_STRING));
+    requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgent));
     return requestHeaders;
   }
 
   AbfsUriQueryBuilder createDefaultUriQueryBuilder() {
     final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_TIMEOUT, AbfsHttpConstants.DEFAULT_TIMEOUT);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_TIMEOUT, DEFAULT_TIMEOUT);
     return abfsUriQueryBuilder;
   }
 
@@ -93,12 +96,12 @@ public class AbfsClient {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
 
     final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_PUT,
+            HTTP_METHOD_PUT,
             url,
             requestHeaders);
     op.execute();
@@ -109,19 +112,19 @@ public class AbfsClient {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     // JDK7 does not support PATCH, so to workaround the issue we will use
     // PUT and specify the real method in the X-Http-Method-Override header.
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
-            AbfsHttpConstants.HTTP_METHOD_PATCH));
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
 
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES,
+    requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES,
             properties));
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
 
     final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_PUT,
+            HTTP_METHOD_PUT,
             url,
             requestHeaders);
     op.execute();
@@ -133,16 +136,16 @@ public class AbfsClient {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath));
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation);
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults));
 
     final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_GET,
+            HTTP_METHOD_GET,
             url,
             requestHeaders);
     op.execute();
@@ -153,12 +156,12 @@ public class AbfsClient {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
 
     final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_HEAD,
+            HTTP_METHOD_HEAD,
             url,
             requestHeaders);
     op.execute();
@@ -169,12 +172,12 @@ public class AbfsClient {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
 
     final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_DELETE,
+            HTTP_METHOD_DELETE,
             url,
             requestHeaders);
     op.execute();
@@ -185,16 +188,16 @@ public class AbfsClient {
           throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     if (!overwrite) {
-      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, "*"));
+      requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, "*"));
     }
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, isFile ? AbfsHttpConstants.FILE : AbfsHttpConstants.DIRECTORY);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, isFile ? FILE : DIRECTORY);
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_PUT,
+            HTTP_METHOD_PUT,
             url,
             requestHeaders);
     op.execute();
@@ -205,17 +208,17 @@ public class AbfsClient {
           throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
 
-    final String encodedRenameSource = urlEncode(AbfsHttpConstants.FORWARD_SLASH + this.getFileSystem() + source);
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_RENAME_SOURCE, encodedRenameSource));
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.STAR));
+    final String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source);
+    requestHeaders.add(new AbfsHttpHeader(X_MS_RENAME_SOURCE, encodedRenameSource));
+    requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR));
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
 
     final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_PUT,
+            HTTP_METHOD_PUT,
             url,
             requestHeaders);
     op.execute();
@@ -227,17 +230,17 @@ public class AbfsClient {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     // JDK7 does not support PATCH, so to workaround the issue we will use
     // PUT and specify the real method in the X-Http-Method-Override header.
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
-            AbfsHttpConstants.HTTP_METHOD_PATCH));
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.APPEND_ACTION);
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position));
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_PUT,
+            HTTP_METHOD_PUT,
             url,
             requestHeaders, buffer, offset, length);
     op.execute();
@@ -245,44 +248,46 @@ public class AbfsClient {
   }
 
 
-  public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData) throws AzureBlobFileSystemException {
+  public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData)
+      throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     // JDK7 does not support PATCH, so to workaround the issue we will use
     // PUT and specify the real method in the X-Http-Method-Override header.
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
-            AbfsHttpConstants.HTTP_METHOD_PATCH));
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.FLUSH_ACTION);
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position));
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData));
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_PUT,
+            HTTP_METHOD_PUT,
             url,
             requestHeaders);
     op.execute();
     return op;
   }
 
-  public AbfsRestOperation setPathProperties(final String path, final String properties) throws AzureBlobFileSystemException {
+  public AbfsRestOperation setPathProperties(final String path, final String properties)
+      throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     // JDK7 does not support PATCH, so to workaround the issue we will use
     // PUT and specify the real method in the X-Http-Method-Override header.
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
-            AbfsHttpConstants.HTTP_METHOD_PATCH));
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
 
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES, properties));
+    requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, properties));
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_PROPERTIES_ACTION);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_PROPERTIES_ACTION);
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_PUT,
+            HTTP_METHOD_PUT,
             url,
             requestHeaders);
     op.execute();
@@ -297,7 +302,7 @@ public class AbfsClient {
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_HEAD,
+            HTTP_METHOD_HEAD,
             url,
             requestHeaders);
     op.execute();
@@ -307,9 +312,9 @@ public class AbfsClient {
   public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset,
                                 final int bufferLength, final String eTag) throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.RANGE,
+    requestHeaders.add(new AbfsHttpHeader(RANGE,
             String.format("bytes=%d-%d", position, position + bufferLength - 1)));
-    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag));
+    requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
 
@@ -317,7 +322,7 @@ public class AbfsClient {
 
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_GET,
+            HTTP_METHOD_GET,
             url,
             requestHeaders,
             buffer,
@@ -333,13 +338,13 @@ public class AbfsClient {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
-    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = new AbfsRestOperation(
             this,
-            AbfsHttpConstants.HTTP_METHOD_DELETE,
+            HTTP_METHOD_DELETE,
             url,
             requestHeaders);
     op.execute();
@@ -347,7 +352,7 @@ public class AbfsClient {
   }
 
   private URL createRequestUrl(final String query) throws AzureBlobFileSystemException {
-    return createRequestUrl(AbfsHttpConstants.EMPTY_STRING, query);
+    return createRequestUrl(EMPTY_STRING, query);
   }
 
   private URL createRequestUrl(final String path, final String query)
@@ -357,8 +362,8 @@ public class AbfsClient {
     try {
       encodedPath = urlEncode(path);
     } catch (AzureBlobFileSystemException ex) {
-      this.LOG.debug(
-              "Unexpected error.", ex);
+      LOG.debug("Unexpected error.", ex);
+      throw new InvalidUriException(path);
     }
 
     final StringBuilder sb = new StringBuilder();
@@ -378,9 +383,9 @@ public class AbfsClient {
   private static String urlEncode(final String value) throws AzureBlobFileSystemException {
     String encodedString = null;
     try {
-      encodedString =  URLEncoder.encode(value, AbfsHttpConstants.UTF_8)
-          .replace(AbfsHttpConstants.PLUS, AbfsHttpConstants.PLUS_ENCODE)
-          .replace(AbfsHttpConstants.FORWARD_SLASH_ENCODE, AbfsHttpConstants.FORWARD_SLASH);
+      encodedString =  URLEncoder.encode(value, UTF_8)
+          .replace(PLUS, PLUS_ENCODE)
+          .replace(FORWARD_SLASH_ENCODE, FORWARD_SLASH);
     } catch (UnsupportedEncodingException ex) {
         throw new InvalidUriException(value);
     }
@@ -391,11 +396,11 @@ public class AbfsClient {
   private String initializeUserAgent() {
     final String userAgentComment = String.format(Locale.ROOT,
             "(JavaJRE %s; %s %s)",
-            System.getProperty(AbfsHttpConstants.JAVA_VERSION),
-            System.getProperty(AbfsHttpConstants.OS_NAME)
-                    .replaceAll(AbfsHttpConstants.SINGLE_WHITE_SPACE, AbfsHttpConstants.EMPTY_STRING),
-            System.getProperty(AbfsHttpConstants.OS_VERSION));
+            System.getProperty(JAVA_VERSION),
+            System.getProperty(OS_NAME)
+                    .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING),
+            System.getProperty(OS_VERSION));
 
-    return String.format(AbfsHttpConstants.CLIENT_VERSION + " %s", userAgentComment);
+    return String.format(CLIENT_VERSION + " %s", userAgentComment);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
index 0ea9365..53f6900 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
@@ -30,12 +30,12 @@ import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.JsonToken;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Represents an HTTP operation.
@@ -427,4 +427,4 @@ public class AbfsHttpOperation {
   private boolean isNullInputStream(InputStream stream) {
     return stream == null ? true : false;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
index 6554380..848ce8a 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 
 /**
- * The AbfsInputStream for AbfsClient
+ * The AbfsInputStream for AbfsClient.
  */
 public class AbfsInputStream extends FSInputStream {
   private final AbfsClient client;
@@ -59,7 +59,6 @@ public class AbfsInputStream extends FSInputStream {
       final int bufferSize,
       final int readAheadQueueDepth,
       final String eTag) {
-    super();
     this.client = client;
     this.statistics = statistics;
     this.path = path;
@@ -379,4 +378,4 @@ public class AbfsInputStream extends FSInputStream {
   public boolean markSupported() {
     return false;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
index de5c934..2dbcee5 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.azurebfs.services;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.io.OutputStream;
 import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -35,7 +36,7 @@ import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 
 /**
- * The BlobFsOutputStream for Rest AbfsClient
+ * The BlobFsOutputStream for Rest AbfsClient.
  */
 public class AbfsOutputStream extends OutputStream implements Syncable {
   private final AbfsClient client;
@@ -79,8 +80,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
         maxConcurrentRequestCount,
         10L,
         TimeUnit.SECONDS,
-        new LinkedBlockingQueue());
-    this.completionService = new ExecutorCompletionService(this.threadExecutor);
+        new LinkedBlockingQueue<>());
+    this.completionService = new ExecutorCompletionService<>(this.threadExecutor);
   }
 
   /**
@@ -111,9 +112,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
   @Override
   public synchronized void write(final byte[] data, final int off, final int length)
       throws IOException {
-    if (this.lastError != null) {
-      throw this.lastError;
-    }
+    maybeThrowLastError();
 
     Preconditions.checkArgument(data != null, "null data");
 
@@ -143,6 +142,19 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
   }
 
   /**
+   * Throw the last error recorded if not null.
+   * After the stream is closed, this is always set to
+   * an exception, so acts as a guard against method invocation once
+   * closed.
+   * @throws IOException if lastError is set
+   */
+  private void maybeThrowLastError() throws IOException {
+    if (lastError != null) {
+      throw lastError;
+    }
+  }
+
+  /**
    * Flushes this output stream and forces any buffered output bytes to be
    * written out. If any data remains in the payload it is committed to the
    * service. Data is queued for writing and forced out to the service
@@ -150,7 +162,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
    */
   @Override
   public void flush() throws IOException {
-    this.flushInternalAsync();
+    flushInternalAsync();
   }
 
   /** Similar to posix fsync, flush out the data in client's user buffer
@@ -159,7 +171,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
    */
   @Override
   public void hsync() throws IOException {
-    this.flushInternal();
+    flushInternal();
   }
 
   /** Flush out the data in client's user buffer. After the return of
@@ -168,7 +180,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
    */
   @Override
   public void hflush() throws IOException {
-    this.flushInternal();
+    flushInternal();
   }
 
   /**
@@ -186,34 +198,30 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
     }
 
     try {
-      this.flushInternal();
-      this.threadExecutor.shutdown();
+      flushInternal();
+      threadExecutor.shutdown();
     } finally {
-      this.lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
-      this.buffer = null;
-      this.bufferIndex = 0;
-      this.closed = true;
-      this.writeOperations.clear();
-      if (!this.threadExecutor.isShutdown()) {
-        this.threadExecutor.shutdownNow();
+      lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+      buffer = null;
+      bufferIndex = 0;
+      closed = true;
+      writeOperations.clear();
+      if (!threadExecutor.isShutdown()) {
+        threadExecutor.shutdownNow();
       }
     }
   }
 
   private synchronized void flushInternal() throws IOException {
-    if (this.lastError != null) {
-      throw this.lastError;
-    }
-    this.writeCurrentBufferToService();
-    this.flushWrittenBytesToService();
+    maybeThrowLastError();
+    writeCurrentBufferToService();
+    flushWrittenBytesToService();
   }
 
   private synchronized void flushInternalAsync() throws IOException {
-    if (this.lastError != null) {
-      throw this.lastError;
-    }
-    this.writeCurrentBufferToService();
-    this.flushWrittenBytesToServiceAsync();
+    maybeThrowLastError();
+    writeCurrentBufferToService();
+    flushWrittenBytesToServiceAsync();
   }
 
   private synchronized void writeCurrentBufferToService() throws IOException {
@@ -221,19 +229,19 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
       return;
     }
 
-    final byte[] bytes = this.buffer;
+    final byte[] bytes = buffer;
     final int bytesLength = bufferIndex;
 
-    this.buffer = new byte[bufferSize];
-    this.bufferIndex = 0;
-    final long offset = this.position;
-    this.position += bytesLength;
+    buffer = new byte[bufferSize];
+    bufferIndex = 0;
+    final long offset = position;
+    position += bytesLength;
 
-    if (this.threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) {
-      this.waitForTaskToComplete();
+    if (threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) {
+      waitForTaskToComplete();
     }
 
-    final Future job = this.completionService.submit(new Callable<Void>() {
+    final Future<Void> job = completionService.submit(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
         client.append(path, offset, bytes, 0,
@@ -242,25 +250,25 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
       }
     });
 
-    this.writeOperations.add(new WriteOperation(job, offset, bytesLength));
+    writeOperations.add(new WriteOperation(job, offset, bytesLength));
 
     // Try to shrink the queue
     shrinkWriteOperationQueue();
   }
 
   private synchronized void flushWrittenBytesToService() throws IOException {
-    for (WriteOperation writeOperation : this.writeOperations) {
+    for (WriteOperation writeOperation : writeOperations) {
       try {
         writeOperation.task.get();
       } catch (Exception ex) {
-        if (AzureBlobFileSystemException.class.isInstance(ex.getCause())) {
-          ex = AzureBlobFileSystemException.class.cast(ex.getCause());
+        if (ex.getCause() instanceof AzureBlobFileSystemException) {
+          ex = (AzureBlobFileSystemException)ex.getCause();
         }
-        this.lastError = new IOException(ex);
-        throw this.lastError;
+        lastError = new IOException(ex);
+        throw lastError;
       }
     }
-    flushWrittenBytesToServiceInternal(this.position, false);
+    flushWrittenBytesToServiceInternal(position, false);
   }
 
   private synchronized void flushWrittenBytesToServiceAsync() throws IOException {
@@ -273,7 +281,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
     this.lastTotalAppendOffset = 0;
   }
 
-  private synchronized void flushWrittenBytesToServiceInternal(final long offset, final boolean retainUncommitedData) throws IOException {
+  private synchronized void flushWrittenBytesToServiceInternal(final long offset,
+      final boolean retainUncommitedData) throws IOException {
     try {
       client.flush(path, offset, retainUncommitedData);
     } catch (AzureBlobFileSystemException ex) {
@@ -288,31 +297,33 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
    */
   private synchronized void shrinkWriteOperationQueue() throws IOException {
     try {
-      while (this.writeOperations.peek() != null && this.writeOperations.peek().task.isDone()) {
-        this.writeOperations.peek().task.get();
-        this.lastTotalAppendOffset += this.writeOperations.peek().length;
-        this.writeOperations.remove();
+      while (writeOperations.peek() != null && writeOperations.peek().task.isDone()) {
+        writeOperations.peek().task.get();
+        lastTotalAppendOffset += writeOperations.peek().length;
+        writeOperations.remove();
       }
     } catch (Exception e) {
-      if (AzureBlobFileSystemException.class.isInstance(e.getCause())) {
-        this.lastError = IOException.class.cast(e.getCause());
+      if (e.getCause() instanceof AzureBlobFileSystemException) {
+        lastError = (AzureBlobFileSystemException)e.getCause();
       } else {
-        this.lastError = new IOException(e);
+        lastError = new IOException(e);
       }
-      throw this.lastError;
+      throw lastError;
     }
   }
 
   private void waitForTaskToComplete() throws IOException {
     boolean completed;
-    for (completed = false; this.completionService.poll() != null; completed = true) {}
+    for (completed = false; completionService.poll() != null; completed = true) {
+      // keep polling until there is no data
+    }
 
     if (!completed) {
       try {
-        this.completionService.take();
+        completionService.take();
       } catch (InterruptedException e) {
-        this.lastError = new IOException(e);
-        throw this.lastError;
+        lastError = (IOException)new InterruptedIOException(e.toString()).initCause(e);
+        throw lastError;
       }
     }
   }
@@ -332,4 +343,4 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
       this.length = length;
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
index 17fc35a..6126398 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
@@ -23,15 +23,16 @@ import java.net.HttpURLConnection;
 import java.net.URL;
 import java.util.List;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
 
-import org.slf4j.Logger;
-
 /**
- * The AbfsRestOperation for Rest AbfsClient
+ * The AbfsRestOperation for Rest AbfsClient.
  */
 public class AbfsRestOperation {
   // Blob FS client, which has the credentials, retry policy, and logs.
@@ -47,7 +48,7 @@ public class AbfsRestOperation {
   // request body and all the download methods have a response body.
   private final boolean hasRequestBody;
 
-  private final Logger logger;
+  private final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
 
   // For uploads, this is the request entity body.  For downloads,
   // this will hold the response entity body.
@@ -79,7 +80,6 @@ public class AbfsRestOperation {
     this.requestHeaders = requestHeaders;
     this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method)
             || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method));
-    this.logger = client.LOG;
   }
 
   /**
@@ -150,11 +150,11 @@ public class AbfsRestOperation {
 
       httpOperation.processResponse(buffer, bufferOffset, bufferLength);
     } catch (IOException ex) {
-      if (logger.isDebugEnabled()) {
+      if (LOG.isDebugEnabled()) {
         if (httpOperation != null) {
-          logger.debug("HttpRequestFailure: " + httpOperation.toString(), ex);
+          LOG.debug("HttpRequestFailure: " + httpOperation.toString(), ex);
         } else {
-          logger.debug("HttpRequestFailure: " + method + "," + url, ex);
+          LOG.debug("HttpRequestFailure: " + method + "," + url, ex);
         }
       }
       if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
@@ -163,8 +163,8 @@ public class AbfsRestOperation {
       return false;
     }
 
-    if (logger.isDebugEnabled()) {
-      logger.debug("HttpRequest: " + httpOperation.toString());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("HttpRequest: " + httpOperation.toString());
     }
 
     if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {
@@ -175,4 +175,4 @@ public class AbfsRestOperation {
 
     return true;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
index bac66af..3624853 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
@@ -18,13 +18,13 @@
 
 package org.apache.hadoop.fs.azurebfs.services;
 
-import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
-
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+
 /**
- * The UrlQueryBuilder for Rest AbfsClient
+ * The UrlQueryBuilder for Rest AbfsClient.
  */
 public class AbfsUriQueryBuilder {
   private Map<String, String> parameters;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
index 54aa1ab..5eb7a66 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
@@ -43,7 +43,7 @@ public class ExponentialRetryPolicy {
   private static final int DEFAULT_MAX_BACKOFF = 1000 * 30;
 
   /**
-   *Represents the default minimum amount of time used when calculating the exponential
+   * Represents the default minimum amount of time used when calculating the exponential
    * delay between retries.
    */
   private static final int DEFAULT_MIN_BACKOFF = 1000 * 3;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java
index 1fac13d..00e4f00 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java
@@ -18,10 +18,10 @@
 
 package org.apache.hadoop.fs.azurebfs.services;
 
-import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
-
 import java.util.concurrent.CountDownLatch;
 
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+
 class ReadBuffer {
 
   private AbfsInputStream stream;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
index 164e549..5b71cf0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
@@ -28,7 +28,7 @@ import java.util.Stack;
 import java.util.concurrent.CountDownLatch;
 
 /**
- * The Read Buffer Manager for Rest AbfsClient
+ * The Read Buffer Manager for Rest AbfsClient.
  */
 final class ReadBufferManager {
   private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class);
@@ -40,11 +40,11 @@ final class ReadBufferManager {
 
   private Thread[] threads = new Thread[NUM_THREADS];
   private byte[][] buffers;    // array of byte[] buffers, to hold the data that is read
-  private Stack<Integer> freeList = new Stack<Integer>();   // indices in buffers[] array that are available
+  private Stack<Integer> freeList = new Stack<>();   // indices in buffers[] array that are available
 
-  private Queue<ReadBuffer> readAheadQueue = new LinkedList<ReadBuffer>(); // queue of requests that are not picked up by any worker thread yet
-  private LinkedList<ReadBuffer> inProgressList = new LinkedList<ReadBuffer>(); // requests being processed by worker threads
-  private LinkedList<ReadBuffer> completedReadList = new LinkedList<ReadBuffer>(); // buffers available for reading
+  private Queue<ReadBuffer> readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet
+  private LinkedList<ReadBuffer> inProgressList = new LinkedList<>(); // requests being processed by worker threads
+  private LinkedList<ReadBuffer> completedReadList = new LinkedList<>(); // buffers available for reading
   private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
 
   static {
@@ -85,7 +85,7 @@ final class ReadBufferManager {
 
 
   /**
-   * {@link AbfsInputStream} calls this method to queue read-aheads
+   * {@link AbfsInputStream} calls this method to queue read-aheads.
    *
    * @param stream          The {@link AbfsInputStream} for which to do the read-ahead
    * @param requestedOffset The offset in the file which shoukd be read
@@ -93,15 +93,15 @@ final class ReadBufferManager {
    */
   void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength) {
     if (LOGGER.isTraceEnabled()) {
-      LOGGER.trace("Start Queueing readAhead for " + stream.getPath() + " offset " + requestedOffset
-          + " length " + requestedLength);
+      LOGGER.trace("Start Queueing readAhead for {} offset {} length {}",
+          stream.getPath(), requestedOffset, requestedLength);
     }
     ReadBuffer buffer;
     synchronized (this) {
       if (isAlreadyQueued(stream, requestedOffset)) {
         return; // already queued, do not queue again
       }
-      if (freeList.size() == 0 && !tryEvict()) {
+      if (freeList.isEmpty() && !tryEvict()) {
         return; // no buffers available, cannot queue anything
       }
 
@@ -121,8 +121,8 @@ final class ReadBufferManager {
       notifyAll();
     }
     if (LOGGER.isTraceEnabled()) {
-      LOGGER.trace("Done q-ing readAhead for file " + stream.getPath() + " offset " + requestedOffset
-          + " buffer idx " + buffer.getBufferindex());
+      LOGGER.trace("Done q-ing readAhead for file {} offset {} buffer idx {}",
+          stream.getPath(), requestedOffset, buffer.getBufferindex());
     }
   }
 
@@ -144,7 +144,8 @@ final class ReadBufferManager {
   int getBlock(final AbfsInputStream stream, final long position, final int length, final byte[] buffer) {
     // not synchronized, so have to be careful with locking
     if (LOGGER.isTraceEnabled()) {
-      LOGGER.trace("getBlock for file " + stream.getPath() + " position " + position + " thread " + Thread.currentThread().getName());
+      LOGGER.trace("getBlock for file {}  position {}  thread {}",
+          stream.getPath(), position, Thread.currentThread().getName());
     }
 
     waitForProcess(stream, position);
@@ -155,12 +156,13 @@ final class ReadBufferManager {
     }
     if (bytesRead > 0) {
       if (LOGGER.isTraceEnabled()) {
-        LOGGER.trace("Done read from Cache for " + stream.getPath() + " position " + position + " length " + bytesRead);
+        LOGGER.trace("Done read from Cache for {} position {} length {}",
+            stream.getPath(), position, bytesRead);
       }
       return bytesRead;
     }
 
-    // otherwise, just say we got nothing - calling thread can do it's own read
+    // otherwise, just say we got nothing - calling thread can do its own read
     return 0;
   }
 
@@ -179,8 +181,8 @@ final class ReadBufferManager {
     if (readBuf != null) {         // if in in-progress queue, then block for it
       try {
         if (LOGGER.isTraceEnabled()) {
-          LOGGER.trace("got a relevant read buffer for file " + stream.getPath() + " offset " + readBuf.getOffset()
-                  + " buffer idx " + readBuf.getBufferindex());
+          LOGGER.trace("got a relevant read buffer for file {} offset {} buffer idx {}",
+              stream.getPath(), readBuf.getOffset(), readBuf.getBufferindex());
         }
         readBuf.getLatch().await();  // blocking wait on the caller stream's thread
         // Note on correctness: readBuf gets out of inProgressList only in 1 place: after worker thread
@@ -193,8 +195,8 @@ final class ReadBufferManager {
         Thread.currentThread().interrupt();
       }
       if (LOGGER.isTraceEnabled()) {
-        LOGGER.trace("latch done for file " + stream.getPath() + " buffer idx " + readBuf.getBufferindex()
-                + " length " + readBuf.getLength());
+        LOGGER.trace("latch done for file {} buffer idx {} length {}",
+            stream.getPath(), readBuf.getBufferindex(), readBuf.getLength());
       }
     }
   }
@@ -254,8 +256,8 @@ final class ReadBufferManager {
     freeList.push(buf.getBufferindex());
     completedReadList.remove(buf);
     if (LOGGER.isTraceEnabled()) {
-      LOGGER.trace("Evicting buffer idx " + buf.getBufferindex() + "; was used for file " + buf.getStream().getPath()
-          + " offset " + buf.getOffset() + " length " + buf.getLength());
+      LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}",
+          buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength());
     }
     return true;
   }
@@ -344,13 +346,14 @@ final class ReadBufferManager {
       inProgressList.add(buffer);
     }
     if (LOGGER.isTraceEnabled()) {
-      LOGGER.trace("ReadBufferWorker picked file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset());
+      LOGGER.trace("ReadBufferWorker picked file {} for offset {}",
+          buffer.getStream().getPath(), buffer.getOffset());
     }
     return buffer;
   }
 
   /**
-   * ReadBufferWorker thread calls this method to post completion
+   * ReadBufferWorker thread calls this method to post completion.
    *
    * @param buffer            the buffer whose read was completed
    * @param result            the {@link ReadBufferStatus} after the read operation in the worker thread
@@ -358,8 +361,8 @@ final class ReadBufferManager {
    */
   void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) {
     if (LOGGER.isTraceEnabled()) {
-      LOGGER.trace("ReadBufferWorker completed file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset()
-          + " bytes " + bytesActuallyRead);
+      LOGGER.trace("ReadBufferWorker completed file {} for offset {} bytes {}",
+          buffer.getStream().getPath(),  buffer.getOffset(), bytesActuallyRead);
     }
     synchronized (this) {
       inProgressList.remove(buffer);
@@ -380,8 +383,9 @@ final class ReadBufferManager {
   /**
    * Similar to System.currentTimeMillis, except implemented with System.nanoTime().
    * System.currentTimeMillis can go backwards when system clock is changed (e.g., with NTP time synchronization),
-   * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing,
-   * so it is much more suitable to measuring intervals.
+   * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing per CPU core.
+   * Note: it is not monotonic across Sockets, and even within a CPU, its only the
+   * more recent parts which share a clock across all cores.
    *
    * @return current time in milliseconds
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java
index 2d0c96e..af69de0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java
@@ -18,10 +18,10 @@
 
 package org.apache.hadoop.fs.azurebfs.services;
 
-import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
-
 import java.util.concurrent.CountDownLatch;
 
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+
 class ReadBufferWorker implements Runnable {
 
   protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java
index dd59892..105a1a2 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java
@@ -22,6 +22,7 @@ import javax.crypto.Mac;
 import javax.crypto.spec.SecretKeySpec;
 import java.io.UnsupportedEncodingException;
 import java.net.HttpURLConnection;
+import java.net.URL;
 import java.net.URLDecoder;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
@@ -38,11 +39,11 @@ import java.util.TimeZone;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.codec.Charsets;
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.codec.Charsets;
 /**
  * Represents the shared key credentials used to access an Azure Storage
  * account.
@@ -89,7 +90,7 @@ public class SharedKeyCredentials {
   }
 
   private String computeHmac256(final String stringToSign) {
-    byte[] utf8Bytes = null;
+    byte[] utf8Bytes;
     try {
       utf8Bytes = stringToSign.getBytes(AbfsHttpConstants.UTF_8);
     } catch (final UnsupportedEncodingException e) {
@@ -158,7 +159,7 @@ public class SharedKeyCredentials {
   }
 
   /**
-   * Initialie the HmacSha256 associated with the account key.
+   * Initialize the HmacSha256 associated with the account key.
    */
   private void initializeMac() {
     // Initializes the HMAC-SHA256 Mac and SecretKey.
@@ -171,7 +172,7 @@ public class SharedKeyCredentials {
   }
 
   /**
-   * Append a string to a string builder with a newline constant
+   * Append a string to a string builder with a newline constant.
    *
    * @param builder the StringBuilder object
    * @param element the string to append.
@@ -194,9 +195,10 @@ public class SharedKeyCredentials {
    * @param conn          the HttpURLConnection for the operation.
    * @return A canonicalized string.
    */
-  private static String canonicalizeHttpRequest(final java.net.URL address, final String accountName,
-                                                final String method, final String contentType, final long contentLength, final String date,
-                                                final HttpURLConnection conn) throws UnsupportedEncodingException {
+  private static String canonicalizeHttpRequest(final URL address,
+      final String accountName, final String method, final String contentType,
+      final long contentLength, final String date, final HttpURLConnection conn)
+      throws UnsupportedEncodingException {
 
     // The first element should be the Method of the request.
     // I.e. GET, POST, PUT, or HEAD.
@@ -246,7 +248,8 @@ public class SharedKeyCredentials {
    * @param accountName the account name for the request.
    * @return the canonicalized resource string.
    */
-  private static String getCanonicalizedResource(final java.net.URL address, final String accountName) throws UnsupportedEncodingException {
+  private static String getCanonicalizedResource(final URL address,
+      final String accountName) throws UnsupportedEncodingException {
     // Resource path
     final StringBuilder resourcepath = new StringBuilder(AbfsHttpConstants.FORWARD_SLASH);
     resourcepath.append(accountName);
@@ -263,7 +266,7 @@ public class SharedKeyCredentials {
 
     final Map<String, String[]> queryVariables = parseQueryString(address.getQuery());
 
-    final Map<String, String> lowercasedKeyNameValue = new HashMap<String, String>();
+    final Map<String, String> lowercasedKeyNameValue = new HashMap<>();
 
     for (final Entry<String, String[]> entry : queryVariables.entrySet()) {
       // sort the value and organize it as comma separated values
@@ -303,14 +306,17 @@ public class SharedKeyCredentials {
   }
 
   /**
-   * Gets all the values for the given header in the one to many map, performs a trimStart() on each return value
+   * Gets all the values for the given header in the one to many map,
+   * performs a trimStart() on each return value.
    *
    * @param headers    a one to many map of key / values representing the header values for the connection.
    * @param headerName the name of the header to lookup
    * @return an ArrayList<String> of all trimmed values corresponding to the requested headerName. This may be empty
    * if the header is not found.
    */
-  private static ArrayList<String> getHeaderValues(final Map<String, List<String>> headers, final String headerName) {
+  private static ArrayList<String> getHeaderValues(
+      final Map<String, List<String>> headers,
+      final String headerName) {
 
     final ArrayList<String> arrayOfValues = new ArrayList<String>();
     List<String> values = null;
@@ -338,7 +344,7 @@ public class SharedKeyCredentials {
    * @return a HashMap<String, String[]> of the key values.
    */
   private static HashMap<String, String[]> parseQueryString(String parseString) throws UnsupportedEncodingException {
-    final HashMap<String, String[]> retVals = new HashMap<String, String[]>();
+    final HashMap<String, String[]> retVals = new HashMap<>();
     if (parseString == null || parseString.isEmpty()) {
       return retVals;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
new file mode 100644
index 0000000..a4b3483
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
@@ -0,0 +1,72 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Hadoop Azure Support: ABFS  — Azure Data Lake Storage Gen2
+
+<!-- MACRO{toc|fromDepth=1|toDepth=3} -->
+
+## Introduction
+
+The `hadoop-azure` module provides support for the Azure Data Lake Storage Gen2
+storage layer through the "abfs" connector
+
+To make it part of Apache Hadoop's default classpath, simply make sure that
+`HADOOP_OPTIONAL_TOOLS` in `hadoop-env.sh` has `hadoop-azure` in the list.
+
+## Features
+
+* Read and write data stored in an Azure Blob Storage account.
+* *Fully Consistent* view of the storage across all clients.
+* Can read data written through the wasb: connector.
+* Present a hierarchical file system view by implementing the standard Hadoop
+  [`FileSystem`](../api/org/apache/hadoop/fs/FileSystem.html) interface.
+* Supports configuration of multiple Azure Blob Storage accounts.
+* Can act as a source or destination of data in Hadoop MapReduce, Apache Hive, Apache Spark
+* Tested at scale on both Linux and Windows.
+* Can be used as a replacement for HDFS on Hadoop clusters deployed in Azure infrastructure.
+
+
+
+## Limitations
+
+* File last access time is not tracked.
+
+
+## Technical notes
+
+### Security
+
+### Consistency and Concurrency
+
+*TODO*: complete/review
+
+The abfs client has a fully consistent view of the store, which has complete Create Read Update and Delete consistency for data and metadata.
+(Compare and contrast with S3 which only offers Create consistency; S3Guard adds CRUD to metadata, but not the underlying data).
+
+### Performance
+
+*TODO*: check these.
+
+* File Rename: `O(1)`.
+* Directory Rename: `O(files)`.
+* Directory Delete: `O(files)`.
+
+## Testing ABFS
+
+See the relevant section in [Testing Azure](testing_azure.html).
+
+## References
+
+* [A closer look at Azure Data Lake Storage Gen2](https://azure.microsoft.com/en-gb/blog/a-closer-look-at-azure-data-lake-storage-gen2/);
+MSDN Article from June 28, 2018.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
index b58e68b..c148807 100644
--- a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
+++ b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
@@ -574,3 +574,79 @@ mvn test -Dtest=CleanupTestContainers
 
 This will delete the containers; the output log of the test run will
 provide the details and summary of the operation.
+
+
+## Testing ABFS
+
+The ABFS Connector tests share the same account as the wasb tests; this is
+needed for cross-connector compatibility tests.
+
+This makes for a somewhat complex set of configuration options.
+
+Here are the settings for an account `ACCOUNTNAME`
+
+```xml
+<property>
+  <name>abfs.account.name</name>
+  <value>ACCOUNTNAME</value>
+</property>
+
+<property>
+  <name>abfs.account.full.name</name>
+  <value>${abfs.account.name}.dfs.core.windows.net</value>
+</property>
+
+<property>
+  <name>abfs.account.key</name>
+  <value>SECRETKEY==</value>
+</property>
+
+<property>
+  <name>fs.azure.account.key.ACCOUNTNAME.dfs.core.windows.net</name>
+  <value>${abfs.account.key}</value>
+</property>
+
+<property>
+  <name>fs.azure.account.key.ACCOUNTNAME.blob.core.windows.net</name>
+  <value>${abfs.account.key}</value>
+</property>
+
+<property>
+  <name>fs.azure.test.account.key.ACCOUNTNAME.dfs.core.windows.net</name>
+  <value>${abfs.account.key}</value>
+</property>
+
+<property>
+  <name>fs.azure.test.account.key.ACCOUNTNAME.blob.core.windows.net</name>
+  <value>${abfs.account.key}</value>
+</property>
+
+<property>
+  <name>fs.azure.account.key.ACCOUNTNAME</name>
+  <value>${abfs.account.key}</value>
+</property>
+
+<property>
+  <name>fs.azure.test.account.key.ACCOUNTNAME</name>
+  <value>${abfs.account.key}</value>
+</property>
+
+<property>
+  <name>fs.azure.test.account.name</name>
+  <value>${abfs.account.full.name}</value>
+</property>
+
+<property>
+  <name>fs.contract.test.fs.abfs</name>
+  <value>abfs://TESTCONTAINER@ACCOUNTNAME.dfs.core.windows.net</value>
+  <description>Container for contract tests</description>
+</property>
+
+<property>
+  <name>fs.contract.test.fs.abfss</name>
+  <value>abfss://TESTCONTAINER@ACCOUNTNAME.dfs.core.windows.net</value>
+  <description>Container for contract tests</description>
+</property>
+
+
+```

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java
index fd21bd2..db4a843 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java
@@ -18,10 +18,19 @@
 
 package org.apache.hadoop.fs.azure.contract;
 
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.junit.Test;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Append test, skipping one of them.
@@ -38,4 +47,18 @@ public class ITestAzureNativeContractAppend extends AbstractContractAppendTest {
   public void testRenameFileBeingAppended() throws Throwable {
     skip("Skipping as renaming an opened file is not supported");
   }
+
+  /**
+   * Wasb returns a different exception, so change the intercept logic here.
+   */
+  @Override
+  @Test
+  public void testAppendDirectory() throws Exception {
+    final FileSystem fs = getFileSystem();
+
+    final Path folderPath = path("testAppendDirectory");
+    fs.mkdirs(folderPath);
+    intercept(FileNotFoundException.class,
+        () -> fs.append(folderPath));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
new file mode 100644
index 0000000..106fa09
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
@@ -0,0 +1,304 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Hashtable;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+
+import com.google.common.base.Preconditions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.AbstractWasbTestWithTimeout;
+import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
+import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
+import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.junit.Assume.assumeTrue;
+import static org.hamcrest.CoreMatchers.*;
+
+/**
+ * Base for AzureBlobFileSystem Integration tests.
+ *
+ * <I>Important: This is for integration tests only.</I>
+ */
+public abstract class AbstractAbfsIntegrationTest extends
+    AbstractWasbTestWithTimeout {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class);
+
+  private final boolean isEmulator;
+  private NativeAzureFileSystem wasb;
+  private AzureBlobFileSystem abfs;
+  private String abfsScheme;
+
+  private Configuration configuration;
+  private String fileSystemName;
+  private String accountName;
+  private String testUrl;
+
+  protected AbstractAbfsIntegrationTest(final boolean secure) {
+    this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME);
+  }
+
+  protected AbstractAbfsIntegrationTest() {
+    this(FileSystemUriSchemes.ABFS_SCHEME);
+  }
+
+  private AbstractAbfsIntegrationTest(final String scheme) {
+    abfsScheme = scheme;
+    fileSystemName = ABFS_TEST_CONTAINER_PREFIX + UUID.randomUUID().toString();
+    configuration = new Configuration();
+    configuration.addResource(ABFS_TEST_RESOURCE_XML);
+
+    String accountName = configuration.get(FS_AZURE_TEST_ACCOUNT_NAME, "");
+    assumeTrue("Not set: " + FS_AZURE_TEST_ACCOUNT_NAME,
+        !accountName.isEmpty());
+    assertThat("The key in " + FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
+            + " is not bound to an ABFS account",
+        accountName, containsString("dfs.core.windows.net"));
+    String fullKey = FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
+        + accountName;
+    assumeTrue("Not set: " + fullKey,
+        configuration.get(fullKey) != null);
+
+    final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
+    URI defaultUri = null;
+
+    try {
+      defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
+    } catch (Exception ex) {
+      throw new AssertionError(ex);
+    }
+
+    this.testUrl = defaultUri.toString();
+    configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
+    configuration.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    this.isEmulator = this.configuration.getBoolean(FS_AZURE_EMULATOR_ENABLED, false);
+    this.accountName = this.configuration.get(FS_AZURE_TEST_ACCOUNT_NAME);
+  }
+
+
+  @Before
+  public void setup() throws Exception {
+    //Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem.
+    createFileSystem();
+
+    if (!isEmulator) {
+      final URI wasbUri = new URI(abfsUrlToWasbUrl(getTestUrl()));
+      final AzureNativeFileSystemStore azureNativeFileSystemStore =
+          new AzureNativeFileSystemStore();
+      azureNativeFileSystemStore.initialize(
+          wasbUri,
+          getConfiguration(),
+          new AzureFileSystemInstrumentation(getConfiguration()));
+
+      wasb = new NativeAzureFileSystem(azureNativeFileSystemStore);
+      wasb.initialize(wasbUri, configuration);
+    }
+  }
+
+  @After
+  public void teardown() throws Exception {
+    try {
+      IOUtils.closeStream(wasb);
+      wasb = null;
+
+      if (abfs == null) {
+        return;
+      }
+
+      final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore();
+      abfsStore.deleteFilesystem();
+
+      AbfsRestOperationException ex = intercept(
+              AbfsRestOperationException.class,
+              new Callable<Hashtable<String, String>>() {
+                @Override
+                public Hashtable<String, String> call() throws Exception {
+                  return abfsStore.getFilesystemProperties();
+                }
+              });
+      if (FILE_SYSTEM_NOT_FOUND.getStatusCode() != ex.getStatusCode()) {
+        LOG.warn("Deleted test filesystem may still exist: {}", abfs, ex);
+      }
+    } catch (Exception e) {
+      LOG.warn("During cleanup: {}", e, e);
+    } finally {
+      IOUtils.closeStream(abfs);
+      abfs = null;
+    }
+  }
+
+  public AzureBlobFileSystem getFileSystem() throws IOException {
+    return abfs;
+  }
+
+  /**
+   * Creates the filesystem; updates the {@link #abfs} field.
+   * @return the created filesystem.
+   * @throws IOException failure during create/init.
+   */
+  public AzureBlobFileSystem createFileSystem() throws IOException {
+    Preconditions.checkState(abfs == null,
+        "existing ABFS instance exists: %s", abfs);
+    abfs = (AzureBlobFileSystem) FileSystem.newInstance(configuration);
+    return abfs;
+  }
+
+
+  protected NativeAzureFileSystem getWasbFileSystem() {
+    return wasb;
+  }
+
+  protected String getHostName() {
+    return configuration.get(FS_AZURE_TEST_HOST_NAME);
+  }
+
+  protected void setTestUrl(String testUrl) {
+    this.testUrl = testUrl;
+  }
+
+  protected String getTestUrl() {
+    return testUrl;
+  }
+
+  protected void setFileSystemName(String fileSystemName) {
+    this.fileSystemName = fileSystemName;
+  }
+  protected String getFileSystemName() {
+    return fileSystemName;
+  }
+
+  protected String getAccountName() {
+    return configuration.get(FS_AZURE_TEST_ACCOUNT_NAME);
+  }
+
+  protected String getAccountKey() {
+    return configuration.get(
+        FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
+            + getAccountName());
+  }
+
+  protected Configuration getConfiguration() {
+    return configuration;
+  }
+
+  protected boolean isEmulator() {
+    return isEmulator;
+  }
+
+  /**
+   * Write a buffer to a file.
+   * @param path path
+   * @param buffer buffer
+   * @throws IOException failure
+   */
+  protected void write(Path path, byte[] buffer) throws IOException {
+    ContractTestUtils.writeDataset(getFileSystem(), path, buffer, buffer.length,
+        CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, false);
+  }
+
+  /**
+   * Touch a file in the test store. Will overwrite any existing file.
+   * @param path path
+   * @throws IOException failure.
+   */
+  protected void touch(Path path) throws IOException {
+    ContractTestUtils.touch(getFileSystem(), path);
+  }
+
+  protected static String wasbUrlToAbfsUrl(final String wasbUrl) {
+    return convertTestUrls(
+        wasbUrl, FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX,
+        FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX);
+  }
+
+  protected static String abfsUrlToWasbUrl(final String abfsUrl) {
+    return convertTestUrls(
+        abfsUrl, FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX,
+        FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX);
+  }
+
+  private static String convertTestUrls(
+      final String url,
+      final String fromNonSecureScheme,
+      final String fromSecureScheme,
+      final String fromDnsPrefix,
+      final String toNonSecureScheme,
+      final String toSecureScheme,
+      final String toDnsPrefix) {
+    String data = null;
+    if (url.startsWith(fromNonSecureScheme + "://")) {
+      data = url.replace(fromNonSecureScheme + "://", toNonSecureScheme + "://");
+    } else if (url.startsWith(fromSecureScheme + "://")) {
+      data = url.replace(fromSecureScheme + "://", toSecureScheme + "://");
+    }
+
+
+    if (data != null) {
+      data = data.replace("." + fromDnsPrefix + ".",
+          "." + toDnsPrefix + ".");
+    }
+    return data;
+  }
+
+  public Path getTestPath() {
+    Path path = new Path(UriUtils.generateUniqueTestPath());
+    return path;
+  }
+
+  /**
+   * Create a path under the test path provided by
+   * {@link #getTestPath()}.
+   * @param filepath path string in
+   * @return a path qualified by the test filesystem
+   * @throws IOException IO problems
+   */
+  protected Path path(String filepath) throws IOException {
+    return getFileSystem().makeQualified(
+        new Path(getTestPath(), filepath));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java
new file mode 100644
index 0000000..cfda7a7
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.azurebfs;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
+
+import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.assumeScaleTestsEnabled;
+
+/**
+ * Integration tests at bigger scale; configurable as to
+ * size, off by default.
+ */
+public class AbstractAbfsScaleTest extends AbstractAbfsIntegrationTest  {
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(AbstractAbfsScaleTest.class);
+
+  @Override
+  protected int getTestTimeoutMillis() {
+    return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS;
+  }
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    LOG.debug("Scale test operation count = {}", getOperationCount());
+    assumeScaleTestsEnabled(getConfiguration());
+  }
+
+  protected long getOperationCount() {
+    return getConfiguration().getLong(AzureTestConstants.KEY_OPERATION_COUNT,
+        AzureTestConstants.DEFAULT_OPERATION_COUNT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
deleted file mode 100644
index 74a530c..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
+++ /dev/null
@@ -1,206 +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.fs.azurebfs;
-
-import java.net.URI;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
-import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
-import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
-import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
-
-import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
-import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assume.assumeNotNull;
-
-/**
- * Provide dependencies for AzureBlobFileSystem tests.
- */
-public abstract class DependencyInjectedTest {
-  private final boolean isEmulator;
-  private NativeAzureFileSystem wasb;
-  private String abfsScheme;
-
-  private Configuration configuration;
-  private String fileSystemName;
-  private String accountName;
-  private String testUrl;
-
-  public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-";
-
-  public DependencyInjectedTest(final boolean secure) {
-    this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME);
-  }
-
-  protected DependencyInjectedTest() {
-    this(FileSystemUriSchemes.ABFS_SCHEME);
-  }
-
-  private DependencyInjectedTest(final String scheme) {
-    abfsScheme = scheme;
-    fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString();
-    configuration = new Configuration();
-    configuration.addResource("azure-bfs-test.xml");
-
-    assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME));
-    assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + configuration.get(TestConfigurationKeys
-        .FS_AZURE_TEST_ACCOUNT_NAME)));
-
-    final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
-    URI defaultUri = null;
-
-    try {
-      defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
-    } catch (Exception ex) {
-      Assert.fail(ex.getMessage());
-    }
-
-    this.testUrl = defaultUri.toString();
-    configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
-    configuration.setBoolean(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
-    this.isEmulator = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
-    this.accountName = this.configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
-  }
-
-  @Before
-  public void initialize() throws Exception {
-    //Create filesystem first to make sure getWasbFileSystem() can return an existed filesystem.
-    this.getFileSystem();
-
-    if (!this.isEmulator) {
-      final URI wasbUri = new URI(abfsUrlToWasbUrl(this.getTestUrl()));
-      final AzureNativeFileSystemStore azureNativeFileSystemStore = new AzureNativeFileSystemStore();
-      azureNativeFileSystemStore.initialize(
-          wasbUri,
-          this.getConfiguration(),
-          new AzureFileSystemInstrumentation(this.getConfiguration()));
-
-      this.wasb = new NativeAzureFileSystem(azureNativeFileSystemStore);
-      this.wasb.initialize(wasbUri, configuration);
-    }
-  }
-
-  @After
-  public void testCleanup() throws Exception {
-    if (this.wasb != null) {
-      this.wasb.close();
-    }
-
-    FileSystem.closeAll();
-
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
-    abfsStore.deleteFilesystem();
-
-    AbfsRestOperationException ex = intercept(
-            AbfsRestOperationException.class,
-            new Callable<Void>() {
-              @Override
-              public Void call() throws Exception {
-                fs.getAbfsStore().getFilesystemProperties();
-                return null;
-              }
-            });
-
-    assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode());
-  }
-
-  public AzureBlobFileSystem getFileSystem() throws Exception {
-    return (AzureBlobFileSystem) FileSystem.get(this.configuration);
-  }
-
-  protected NativeAzureFileSystem getWasbFileSystem() {
-    return this.wasb;
-  }
-
-  protected String getHostName() {
-    return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME);
-  }
-
-  protected void updateTestUrl(String testUrl) {
-    this.testUrl = testUrl;
-  }
-  protected String getTestUrl() {
-    return testUrl;
-  }
-
-  protected void updateFileSystemName(String fileSystemName) {
-    this.fileSystemName = fileSystemName;
-  }
-  protected String getFileSystemName() {
-    return fileSystemName;
-  }
-
-  protected String getAccountName() {
-    return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
-  }
-
-  protected String getAccountKey() {
-    return configuration.get(
-        TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
-            + getAccountName());
-  }
-
-  protected Configuration getConfiguration() {
-    return this.configuration;
-  }
-
-  protected boolean isEmulator() {
-    return isEmulator;
-  }
-
-  protected static String wasbUrlToAbfsUrl(final String wasbUrl) {
-    return convertTestUrls(
-        wasbUrl, FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX,
-        FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX);
-  }
-
-  protected static String abfsUrlToWasbUrl(final String abfsUrl) {
-    return convertTestUrls(
-        abfsUrl, FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX,
-        FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX);
-  }
-
-  private static String convertTestUrls(
-      final String url, final String fromNonSecureScheme, final String fromSecureScheme, final String fromDnsPrefix,
-      final String toNonSecureScheme, final String toSecureScheme, final String toDnsPrefix) {
-    String data = null;
-    if (url.startsWith(fromNonSecureScheme + "://")) {
-      data = url.replace(fromNonSecureScheme + "://", toNonSecureScheme + "://");
-    } else if (url.startsWith(fromSecureScheme + "://")) {
-      data = url.replace(fromSecureScheme + "://", toSecureScheme + "://");
-    }
-
-    data = data.replace("." + fromDnsPrefix + ".", "." + toDnsPrefix + ".");
-    return data;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
index 10d42d1..f2e26ec 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
@@ -25,13 +25,13 @@ import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
-
-import static org.junit.Assert.assertEquals;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 
 /**
  * Test append operations.
  */
-public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemAppend extends
+    AbstractAbfsIntegrationTest {
   private static final Path TEST_FILE_PATH = new Path("testfile");
   private static final Path TEST_FOLDER_PATH = new Path("testFolder");
   public ITestAzureBlobFileSystemAppend() {
@@ -40,7 +40,7 @@ public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest {
 
   @Test(expected = FileNotFoundException.class)
   public void testAppendDirShouldFail() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     final Path filePath = TEST_FILE_PATH;
     fs.mkdirs(filePath);
     fs.append(filePath, 0);
@@ -48,21 +48,21 @@ public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest {
 
   @Test
   public void testAppendWithLength0() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
-    final byte[] b = new byte[1024];
-    new Random().nextBytes(b);
-    stream.write(b, 1000, 0);
-
-    assertEquals(0, stream.getPos());
+    final AzureBlobFileSystem fs = getFileSystem();
+    try(FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
+      final byte[] b = new byte[1024];
+      new Random().nextBytes(b);
+      stream.write(b, 1000, 0);
+      assertEquals(0, stream.getPos());
+    }
   }
 
 
   @Test(expected = FileNotFoundException.class)
   public void testAppendFileAfterDelete() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     final Path filePath = TEST_FILE_PATH;
-    fs.create(filePath);
+    ContractTestUtils.touch(fs, filePath);
     fs.delete(filePath, false);
 
     fs.append(filePath);
@@ -70,7 +70,7 @@ public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest {
 
   @Test(expected = FileNotFoundException.class)
   public void testAppendDirectory() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     final Path folderPath = TEST_FOLDER_PATH;
     fs.mkdirs(folderPath);
     fs.append(folderPath);


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


[43/50] [abbrv] hadoop git commit: HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
new file mode 100644
index 0000000..de5c934
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -0,0 +1,335 @@
+/**
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+
+/**
+ * The BlobFsOutputStream for Rest AbfsClient
+ */
+public class AbfsOutputStream extends OutputStream implements Syncable {
+  private final AbfsClient client;
+  private final String path;
+  private long position;
+  private boolean closed;
+  private volatile IOException lastError;
+
+  private long lastFlushOffset;
+  private long lastTotalAppendOffset = 0;
+
+  private final int bufferSize;
+  private byte[] buffer;
+  private int bufferIndex;
+  private final int maxConcurrentRequestCount;
+
+  private ConcurrentLinkedDeque<WriteOperation> writeOperations;
+  private final ThreadPoolExecutor threadExecutor;
+  private final ExecutorCompletionService<Void> completionService;
+
+  public AbfsOutputStream(
+      final AbfsClient client,
+      final String path,
+      final long position,
+      final int bufferSize) {
+    this.client = client;
+    this.path = path;
+    this.position = position;
+    this.closed = false;
+    this.lastError = null;
+    this.lastFlushOffset = 0;
+    this.bufferSize = bufferSize;
+    this.buffer = new byte[bufferSize];
+    this.bufferIndex = 0;
+    this.writeOperations = new ConcurrentLinkedDeque<>();
+
+    this.maxConcurrentRequestCount = 4 * Runtime.getRuntime().availableProcessors();
+
+    this.threadExecutor
+        = new ThreadPoolExecutor(maxConcurrentRequestCount,
+        maxConcurrentRequestCount,
+        10L,
+        TimeUnit.SECONDS,
+        new LinkedBlockingQueue());
+    this.completionService = new ExecutorCompletionService(this.threadExecutor);
+  }
+
+  /**
+   * Writes the specified byte to this output stream. The general contract for
+   * write is that one byte is written to the output stream. The byte to be
+   * written is the eight low-order bits of the argument b. The 24 high-order
+   * bits of b are ignored.
+   *
+   * @param byteVal the byteValue to write.
+   * @throws IOException if an I/O error occurs. In particular, an IOException may be
+   *                     thrown if the output stream has been closed.
+   */
+  @Override
+  public void write(final int byteVal) throws IOException {
+    write(new byte[]{(byte) (byteVal & 0xFF)});
+  }
+
+  /**
+   * Writes length bytes from the specified byte array starting at off to
+   * this output stream.
+   *
+   * @param data   the byte array to write.
+   * @param off the start off in the data.
+   * @param length the number of bytes to write.
+   * @throws IOException if an I/O error occurs. In particular, an IOException may be
+   *                     thrown if the output stream has been closed.
+   */
+  @Override
+  public synchronized void write(final byte[] data, final int off, final int length)
+      throws IOException {
+    if (this.lastError != null) {
+      throw this.lastError;
+    }
+
+    Preconditions.checkArgument(data != null, "null data");
+
+    if (off < 0 || length < 0 || length > data.length - off) {
+      throw new IndexOutOfBoundsException();
+    }
+
+    int currentOffset = off;
+    int writableBytes = bufferSize - bufferIndex;
+    int numberOfBytesToWrite = length;
+
+    while (numberOfBytesToWrite > 0) {
+      if (writableBytes <= numberOfBytesToWrite) {
+        System.arraycopy(data, currentOffset, buffer, bufferIndex, writableBytes);
+        bufferIndex += writableBytes;
+        writeCurrentBufferToService();
+        currentOffset += writableBytes;
+        numberOfBytesToWrite = numberOfBytesToWrite - writableBytes;
+      } else {
+        System.arraycopy(data, currentOffset, buffer, bufferIndex, numberOfBytesToWrite);
+        bufferIndex += numberOfBytesToWrite;
+        numberOfBytesToWrite = 0;
+      }
+
+      writableBytes = bufferSize - bufferIndex;
+    }
+  }
+
+  /**
+   * Flushes this output stream and forces any buffered output bytes to be
+   * written out. If any data remains in the payload it is committed to the
+   * service. Data is queued for writing and forced out to the service
+   * before the call returns.
+   */
+  @Override
+  public void flush() throws IOException {
+    this.flushInternalAsync();
+  }
+
+  /** Similar to posix fsync, flush out the data in client's user buffer
+   * all the way to the disk device (but the disk may have it in its cache).
+   * @throws IOException if error occurs
+   */
+  @Override
+  public void hsync() throws IOException {
+    this.flushInternal();
+  }
+
+  /** Flush out the data in client's user buffer. After the return of
+   * this call, new readers will see the data.
+   * @throws IOException if any error occurs
+   */
+  @Override
+  public void hflush() throws IOException {
+    this.flushInternal();
+  }
+
+  /**
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete. Close the access to the stream and
+   * shutdown the upload thread pool.
+   * If the blob was created, its lease will be released.
+   * Any error encountered caught in threads and stored will be rethrown here
+   * after cleanup.
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+
+    try {
+      this.flushInternal();
+      this.threadExecutor.shutdown();
+    } finally {
+      this.lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+      this.buffer = null;
+      this.bufferIndex = 0;
+      this.closed = true;
+      this.writeOperations.clear();
+      if (!this.threadExecutor.isShutdown()) {
+        this.threadExecutor.shutdownNow();
+      }
+    }
+  }
+
+  private synchronized void flushInternal() throws IOException {
+    if (this.lastError != null) {
+      throw this.lastError;
+    }
+    this.writeCurrentBufferToService();
+    this.flushWrittenBytesToService();
+  }
+
+  private synchronized void flushInternalAsync() throws IOException {
+    if (this.lastError != null) {
+      throw this.lastError;
+    }
+    this.writeCurrentBufferToService();
+    this.flushWrittenBytesToServiceAsync();
+  }
+
+  private synchronized void writeCurrentBufferToService() throws IOException {
+    if (bufferIndex == 0) {
+      return;
+    }
+
+    final byte[] bytes = this.buffer;
+    final int bytesLength = bufferIndex;
+
+    this.buffer = new byte[bufferSize];
+    this.bufferIndex = 0;
+    final long offset = this.position;
+    this.position += bytesLength;
+
+    if (this.threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) {
+      this.waitForTaskToComplete();
+    }
+
+    final Future job = this.completionService.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        client.append(path, offset, bytes, 0,
+            bytesLength);
+        return null;
+      }
+    });
+
+    this.writeOperations.add(new WriteOperation(job, offset, bytesLength));
+
+    // Try to shrink the queue
+    shrinkWriteOperationQueue();
+  }
+
+  private synchronized void flushWrittenBytesToService() throws IOException {
+    for (WriteOperation writeOperation : this.writeOperations) {
+      try {
+        writeOperation.task.get();
+      } catch (Exception ex) {
+        if (AzureBlobFileSystemException.class.isInstance(ex.getCause())) {
+          ex = AzureBlobFileSystemException.class.cast(ex.getCause());
+        }
+        this.lastError = new IOException(ex);
+        throw this.lastError;
+      }
+    }
+    flushWrittenBytesToServiceInternal(this.position, false);
+  }
+
+  private synchronized void flushWrittenBytesToServiceAsync() throws IOException {
+    shrinkWriteOperationQueue();
+
+    if (this.lastTotalAppendOffset > this.lastFlushOffset) {
+      this.flushWrittenBytesToServiceInternal(this.lastTotalAppendOffset, true);
+    }
+
+    this.lastTotalAppendOffset = 0;
+  }
+
+  private synchronized void flushWrittenBytesToServiceInternal(final long offset, final boolean retainUncommitedData) throws IOException {
+    try {
+      client.flush(path, offset, retainUncommitedData);
+    } catch (AzureBlobFileSystemException ex) {
+      throw new IOException(ex);
+    }
+    this.lastFlushOffset = offset;
+  }
+
+  /**
+   * Try to remove the completed write operations from the beginning of write
+   * operation FIFO queue.
+   */
+  private synchronized void shrinkWriteOperationQueue() throws IOException {
+    try {
+      while (this.writeOperations.peek() != null && this.writeOperations.peek().task.isDone()) {
+        this.writeOperations.peek().task.get();
+        this.lastTotalAppendOffset += this.writeOperations.peek().length;
+        this.writeOperations.remove();
+      }
+    } catch (Exception e) {
+      if (AzureBlobFileSystemException.class.isInstance(e.getCause())) {
+        this.lastError = IOException.class.cast(e.getCause());
+      } else {
+        this.lastError = new IOException(e);
+      }
+      throw this.lastError;
+    }
+  }
+
+  private void waitForTaskToComplete() throws IOException {
+    boolean completed;
+    for (completed = false; this.completionService.poll() != null; completed = true) {}
+
+    if (!completed) {
+      try {
+        this.completionService.take();
+      } catch (InterruptedException e) {
+        this.lastError = new IOException(e);
+        throw this.lastError;
+      }
+    }
+  }
+
+  private static class WriteOperation {
+    private final Future<Void> task;
+    private final long startOffset;
+    private final long length;
+
+    WriteOperation(final Future<Void> task, final long startOffset, final long length) {
+      Preconditions.checkNotNull(task, "task");
+      Preconditions.checkArgument(startOffset >= 0, "startOffset");
+      Preconditions.checkArgument(length >= 0, "length");
+
+      this.task = task;
+      this.startOffset = startOffset;
+      this.length = length;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
new file mode 100644
index 0000000..17fc35a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
@@ -0,0 +1,178 @@
+/**
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.List;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
+
+import org.slf4j.Logger;
+
+/**
+ * The AbfsRestOperation for Rest AbfsClient
+ */
+public class AbfsRestOperation {
+  // Blob FS client, which has the credentials, retry policy, and logs.
+  private final AbfsClient client;
+  // the HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE)
+  private final String method;
+  // full URL including query parameters
+  private final URL url;
+  // all the custom HTTP request headers provided by the caller
+  private final List<AbfsHttpHeader> requestHeaders;
+
+  // This is a simple operation class, where all the upload methods have a
+  // request body and all the download methods have a response body.
+  private final boolean hasRequestBody;
+
+  private final Logger logger;
+
+  // For uploads, this is the request entity body.  For downloads,
+  // this will hold the response entity body.
+  private byte[] buffer;
+  private int bufferOffset;
+  private int bufferLength;
+
+  private AbfsHttpOperation result;
+
+  public AbfsHttpOperation getResult() {
+    return result;
+  }
+
+  /**
+   * Initializes a new REST operation.
+   *
+   * @param client The Blob FS client.
+   * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
+   * @param url The full URL including query string parameters.
+   * @param requestHeaders The HTTP request headers.
+   */
+  AbfsRestOperation(final AbfsClient client,
+                    final String method,
+                    final URL url,
+                    final List<AbfsHttpHeader> requestHeaders) {
+    this.client = client;
+    this.method = method;
+    this.url = url;
+    this.requestHeaders = requestHeaders;
+    this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method)
+            || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method));
+    this.logger = client.LOG;
+  }
+
+  /**
+   * Initializes a new REST operation.
+   *
+   * @param client The Blob FS client.
+   * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
+   * @param url The full URL including query string parameters.
+   * @param requestHeaders The HTTP request headers.
+   * @param buffer For uploads, this is the request entity body.  For downloads,
+   *               this will hold the response entity body.
+   * @param bufferOffset An offset into the buffer where the data beings.
+   * @param bufferLength The length of the data in the buffer.
+   */
+  AbfsRestOperation(AbfsClient client,
+                    String method,
+                    URL url,
+                    List<AbfsHttpHeader> requestHeaders,
+                    byte[] buffer,
+                    int bufferOffset,
+                    int bufferLength) {
+    this(client, method, url, requestHeaders);
+    this.buffer = buffer;
+    this.bufferOffset = bufferOffset;
+    this.bufferLength = bufferLength;
+  }
+
+  /**
+   * Executes the REST operation with retry, by issuing one or more
+   * HTTP operations.
+   */
+  void execute() throws AzureBlobFileSystemException {
+    int retryCount = 0;
+    while (!executeHttpOperation(retryCount++)) {
+      try {
+        Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount));
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    if (result.getStatusCode() > HttpURLConnection.HTTP_BAD_REQUEST) {
+      throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(),
+          result.getStorageErrorMessage(), null, result);
+    }
+  }
+
+  /**
+   * Executes a single HTTP operation to complete the REST operation.  If it
+   * fails, there may be a retry.  The retryCount is incremented with each
+   * attempt.
+   */
+  private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileSystemException {
+    AbfsHttpOperation httpOperation = null;
+    try {
+      // initialize the HTTP request and open the connection
+      httpOperation = new AbfsHttpOperation(url, method, requestHeaders);
+
+      // sign the HTTP request
+      client.getSharedKeyCredentials().signRequest(
+          httpOperation.getConnection(),
+          hasRequestBody ? bufferLength : 0);
+
+      if (hasRequestBody) {
+        // HttpUrlConnection requires
+        httpOperation.sendRequest(buffer, bufferOffset, bufferLength);
+      }
+
+      httpOperation.processResponse(buffer, bufferOffset, bufferLength);
+    } catch (IOException ex) {
+      if (logger.isDebugEnabled()) {
+        if (httpOperation != null) {
+          logger.debug("HttpRequestFailure: " + httpOperation.toString(), ex);
+        } else {
+          logger.debug("HttpRequestFailure: " + method + "," + url, ex);
+        }
+      }
+      if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
+        throw new InvalidAbfsRestOperationException(ex);
+      }
+      return false;
+    }
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("HttpRequest: " + httpOperation.toString());
+    }
+
+    if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {
+      return false;
+    }
+
+    result = httpOperation;
+
+    return true;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java
new file mode 100644
index 0000000..1cbf6b5
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java
@@ -0,0 +1,81 @@
+/**
+ * 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.azurebfs.services;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.inject.AbstractModule;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
+
+/**
+ * This class is responsible to configure all the services used by Azure Blob File System.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+class AbfsServiceInjectorImpl extends AbstractModule {
+  private final Configuration configuration;
+  private final Map<Class, Class> providers;
+  private final Map<Class, Object> instances;
+
+  AbfsServiceInjectorImpl(Configuration configuration) {
+    this.providers = new HashMap<>();
+    this.instances = new HashMap<>();
+    this.configuration = configuration;
+
+    this.instances.put(Configuration.class, this.configuration);
+
+    this.providers.put(ConfigurationService.class, ConfigurationServiceImpl.class);
+
+    this.providers.put(AbfsHttpService.class, AbfsHttpServiceImpl.class);
+    this.providers.put(AbfsHttpClientFactory.class, AbfsHttpClientFactoryImpl.class);
+
+    this.providers.put(TracingService.class, TracingServiceImpl.class);
+  }
+
+  @Override
+  protected void configure() {
+    for (Map.Entry<Class, Object> entrySet : this.instances.entrySet()) {
+      bind(entrySet.getKey()).toInstance(entrySet.getValue());
+    }
+
+    for (Map.Entry<Class, Class> entrySet : this.providers.entrySet()) {
+      bind(entrySet.getKey()).to(entrySet.getValue());
+    }
+  }
+
+  protected Configuration getConfiguration() {
+    return this.configuration;
+  }
+
+  protected Map<Class, Class> getProviders() {
+    return this.providers;
+  }
+
+  protected Map<Class, Object> getInstances() {
+    return this.instances;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java
new file mode 100644
index 0000000..8560620
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.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.fs.azurebfs.services;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider;
+import org.apache.hadoop.fs.azurebfs.contracts.services.InjectableService;
+
+/**
+ * Dependency injected Azure Storage services provider.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class AbfsServiceProviderImpl implements AbfsServiceProvider {
+  private static AbfsServiceProviderImpl abfsServiceProvider;
+  private final Injector abfsServiceInjector;
+
+  private AbfsServiceProviderImpl(final Configuration configuration) {
+    this.abfsServiceInjector = Guice.createInjector(new AbfsServiceInjectorImpl(Preconditions.checkNotNull(configuration, "configuration")));
+  }
+
+  @VisibleForTesting
+  private AbfsServiceProviderImpl(final Injector abfsServiceInjector) {
+    Preconditions.checkNotNull(abfsServiceInjector, "abfsServiceInjector");
+    this.abfsServiceInjector = abfsServiceInjector;
+  }
+
+  /**
+   * Create an instance or returns existing instance of service provider.
+   * This method must be marked as synchronized to ensure thread-safety.
+   * @param configuration hadoop configuration.
+   * @return AbfsServiceProvider the service provider instance.
+   */
+  public static synchronized AbfsServiceProvider create(final Configuration configuration) {
+    if (abfsServiceProvider == null) {
+      abfsServiceProvider = new AbfsServiceProviderImpl(configuration);
+    }
+
+    return abfsServiceProvider;
+  }
+
+  /**
+   * Returns current instance of service provider.
+   * @return AbfsServiceProvider the service provider instance.
+   */
+  public static AbfsServiceProvider instance() {
+    return abfsServiceProvider;
+  }
+
+  @VisibleForTesting
+  static synchronized AbfsServiceProvider create(Injector serviceInjector) {
+    abfsServiceProvider = new AbfsServiceProviderImpl(serviceInjector);
+    return abfsServiceProvider;
+  }
+
+  /**
+   * Returns an instance of resolved injectable service by class name.
+   * The injectable service must be configured first to be resolvable.
+   * @param clazz the injectable service which is expected to be returned.
+   * @param <T> The type of injectable service.
+   * @return T instance
+   * @throws ServiceResolutionException if the service is not resolvable.
+   */
+  @Override
+  public <T extends InjectableService> T get(final Class<T> clazz) throws ServiceResolutionException {
+    try {
+      return this.abfsServiceInjector.getInstance(clazz);
+    } catch (Exception ex) {
+      throw new ServiceResolutionException(clazz.getSimpleName(), ex);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
new file mode 100644
index 0000000..bac66af
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * The UrlQueryBuilder for Rest AbfsClient
+ */
+public class AbfsUriQueryBuilder {
+  private Map<String, String> parameters;
+
+  public AbfsUriQueryBuilder() {
+    this.parameters = new HashMap<>();
+  }
+
+  public void addQuery(final String name, final String value) {
+    if (value != null && !value.isEmpty()) {
+      this.parameters.put(name, value);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    boolean first = true;
+
+    for (Map.Entry<String, String> entry : parameters.entrySet()) {
+      if (first) {
+        sb.append(AbfsHttpConstants.QUESTION_MARK);
+        first = false;
+      } else {
+        sb.append(AbfsHttpConstants.AND_MARK);
+      }
+      sb.append(entry.getKey()).append(AbfsHttpConstants.EQUAL).append(entry.getValue());
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java
new file mode 100644
index 0000000..568ee5d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java
@@ -0,0 +1,317 @@
+/**
+ * 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.azurebfs.services;
+
+import java.lang.reflect.Field;
+import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
+
+@Singleton
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+class ConfigurationServiceImpl implements ConfigurationService {
+  private final Configuration configuration;
+  private final boolean isSecure;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE,
+      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
+      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
+      DefaultValue = FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE)
+  private int writeBufferSize;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_READ_BUFFER_SIZE,
+      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
+      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
+      DefaultValue = FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE)
+  private int readBufferSize;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL,
+      DefaultValue = FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL)
+  private int minBackoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL,
+      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL)
+  private int maxBackoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BACKOFF_INTERVAL,
+      DefaultValue = FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL)
+  private int backoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_IO_RETRIES,
+      MinValue = 0,
+      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS)
+  private int maxIoRetries;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_SIZE_PROPERTY_NAME,
+      MinValue = 0,
+      MaxValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE,
+      DefaultValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE)
+  private long azureBlockSize;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
+      DefaultValue = FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT)
+  private String azureBlockLocationHost;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_OUT,
+      MinValue = 1,
+      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_WRITE_THREADS)
+  private int maxConcurrentWriteThreads;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_IN,
+      MinValue = 1,
+      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_READ_THREADS)
+  private int maxConcurrentReadThreads;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_TOLERATE_CONCURRENT_APPEND,
+      DefaultValue = FileSystemConfigurations.DEFAULT_READ_TOLERATE_CONCURRENT_APPEND)
+  private boolean tolerateOobAppends;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ATOMIC_RENAME_KEY,
+          DefaultValue = FileSystemConfigurations.DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES)
+  private String azureAtomicDirs;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
+      DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION)
+  private boolean createRemoteFileSystemDuringInitialization;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH,
+      DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH)
+  private int readAheadQueueDepth;
+
+  private Map<String, String> storageAccountKeys;
+
+  @Inject
+  ConfigurationServiceImpl(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException {
+    this.configuration = configuration;
+    this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false);
+
+    validateStorageAccountKeys();
+    Field[] fields = this.getClass().getDeclaredFields();
+    for (Field field : fields) {
+      field.setAccessible(true);
+      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateInt(field));
+      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateLong(field));
+      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateString(field));
+      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateBase64String(field));
+      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateBoolean(field));
+      }
+    }
+  }
+
+  @Override
+  public boolean isEmulator() {
+    return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
+  }
+
+  @Override
+  public boolean isSecureMode() {
+    return this.isSecure;
+  }
+
+  @Override
+  public String getStorageAccountKey(final String accountName) throws ConfigurationPropertyNotFoundException {
+    String accountKey = this.storageAccountKeys.get(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName);
+    if (accountKey == null) {
+      throw new ConfigurationPropertyNotFoundException(accountName);
+    }
+
+    return accountKey;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return this.configuration;
+  }
+
+  @Override
+  public int getWriteBufferSize() {
+    return this.writeBufferSize;
+  }
+
+  @Override
+  public int getReadBufferSize() {
+    return this.readBufferSize;
+  }
+
+  @Override
+  public int getMinBackoffIntervalMilliseconds() {
+    return this.minBackoffInterval;
+  }
+
+  @Override
+  public int getMaxBackoffIntervalMilliseconds() {
+    return this.maxBackoffInterval;
+  }
+
+  @Override
+  public int getBackoffIntervalMilliseconds() {
+    return this.backoffInterval;
+  }
+
+  @Override
+  public int getMaxIoRetries() {
+    return this.maxIoRetries;
+  }
+
+  @Override
+  public long getAzureBlockSize() {
+    return this.azureBlockSize;
+  }
+
+  @Override
+  public String getAzureBlockLocationHost() {
+    return this.azureBlockLocationHost;
+  }
+
+  @Override
+  public int getMaxConcurrentWriteThreads() {
+    return this.maxConcurrentWriteThreads;
+  }
+
+  @Override
+  public int getMaxConcurrentReadThreads() {
+    return this.maxConcurrentReadThreads;
+  }
+
+  @Override
+  public boolean getTolerateOobAppends() {
+    return this.tolerateOobAppends;
+  }
+
+  @Override
+  public String getAzureAtomicRenameDirs() {
+    return this.azureAtomicDirs;
+  }
+
+  @Override
+  public boolean getCreateRemoteFileSystemDuringInitialization() {
+    return this.createRemoteFileSystemDuringInitialization;
+  }
+
+  @Override
+  public int getReadAheadQueueDepth() {
+    return this.readAheadQueueDepth;
+  }
+
+  void validateStorageAccountKeys() throws InvalidConfigurationValueException {
+    Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator(
+        ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true);
+    this.storageAccountKeys = this.configuration.getValByRegex(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX);
+
+    for (Map.Entry<String, String> account : this.storageAccountKeys.entrySet()) {
+      validator.validate(account.getValue());
+    }
+  }
+
+  int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new IntegerConfigurationBasicValidator(
+        validator.MinValue(),
+        validator.MaxValue(),
+        validator.DefaultValue(),
+        validator.ConfigurationKey(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new LongConfigurationBasicValidator(
+        validator.MinValue(),
+        validator.MaxValue(),
+        validator.DefaultValue(),
+        validator.ConfigurationKey(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new StringConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class));
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new Base64StringConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new BooleanConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  @VisibleForTesting
+  void setReadBufferSize(int bufferSize) {
+    this.readBufferSize = bufferSize;
+  }
+
+  @VisibleForTesting
+  void setWriteBufferSize(int bufferSize) {
+    this.writeBufferSize = bufferSize;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
new file mode 100644
index 0000000..0c92612
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
@@ -0,0 +1,141 @@
+/**
+ * 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.azurebfs.services;
+
+import java.util.Random;
+import java.net.HttpURLConnection;
+
+class ExponentialRetryPolicy {
+  /**
+   * Represents the default number of retry attempts.
+   */
+  private static final int DEFAULT_CLIENT_RETRY_COUNT = 30;
+
+  /**
+   * Represents the default amount of time used when calculating a random delta in the exponential
+   * delay between retries.
+   */
+  private static final int DEFAULT_CLIENT_BACKOFF = 1000 * 3;
+
+  /**
+   * Represents the default maximum amount of time used when calculating the exponential
+   * delay between retries.
+   */
+  private static final int DEFAULT_MAX_BACKOFF = 1000 * 30;
+
+  /**
+   *Represents the default minimum amount of time used when calculating the exponential
+   * delay between retries.
+   */
+  private static final int DEFAULT_MIN_BACKOFF = 1000 * 3;
+
+  /**
+   *  The minimum random ratio used for delay interval calculation.
+   */
+  private static final double MIN_RANDOM_RATIO = 0.8;
+
+  /**
+   *  The maximum random ratio used for delay interval calculation.
+   */
+  private static final double MAX_RANDOM_RATIO = 1.2;
+
+  /**
+   *  Holds the random number generator used to calculate randomized backoff intervals
+   */
+  private final Random randRef = new Random();
+
+  /**
+   * The value that will be used to calculate a random delta in the exponential delay interval
+   */
+  private final int deltaBackoff;
+
+  /**
+   * The maximum backoff time.
+   */
+  private final int maxBackoff;
+
+  /**
+   * The minimum backoff time.
+   */
+  private final int minBackoff;
+
+  /**
+   * The maximum number of retry attempts.
+   */
+  private final int retryCount;
+
+  /**
+   * Initializes a new instance of the {@link ExponentialRetryPolicy} class.
+   */
+  ExponentialRetryPolicy() {
+    this(DEFAULT_CLIENT_RETRY_COUNT, DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_CLIENT_BACKOFF);
+  }
+
+  /**
+   * Initializes a new instance of the {@link ExponentialRetryPolicy} class.
+   *
+   * @param retryCount The maximum number of retry attempts.
+   * @param minBackoff The minimum backoff time.
+   * @param maxBackoff The maximum backoff time.
+   * @param deltaBackoff The value that will be used to calculate a random delta in the exponential delay
+   *                     between retries.
+   */
+  ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) {
+    this.retryCount = retryCount;
+    this.minBackoff = minBackoff;
+    this.maxBackoff = maxBackoff;
+    this.deltaBackoff = deltaBackoff;
+  }
+
+  /**
+   * Returns if a request should be retried based on the retry count, current response,
+   * and the current strategy.
+   *
+   * @param retryCount The current retry attempt count.
+   * @param statusCode The status code of the response, or -1 for socket error.
+   * @return true if the request should be retried; false otherwise.
+   */
+  public boolean shouldRetry(final int retryCount, final int statusCode) {
+    return retryCount < this.retryCount
+        && (statusCode == -1
+        || statusCode == HttpURLConnection.HTTP_CLIENT_TIMEOUT
+        || (statusCode >= HttpURLConnection.HTTP_INTERNAL_ERROR
+            && statusCode != HttpURLConnection.HTTP_NOT_IMPLEMENTED
+            && statusCode != HttpURLConnection.HTTP_VERSION));
+  }
+
+  /**
+   * Returns backoff interval between 80% and 120% of the desired backoff,
+   * multiply by 2^n-1 for exponential.
+   *
+   * @param retryCount The current retry attempt count.
+   * @return backoff Interval time
+   */
+  public long getRetryInterval(final int retryCount) {
+    final long boundedRandDelta = (int) (this.deltaBackoff * MIN_RANDOM_RATIO)
+        + this.randRef.nextInt((int) (this.deltaBackoff * MAX_RANDOM_RATIO)
+        - (int) (this.deltaBackoff * MIN_RANDOM_RATIO));
+
+    final double incrementDelta = (Math.pow(2, retryCount - 1)) * boundedRandDelta;
+
+    final long retryInterval = (int) Math.round(Math.min(this.minBackoff + incrementDelta, maxBackoff));
+
+    return retryInterval;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java
new file mode 100644
index 0000000..99190e6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java
@@ -0,0 +1,74 @@
+/**
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.htrace.core.HTraceConfiguration;
+import org.apache.htrace.core.Span;
+import org.apache.htrace.core.SpanReceiver;
+import org.apache.htrace.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.htrace.fasterxml.jackson.databind.ObjectWriter;
+import org.apache.htrace.fasterxml.jackson.databind.SerializationFeature;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * LoggerSpanReceiver is a layer between HTrace and log4j only used for {@link org.apache.hadoop.fs.azurebfs.contracts.services.TracingService}
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class LoggerSpanReceiver extends SpanReceiver {
+  private static final ObjectWriter JSON_WRITER =
+      new ObjectMapper()
+          .configure(SerializationFeature.INDENT_OUTPUT, true)
+          .configure(SerializationFeature.WRITE_BIGDECIMAL_AS_PLAIN, true)
+          .configure(SerializationFeature.WRITE_EMPTY_JSON_ARRAYS, false)
+          .configure(SerializationFeature.USE_EQUALITY_FOR_OBJECT_ID, false)
+          .writer();
+
+  public LoggerSpanReceiver(HTraceConfiguration hTraceConfiguration) {
+    Preconditions.checkNotNull(hTraceConfiguration, "hTraceConfiguration");
+  }
+
+  @Override
+  public void receiveSpan(final Span span) {
+    String jsonValue;
+
+    Logger logger = LoggerFactory.getLogger(AzureBlobFileSystem.class);
+
+    try {
+      jsonValue = JSON_WRITER.writeValueAsString(span);
+      logger.trace(jsonValue);
+    } catch (JsonProcessingException e) {
+      logger.error("Json processing error: " + e.getMessage());
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // No-Op
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java
new file mode 100644
index 0000000..1fac13d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java
@@ -0,0 +1,139 @@
+/**
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+
+import java.util.concurrent.CountDownLatch;
+
+class ReadBuffer {
+
+  private AbfsInputStream stream;
+  private long offset;                   // offset within the file for the buffer
+  private int length;                    // actual length, set after the buffer is filles
+  private int requestedLength;           // requested length of the read
+  private byte[] buffer;                 // the buffer itself
+  private int bufferindex = -1;          // index in the buffers array in Buffer manager
+  private ReadBufferStatus status;             // status of the buffer
+  private CountDownLatch latch = null;   // signaled when the buffer is done reading, so any client
+  // waiting on this buffer gets unblocked
+
+  // fields to help with eviction logic
+  private long timeStamp = 0;  // tick at which buffer became available to read
+  private boolean isFirstByteConsumed = false;
+  private boolean isLastByteConsumed = false;
+  private boolean isAnyByteConsumed = false;
+
+  public AbfsInputStream getStream() {
+    return stream;
+  }
+
+  public void setStream(AbfsInputStream stream) {
+    this.stream = stream;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  public void setOffset(long offset) {
+    this.offset = offset;
+  }
+
+  public int getLength() {
+    return length;
+  }
+
+  public void setLength(int length) {
+    this.length = length;
+  }
+
+  public int getRequestedLength() {
+    return requestedLength;
+  }
+
+  public void setRequestedLength(int requestedLength) {
+    this.requestedLength = requestedLength;
+  }
+
+  public byte[] getBuffer() {
+    return buffer;
+  }
+
+  public void setBuffer(byte[] buffer) {
+    this.buffer = buffer;
+  }
+
+  public int getBufferindex() {
+    return bufferindex;
+  }
+
+  public void setBufferindex(int bufferindex) {
+    this.bufferindex = bufferindex;
+  }
+
+  public ReadBufferStatus getStatus() {
+    return status;
+  }
+
+  public void setStatus(ReadBufferStatus status) {
+    this.status = status;
+  }
+
+  public CountDownLatch getLatch() {
+    return latch;
+  }
+
+  public void setLatch(CountDownLatch latch) {
+    this.latch = latch;
+  }
+
+  public long getTimeStamp() {
+    return timeStamp;
+  }
+
+  public void setTimeStamp(long timeStamp) {
+    this.timeStamp = timeStamp;
+  }
+
+  public boolean isFirstByteConsumed() {
+    return isFirstByteConsumed;
+  }
+
+  public void setFirstByteConsumed(boolean isFirstByteConsumed) {
+    this.isFirstByteConsumed = isFirstByteConsumed;
+  }
+
+  public boolean isLastByteConsumed() {
+    return isLastByteConsumed;
+  }
+
+  public void setLastByteConsumed(boolean isLastByteConsumed) {
+    this.isLastByteConsumed = isLastByteConsumed;
+  }
+
+  public boolean isAnyByteConsumed() {
+    return isAnyByteConsumed;
+  }
+
+  public void setAnyByteConsumed(boolean isAnyByteConsumed) {
+    this.isAnyByteConsumed = isAnyByteConsumed;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
new file mode 100644
index 0000000..164e549
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
@@ -0,0 +1,391 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.Stack;
+import java.util.concurrent.CountDownLatch;
+
+/**
+ * The Read Buffer Manager for Rest AbfsClient
+ */
+final class ReadBufferManager {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class);
+
+  private static final int NUM_BUFFERS = 16;
+  private static final int BLOCK_SIZE = 4 * 1024 * 1024;
+  private static final int NUM_THREADS = 8;
+  private static final int THRESHOLD_AGE_MILLISECONDS = 3000; // have to see if 3 seconds is a good threshold
+
+  private Thread[] threads = new Thread[NUM_THREADS];
+  private byte[][] buffers;    // array of byte[] buffers, to hold the data that is read
+  private Stack<Integer> freeList = new Stack<Integer>();   // indices in buffers[] array that are available
+
+  private Queue<ReadBuffer> readAheadQueue = new LinkedList<ReadBuffer>(); // queue of requests that are not picked up by any worker thread yet
+  private LinkedList<ReadBuffer> inProgressList = new LinkedList<ReadBuffer>(); // requests being processed by worker threads
+  private LinkedList<ReadBuffer> completedReadList = new LinkedList<ReadBuffer>(); // buffers available for reading
+  private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
+
+  static {
+    BUFFER_MANAGER = new ReadBufferManager();
+    BUFFER_MANAGER.init();
+  }
+
+  static ReadBufferManager getBufferManager() {
+    return BUFFER_MANAGER;
+  }
+
+  private void init() {
+    buffers = new byte[NUM_BUFFERS][];
+    for (int i = 0; i < NUM_BUFFERS; i++) {
+      buffers[i] = new byte[BLOCK_SIZE];  // same buffers are reused. The byte array never goes back to GC
+      freeList.add(i);
+    }
+    for (int i = 0; i < NUM_THREADS; i++) {
+      Thread t = new Thread(new ReadBufferWorker(i));
+      t.setDaemon(true);
+      threads[i] = t;
+      t.setName("ABFS-prefetch-" + i);
+      t.start();
+    }
+    ReadBufferWorker.UNLEASH_WORKERS.countDown();
+  }
+
+  // hide instance constructor
+  private ReadBufferManager() {
+  }
+
+
+  /*
+   *
+   *  AbfsInputStream-facing methods
+   *
+   */
+
+
+  /**
+   * {@link AbfsInputStream} calls this method to queue read-aheads
+   *
+   * @param stream          The {@link AbfsInputStream} for which to do the read-ahead
+   * @param requestedOffset The offset in the file which shoukd be read
+   * @param requestedLength The length to read
+   */
+  void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength) {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("Start Queueing readAhead for " + stream.getPath() + " offset " + requestedOffset
+          + " length " + requestedLength);
+    }
+    ReadBuffer buffer;
+    synchronized (this) {
+      if (isAlreadyQueued(stream, requestedOffset)) {
+        return; // already queued, do not queue again
+      }
+      if (freeList.size() == 0 && !tryEvict()) {
+        return; // no buffers available, cannot queue anything
+      }
+
+      buffer = new ReadBuffer();
+      buffer.setStream(stream);
+      buffer.setOffset(requestedOffset);
+      buffer.setLength(0);
+      buffer.setRequestedLength(requestedLength);
+      buffer.setStatus(ReadBufferStatus.NOT_AVAILABLE);
+      buffer.setLatch(new CountDownLatch(1));
+
+      Integer bufferIndex = freeList.pop();  // will return a value, since we have checked size > 0 already
+
+      buffer.setBuffer(buffers[bufferIndex]);
+      buffer.setBufferindex(bufferIndex);
+      readAheadQueue.add(buffer);
+      notifyAll();
+    }
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("Done q-ing readAhead for file " + stream.getPath() + " offset " + requestedOffset
+          + " buffer idx " + buffer.getBufferindex());
+    }
+  }
+
+
+  /**
+   * {@link AbfsInputStream} calls this method read any bytes already available in a buffer (thereby saving a
+   * remote read). This returns the bytes if the data already exists in buffer. If there is a buffer that is reading
+   * the requested offset, then this method blocks until that read completes. If the data is queued in a read-ahead
+   * but not picked up by a worker thread yet, then it cancels that read-ahead and reports cache miss. This is because
+   * depending on worker thread availability, the read-ahead may take a while - the calling thread can do it's own
+   * read to get the data faster (copmared to the read waiting in queue for an indeterminate amount of time).
+   *
+   * @param stream   the file to read bytes for
+   * @param position the offset in the file to do a read for
+   * @param length   the length to read
+   * @param buffer   the buffer to read data into. Note that the buffer will be written into from offset 0.
+   * @return the number of bytes read
+   */
+  int getBlock(final AbfsInputStream stream, final long position, final int length, final byte[] buffer) {
+    // not synchronized, so have to be careful with locking
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("getBlock for file " + stream.getPath() + " position " + position + " thread " + Thread.currentThread().getName());
+    }
+
+    waitForProcess(stream, position);
+
+    int bytesRead = 0;
+    synchronized (this) {
+      bytesRead = getBlockFromCompletedQueue(stream, position, length, buffer);
+    }
+    if (bytesRead > 0) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("Done read from Cache for " + stream.getPath() + " position " + position + " length " + bytesRead);
+      }
+      return bytesRead;
+    }
+
+    // otherwise, just say we got nothing - calling thread can do it's own read
+    return 0;
+  }
+
+  /*
+   *
+   *  Internal methods
+   *
+   */
+
+  private void waitForProcess(final AbfsInputStream stream, final long position) {
+    ReadBuffer readBuf;
+    synchronized (this) {
+      clearFromReadAheadQueue(stream, position);
+      readBuf = getFromList(inProgressList, stream, position);
+    }
+    if (readBuf != null) {         // if in in-progress queue, then block for it
+      try {
+        if (LOGGER.isTraceEnabled()) {
+          LOGGER.trace("got a relevant read buffer for file " + stream.getPath() + " offset " + readBuf.getOffset()
+                  + " buffer idx " + readBuf.getBufferindex());
+        }
+        readBuf.getLatch().await();  // blocking wait on the caller stream's thread
+        // Note on correctness: readBuf gets out of inProgressList only in 1 place: after worker thread
+        // is done processing it (in doneReading). There, the latch is set after removing the buffer from
+        // inProgressList. So this latch is safe to be outside the synchronized block.
+        // Putting it in synchronized would result in a deadlock, since this thread would be holding the lock
+        // while waiting, so no one will be able to  change any state. If this becomes more complex in the future,
+        // then the latch cane be removed and replaced with wait/notify whenever inProgressList is touched.
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+      }
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("latch done for file " + stream.getPath() + " buffer idx " + readBuf.getBufferindex()
+                + " length " + readBuf.getLength());
+      }
+    }
+  }
+
+  /**
+   * If any buffer in the completedlist can be reclaimed then reclaim it and return the buffer to free list.
+   * The objective is to find just one buffer - there is no advantage to evicting more than one.
+   *
+   * @return whether the eviction succeeeded - i.e., were we able to free up one buffer
+   */
+  private synchronized boolean tryEvict() {
+    ReadBuffer nodeToEvict = null;
+    if (completedReadList.size() <= 0) {
+      return false;  // there are no evict-able buffers
+    }
+
+    // first, try buffers where all bytes have been consumed (approximated as first and last bytes consumed)
+    for (ReadBuffer buf : completedReadList) {
+      if (buf.isFirstByteConsumed() && buf.isLastByteConsumed()) {
+        nodeToEvict = buf;
+        break;
+      }
+    }
+    if (nodeToEvict != null) {
+      return evict(nodeToEvict);
+    }
+
+    // next, try buffers where any bytes have been consumed (may be a bad idea? have to experiment and see)
+    for (ReadBuffer buf : completedReadList) {
+      if (buf.isAnyByteConsumed()) {
+        nodeToEvict = buf;
+        break;
+      }
+    }
+
+    if (nodeToEvict != null) {
+      return evict(nodeToEvict);
+    }
+
+    // next, try any old nodes that have not been consumed
+    long earliestBirthday = Long.MAX_VALUE;
+    for (ReadBuffer buf : completedReadList) {
+      if (buf.getTimeStamp() < earliestBirthday) {
+        nodeToEvict = buf;
+        earliestBirthday = buf.getTimeStamp();
+      }
+    }
+    if ((currentTimeMillis() - earliestBirthday > THRESHOLD_AGE_MILLISECONDS) && (nodeToEvict != null)) {
+      return evict(nodeToEvict);
+    }
+
+    // nothing can be evicted
+    return false;
+  }
+
+  private boolean evict(final ReadBuffer buf) {
+    freeList.push(buf.getBufferindex());
+    completedReadList.remove(buf);
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("Evicting buffer idx " + buf.getBufferindex() + "; was used for file " + buf.getStream().getPath()
+          + " offset " + buf.getOffset() + " length " + buf.getLength());
+    }
+    return true;
+  }
+
+  private boolean isAlreadyQueued(final AbfsInputStream stream, final long requestedOffset) {
+    // returns true if any part of the buffer is already queued
+    return (isInList(readAheadQueue, stream, requestedOffset)
+        || isInList(inProgressList, stream, requestedOffset)
+        || isInList(completedReadList, stream, requestedOffset));
+  }
+
+  private boolean isInList(final Collection<ReadBuffer> list, final AbfsInputStream stream, final long requestedOffset) {
+    return (getFromList(list, stream, requestedOffset) != null);
+  }
+
+  private ReadBuffer getFromList(final Collection<ReadBuffer> list, final AbfsInputStream stream, final long requestedOffset) {
+    for (ReadBuffer buffer : list) {
+      if (buffer.getStream() == stream) {
+        if (buffer.getStatus() == ReadBufferStatus.AVAILABLE
+            && requestedOffset >= buffer.getOffset()
+            && requestedOffset < buffer.getOffset() + buffer.getLength()) {
+          return buffer;
+        } else if (requestedOffset >= buffer.getOffset()
+            && requestedOffset < buffer.getOffset() + buffer.getRequestedLength()) {
+          return buffer;
+        }
+      }
+    }
+    return null;
+  }
+
+  private void clearFromReadAheadQueue(final AbfsInputStream stream, final long requestedOffset) {
+    ReadBuffer buffer = getFromList(readAheadQueue, stream, requestedOffset);
+    if (buffer != null) {
+      readAheadQueue.remove(buffer);
+      notifyAll();   // lock is held in calling method
+      freeList.push(buffer.getBufferindex());
+    }
+  }
+
+  private int getBlockFromCompletedQueue(final AbfsInputStream stream, final long position, final int length,
+                                         final byte[] buffer) {
+    ReadBuffer buf = getFromList(completedReadList, stream, position);
+    if (buf == null || position >= buf.getOffset() + buf.getLength()) {
+      return 0;
+    }
+    int cursor = (int) (position - buf.getOffset());
+    int availableLengthInBuffer = buf.getLength() - cursor;
+    int lengthToCopy = Math.min(length, availableLengthInBuffer);
+    System.arraycopy(buf.getBuffer(), cursor, buffer, 0, lengthToCopy);
+    if (cursor == 0) {
+      buf.setFirstByteConsumed(true);
+    }
+    if (cursor + lengthToCopy == buf.getLength()) {
+      buf.setLastByteConsumed(true);
+    }
+    buf.setAnyByteConsumed(true);
+    return lengthToCopy;
+  }
+
+  /*
+   *
+   *  ReadBufferWorker-thread-facing methods
+   *
+   */
+
+  /**
+   * ReadBufferWorker thread calls this to get the next buffer that it should work on.
+   *
+   * @return {@link ReadBuffer}
+   * @throws InterruptedException if thread is interrupted
+   */
+  ReadBuffer getNextBlockToRead() throws InterruptedException {
+    ReadBuffer buffer = null;
+    synchronized (this) {
+      //buffer = readAheadQueue.take();  // blocking method
+      while (readAheadQueue.size() == 0) {
+        wait();
+      }
+      buffer = readAheadQueue.remove();
+      notifyAll();
+      if (buffer == null) {
+        return null;            // should never happen
+      }
+      buffer.setStatus(ReadBufferStatus.READING_IN_PROGRESS);
+      inProgressList.add(buffer);
+    }
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("ReadBufferWorker picked file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset());
+    }
+    return buffer;
+  }
+
+  /**
+   * ReadBufferWorker thread calls this method to post completion
+   *
+   * @param buffer            the buffer whose read was completed
+   * @param result            the {@link ReadBufferStatus} after the read operation in the worker thread
+   * @param bytesActuallyRead the number of bytes that the worker thread was actually able to read
+   */
+  void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("ReadBufferWorker completed file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset()
+          + " bytes " + bytesActuallyRead);
+    }
+    synchronized (this) {
+      inProgressList.remove(buffer);
+      if (result == ReadBufferStatus.AVAILABLE && bytesActuallyRead > 0) {
+        buffer.setStatus(ReadBufferStatus.AVAILABLE);
+        buffer.setTimeStamp(currentTimeMillis());
+        buffer.setLength(bytesActuallyRead);
+        completedReadList.add(buffer);
+      } else {
+        freeList.push(buffer.getBufferindex());
+        // buffer should go out of scope after the end of the calling method in ReadBufferWorker, and eligible for GC
+      }
+    }
+    //outside the synchronized, since anyone receiving a wake-up from the latch must see safe-published results
+    buffer.getLatch().countDown(); // wake up waiting threads (if any)
+  }
+
+  /**
+   * Similar to System.currentTimeMillis, except implemented with System.nanoTime().
+   * System.currentTimeMillis can go backwards when system clock is changed (e.g., with NTP time synchronization),
+   * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing,
+   * so it is much more suitable to measuring intervals.
+   *
+   * @return current time in milliseconds
+   */
+  private long currentTimeMillis() {
+    return System.nanoTime() / 1000 / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java
new file mode 100644
index 0000000..2d0c96e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java
@@ -0,0 +1,72 @@
+/**
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+
+import java.util.concurrent.CountDownLatch;
+
+class ReadBufferWorker implements Runnable {
+
+  protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1);
+  private int id;
+
+  ReadBufferWorker(final int id) {
+    this.id = id;
+  }
+
+  /**
+   * return the ID of ReadBufferWorker.
+   */
+  public int getId() {
+    return this.id;
+  }
+
+  /**
+   * Waits until a buffer becomes available in ReadAheadQueue.
+   * Once a buffer becomes available, reads the file specified in it and then posts results back to buffer manager.
+   * Rinse and repeat. Forever.
+   */
+  public void run() {
+    try {
+      UNLEASH_WORKERS.await();
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+    }
+    ReadBufferManager bufferManager = ReadBufferManager.getBufferManager();
+    ReadBuffer buffer;
+    while (true) {
+      try {
+        buffer = bufferManager.getNextBlockToRead();   // blocks, until a buffer is available for this thread
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        return;
+      }
+      if (buffer != null) {
+        try {
+          // do the actual read, from the file.
+          int bytesRead = buffer.getStream().readRemote(buffer.getOffset(), buffer.getBuffer(), 0, buffer.getRequestedLength());
+          bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead);  // post result back to ReadBufferManager
+        } catch (Exception ex) {
+          bufferManager.doneReading(buffer, ReadBufferStatus.READ_FAILED, 0);
+        }
+      }
+    }
+  }
+}


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


[45/50] [abbrv] hadoop git commit: HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/FileSystemOperationUnhandledException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/FileSystemOperationUnhandledException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/FileSystemOperationUnhandledException.java
new file mode 100644
index 0000000..484c838
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/FileSystemOperationUnhandledException.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.fs.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when an unhandled exception is occurred during a file system operation.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class FileSystemOperationUnhandledException extends AzureBlobFileSystemException {
+  public FileSystemOperationUnhandledException(Exception innerException) {
+    super("An unhandled file operation exception", innerException);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java
new file mode 100644
index 0000000..aba1d8c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.fs.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+
+/**
+ * Exception to wrap invalid Azure service error responses.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class InvalidAbfsRestOperationException extends AbfsRestOperationException {
+  public InvalidAbfsRestOperationException(
+      final Exception innerException) {
+    super(
+        AzureServiceErrorCode.UNKNOWN.getStatusCode(),
+        AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+        "InvalidAbfsRestOperationException",
+        innerException);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java
new file mode 100644
index 0000000..7591bac
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java
@@ -0,0 +1,37 @@
+/**
+ * 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.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when a configuration value is invalid
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class InvalidConfigurationValueException extends AzureBlobFileSystemException {
+  public InvalidConfigurationValueException(String configKey, Exception innerException) {
+    super("Invalid configuration value detected for " + configKey, innerException);
+  }
+
+  public InvalidConfigurationValueException(String configKey) {
+    super("Invalid configuration value detected for " + configKey);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidFileSystemPropertyException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidFileSystemPropertyException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidFileSystemPropertyException.java
new file mode 100644
index 0000000..5823fd2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidFileSystemPropertyException.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.fs.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when a file system property is invalid.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class InvalidFileSystemPropertyException extends AzureBlobFileSystemException {
+  public InvalidFileSystemPropertyException(String property) {
+    super(String.format("%s is invalid.", property));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriAuthorityException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriAuthorityException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriAuthorityException.java
new file mode 100644
index 0000000..7aa319c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriAuthorityException.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.fs.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when URI authority is invalid.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class InvalidUriAuthorityException extends AzureBlobFileSystemException {
+  public InvalidUriAuthorityException(String url) {
+    super(String.format("%s has invalid authority.", url));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java
new file mode 100644
index 0000000..a84495a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.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.fs.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when URI is invalid.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class InvalidUriException extends AzureBlobFileSystemException {
+  public InvalidUriException(String url) {
+    super(String.format("%s is invalid.", url));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java
new file mode 100644
index 0000000..694d902
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.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.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
+
+/**
+ * Thrown a service is either not configured to be injected or the service is not existing.
+ * For service registration
+ * @see AbfsServiceProviderImpl
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ServiceResolutionException extends AzureBlobFileSystemException {
+  public ServiceResolutionException(String serviceName, Exception innerException) {
+    super(String.format("%s cannot be resolved.", serviceName), innerException);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TimeoutException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TimeoutException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TimeoutException.java
new file mode 100644
index 0000000..8dd5d71
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TimeoutException.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.fs.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when a timeout happens.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class TimeoutException extends AzureBlobFileSystemException {
+  public TimeoutException(String message) {
+    super(message);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java
new file mode 100644
index 0000000..e4c75f4
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts.exceptions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java
new file mode 100644
index 0000000..67f5633
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java
new file mode 100644
index 0000000..c433f9a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java
@@ -0,0 +1,39 @@
+/**
+ * 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.azurebfs.contracts.services;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+
+/**
+ * AbfsClient factory.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface AbfsHttpClientFactory extends InjectableService {
+  /**
+   * Creates and configures an instance of new AbfsClient
+   * @return AbfsClient instance
+   */
+  AbfsClient create(AzureBlobFileSystem fs) throws AzureBlobFileSystemException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java
new file mode 100644
index 0000000..3107fa3
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java
@@ -0,0 +1,162 @@
+/**
+ * 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.azurebfs.contracts.services;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Hashtable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+
+/**
+ * File System http service to provide network calls for file system operations.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface AbfsHttpService extends InjectableService {
+  /**
+   * Gets filesystem properties on the Azure service.
+   * @param azureBlobFileSystem filesystem to get the properties.
+   * @return Hashtable<String, String> hash table containing all the filesystem properties.
+   */
+  Hashtable<String, String> getFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
+
+
+  /**
+   * Sets filesystem properties on the Azure service.
+   * @param azureBlobFileSystem filesystem to get the properties.
+   * @param properties file system properties to set.
+   */
+  void setFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem, Hashtable<String, String> properties) throws
+      AzureBlobFileSystemException;
+
+  /**
+   * Gets path properties on the Azure service.
+   * @param azureBlobFileSystem filesystem to get the properties of the path.
+   * @param path path to get properties.
+   * @return Hashtable<String, String> hash table containing all the path properties.
+   */
+  Hashtable<String, String> getPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
+
+  /**
+   * Sets path properties on the Azure service.
+   * @param azureBlobFileSystem filesystem to get the properties of the path.
+   * @param path path to set properties.
+   * @param properties hash table containing all the path properties.
+   */
+  void setPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path, Hashtable<String, String> properties) throws
+      AzureBlobFileSystemException;
+
+  /**
+   * Creates filesystem on the Azure service.
+   * @param azureBlobFileSystem filesystem to be created.
+   */
+  void createFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
+
+  /**
+   * Deletes filesystem on the Azure service.
+   * @param azureBlobFileSystem filesystem to be deleted.
+   */
+  void deleteFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
+
+  /**
+   * Creates a file on the Azure service.
+   * @param azureBlobFileSystem filesystem to create file or directory.
+   * @param path path of the file to be created.
+   * @param overwrite should overwrite.
+   * @return OutputStream stream to the file.
+   */
+  OutputStream createFile(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException;
+
+  /**
+   * Creates a directory on the Azure service.
+   * @param azureBlobFileSystem filesystem to create file or directory.
+   * @param path path of the directory to be created.
+   * @return OutputStream stream to the file.
+   */
+  Void createDirectory(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
+
+  /**
+   * Opens a file to read and returns the stream.
+   * @param azureBlobFileSystem filesystem to read a file from.
+   * @param path file path to read.
+   * @return InputStream a stream to the file to read.
+   */
+  InputStream openFileForRead(AzureBlobFileSystem azureBlobFileSystem, Path path, FileSystem.Statistics statistics) throws AzureBlobFileSystemException;
+
+  /**
+   * Opens a file to write and returns the stream.
+   * @param azureBlobFileSystem filesystem to write a file to.
+   * @param path file path to write.
+   * @param overwrite should overwrite.
+   * @return OutputStream a stream to the file to write.
+   */
+  OutputStream openFileForWrite(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException;
+
+  /**
+   * Renames a file or directory from source to destination.
+   * @param azureBlobFileSystem filesystem to rename a path.
+   * @param source source path.
+   * @param destination destination path.
+   */
+  void rename(AzureBlobFileSystem azureBlobFileSystem, Path source, Path destination) throws AzureBlobFileSystemException;
+
+  /**
+   * Deletes a file or directory.
+   * @param azureBlobFileSystem filesystem to delete the path.
+   * @param path file path to be deleted.
+   * @param recursive true if path is a directory and recursive deletion is desired.
+   */
+  void delete(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean recursive) throws AzureBlobFileSystemException;
+
+  /**
+   * Gets path's status under the provided path on the Azure service.
+   * @param azureBlobFileSystem filesystem to perform the get file status operation.
+   * @param path path delimiter.
+   * @return FileStatus FileStatus of the path in the file system.
+   */
+  FileStatus getFileStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
+
+  /**
+   * Lists all the paths under the provided path on the Azure service.
+   * @param azureBlobFileSystem filesystem to perform the list operation.
+   * @param path path delimiter.
+   * @return FileStatus[] list of all paths in the file system.
+   */
+  FileStatus[] listStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
+
+  /**
+   * Closes the client to filesystem to Azure service.
+   * @param azureBlobFileSystem filesystem to perform the list operation.
+   */
+  void closeFileSystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
+
+  /**
+   * Checks for the given path if it is marked as atomic rename directory or not.
+   * @param key
+   * @return True if the given path is listed under atomic rename property otherwise False.
+   */
+  boolean isAtomicRenameKey(String key);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java
new file mode 100644
index 0000000..bd98bae
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java
@@ -0,0 +1,40 @@
+/**
+ * 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.azurebfs.contracts.services;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException;
+
+/**
+ * Dependency injected Azure Storage services provider interface.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface AbfsServiceProvider {
+  /**
+   * Returns an instance of resolved injectable service by class name.
+   * The injectable service must be configured first to be resolvable.
+   * @param clazz the injectable service which is expected to be returned.
+   * @param <T> The type of injectable service.
+   * @return T instance
+   * @throws ServiceResolutionException if the service is not resolvable.
+   */
+  <T extends InjectableService> T get(Class<T> clazz) throws ServiceResolutionException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
new file mode 100644
index 0000000..90e580f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
@@ -0,0 +1,112 @@
+/**
+ * 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.azurebfs.contracts.services;
+
+import java.net.HttpURLConnection;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Azure service error codes.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum AzureServiceErrorCode {
+  FILE_SYSTEM_ALREADY_EXISTS("FilesystemAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null),
+  PATH_ALREADY_EXISTS("PathAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null),
+  PATH_CONFLICT("PathConflict", HttpURLConnection.HTTP_CONFLICT, null),
+  FILE_SYSTEM_NOT_FOUND("FilesystemNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
+  PATH_NOT_FOUND("PathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
+  PRE_CONDITION_FAILED("PreconditionFailed", HttpURLConnection.HTTP_PRECON_FAILED, null),
+  SOURCE_PATH_NOT_FOUND("SourcePathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
+  INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE("InvalidSourceOrDestinationResourceType", HttpURLConnection.HTTP_CONFLICT, null),
+  RENAME_DESTINATION_PARENT_PATH_NOT_FOUND("RenameDestinationParentPathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
+  INVALID_RENAME_SOURCE_PATH("InvalidRenameSourcePath", HttpURLConnection.HTTP_CONFLICT, null),
+  INGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Ingress is over the account limit."),
+  EGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Egress is over the account limit."),
+  UNKNOWN(null, -1, null);
+
+  private final String errorCode;
+  private final int httpStatusCode;
+  private final String errorMessage;
+  AzureServiceErrorCode(String errorCode, int httpStatusCodes, String errorMessage) {
+    this.errorCode = errorCode;
+    this.httpStatusCode = httpStatusCodes;
+    this.errorMessage = errorMessage;
+  }
+
+  public int getStatusCode() {
+    return this.httpStatusCode;
+  }
+
+  public String getErrorCode() {
+    return this.errorCode;
+  }
+
+  public static List<AzureServiceErrorCode> getAzureServiceCode(int httpStatusCode) {
+    List<AzureServiceErrorCode> errorCodes = new ArrayList<>();
+    if (httpStatusCode == UNKNOWN.httpStatusCode) {
+      errorCodes.add(UNKNOWN);
+      return errorCodes;
+    }
+
+    for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) {
+      if (azureServiceErrorCode.httpStatusCode == httpStatusCode) {
+        errorCodes.add(azureServiceErrorCode);
+      }
+    }
+
+    return errorCodes;
+  }
+
+  public static AzureServiceErrorCode getAzureServiceCode(int httpStatusCode, String errorCode) {
+    if (errorCode == null || errorCode.isEmpty() || httpStatusCode == UNKNOWN.httpStatusCode) {
+      return UNKNOWN;
+    }
+
+    for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) {
+      if (errorCode.equalsIgnoreCase(azureServiceErrorCode.errorCode)
+          && azureServiceErrorCode.httpStatusCode == httpStatusCode) {
+        return azureServiceErrorCode;
+      }
+    }
+
+    return UNKNOWN;
+  }
+
+  public static AzureServiceErrorCode getAzureServiceCode(int httpStatusCode, String errorCode, final String errorMessage) {
+    if (errorCode == null || errorCode.isEmpty() || httpStatusCode == UNKNOWN.httpStatusCode || errorMessage == null || errorMessage.isEmpty()) {
+      return UNKNOWN;
+    }
+
+    for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) {
+      if (azureServiceErrorCode.httpStatusCode == httpStatusCode
+          && errorCode.equalsIgnoreCase(azureServiceErrorCode.errorCode)
+          && errorMessage.equalsIgnoreCase(azureServiceErrorCode.errorMessage)
+      ) {
+        return azureServiceErrorCode;
+      }
+    }
+
+    return UNKNOWN;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java
new file mode 100644
index 0000000..ee40c9d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java
@@ -0,0 +1,143 @@
+/**
+ * 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.azurebfs.contracts.services;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
+
+/**
+ * Configuration service collects required Azure Hadoop configurations and provides it to the consumers.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface ConfigurationService extends InjectableService {
+  /**
+   * Checks if ABFS is running from Emulator;
+   * @return is emulator mode.
+   */
+  boolean isEmulator();
+
+  /**
+   * Retrieves storage secure mode from Hadoop configuration;
+   * @return storage secure mode;
+   */
+  boolean isSecureMode();
+
+  /**
+   * Retrieves storage account key for provided account name from Hadoop configuration.
+   * @param accountName the account name to retrieve the key.
+   * @return storage account key;
+   */
+  String getStorageAccountKey(String accountName) throws ConfigurationPropertyNotFoundException;
+
+  /**
+   * Returns Hadoop configuration.
+   * @return Hadoop configuration.
+   */
+  Configuration getConfiguration();
+
+  /**
+   * Retrieves configured write buffer size
+   * @return the size of the write buffer
+   */
+  int getWriteBufferSize();
+
+  /**
+   * Retrieves configured read buffer size
+   * @return the size of the read buffer
+   */
+  int getReadBufferSize();
+
+  /**
+   * Retrieves configured min backoff interval
+   * @return min backoff interval
+   */
+  int getMinBackoffIntervalMilliseconds();
+
+  /**
+   * Retrieves configured max backoff interval
+   * @return max backoff interval
+   */
+  int getMaxBackoffIntervalMilliseconds();
+
+  /**
+   * Retrieves configured backoff interval
+   * @return backoff interval
+   */
+  int getBackoffIntervalMilliseconds();
+
+  /**
+   * Retrieves configured num of retries
+   * @return num of retries
+   */
+  int getMaxIoRetries();
+
+  /**
+   * Retrieves configured azure block size
+   * @return azure block size
+   */
+  long getAzureBlockSize();
+
+  /**
+   * Retrieves configured azure block location host
+   * @return azure block location host
+   */
+  String getAzureBlockLocationHost();
+
+  /**
+   * Retrieves configured number of concurrent threads
+   * @return number of concurrent write threads
+   */
+  int getMaxConcurrentWriteThreads();
+
+  /**
+   * Retrieves configured number of concurrent threads
+   * @return number of concurrent read threads
+   */
+  int getMaxConcurrentReadThreads();
+
+  /**
+   * Retrieves configured boolean for tolerating out of band writes to files
+   * @return configured boolean for tolerating out of band writes to files
+   */
+  boolean getTolerateOobAppends();
+
+  /**
+   * Retrieves the comma-separated list of directories to receive special treatment so that folder
+   * rename is made atomic. The default value for this setting is just '/hbase'.
+   * Example directories list : <value>/hbase,/data</value>
+   * @see <a href="https://hadoop.apache.org/docs/stable/hadoop-azure/index.html#Configuring_Credentials">AtomicRenameProperty</a>
+   * @return atomic rename directories
+   */
+  String getAzureAtomicRenameDirs();
+
+  /**
+   * Retrieves configured boolean for creating remote file system during initialization
+   * @return configured boolean for creating remote file system during initialization
+   */
+  boolean getCreateRemoteFileSystemDuringInitialization();
+
+  /**
+   * Retrieves configured value of read ahead queue
+   * @return depth of read ahead
+   */
+  int getReadAheadQueueDepth();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java
new file mode 100644
index 0000000..8b3801f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java
@@ -0,0 +1,30 @@
+/**
+ * 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.azurebfs.contracts.services;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Marker interface for all the injectable services.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface InjectableService {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java
new file mode 100644
index 0000000..02a7ac9
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.contracts.services;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * The ListResultEntrySchema model.
+ */
+@InterfaceStability.Evolving
+public class ListResultEntrySchema {
+  /**
+   * The name property.
+   */
+  @JsonProperty(value = "name")
+  private String name;
+
+  /**
+   * The isDirectory property.
+   */
+  @JsonProperty(value = "isDirectory")
+  private Boolean isDirectory;
+
+  /**
+   * The lastModified property.
+   */
+  @JsonProperty(value = "lastModified")
+  private String lastModified;
+
+  /**
+   * The eTag property.
+   */
+  @JsonProperty(value = "etag")
+  private String eTag;
+
+  /**
+   * The contentLength property.
+   */
+  @JsonProperty(value = "contentLength")
+  private Long contentLength;
+
+  /**
+   * Get the name value.
+   *
+   * @return the name value
+   */
+  public String name() {
+    return this.name;
+  }
+
+  /**
+   * Set the name value.
+   *
+   * @param name the name value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withName(String name) {
+    this.name = name;
+    return this;
+  }
+
+  /**
+   * Get the isDirectory value.
+   *
+   * @return the isDirectory value
+   */
+  public Boolean isDirectory() {
+    return this.isDirectory;
+  }
+
+  /**
+   * Set the isDirectory value.
+   *
+   * @param isDirectory the isDirectory value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withIsDirectory(final Boolean isDirectory) {
+    this.isDirectory = isDirectory;
+    return this;
+  }
+
+  /**
+   * Get the lastModified value.
+   *
+   * @return the lastModified value
+   */
+  public String lastModified() {
+    return this.lastModified;
+  }
+
+  /**
+   * Set the lastModified value.
+   *
+   * @param lastModified the lastModified value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withLastModified(String lastModified) {
+    this.lastModified = lastModified;
+    return this;
+  }
+
+  /**
+   * Get the etag value.
+   *
+   * @return the etag value
+   */
+  public String eTag() {
+    return this.eTag;
+  }
+
+  /**
+   * Set the eTag value.
+   *
+   * @param eTag the eTag value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withETag(final String eTag) {
+    this.eTag = eTag;
+    return this;
+  }
+
+  /**
+   * Get the contentLength value.
+   *
+   * @return the contentLength value
+   */
+  public Long contentLength() {
+    return this.contentLength;
+  }
+
+  /**
+   * Set the contentLength value.
+   *
+   * @param contentLength the contentLength value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withContentLength(final Long contentLength) {
+    this.contentLength = contentLength;
+    return this;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java
new file mode 100644
index 0000000..baf06dc
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.contracts.services;
+
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * The ListResultSchema model.
+ */
+@InterfaceStability.Evolving
+public class ListResultSchema {
+  /**
+   * The paths property.
+   */
+  @JsonProperty(value = "paths")
+  private List<ListResultEntrySchema> paths;
+
+  /**
+   * * Get the paths value.
+   *
+   * @return the paths value
+   */
+  public List<ListResultEntrySchema> paths() {
+    return this.paths;
+  }
+
+  /**
+   * Set the paths value.
+   *
+   * @param paths the paths value to set
+   * @return the ListSchema object itself.
+   */
+  public ListResultSchema withPaths(final List<ListResultEntrySchema> paths) {
+    this.paths = paths;
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java
new file mode 100644
index 0000000..ad750c8
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java
@@ -0,0 +1,29 @@
+/**
+ * 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.azurebfs.contracts.services;
+
+/**
+ * The ReadBufferStatus for Rest AbfsClient
+ */
+public enum ReadBufferStatus {
+  NOT_AVAILABLE,  // buffers sitting in readaheadqueue have this stats
+  READING_IN_PROGRESS,  // reading is in progress on this buffer. Buffer should be in inProgressList
+  AVAILABLE,  // data is available in buffer. It should be in completedList
+  READ_FAILED  // read completed, but failed.
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java
new file mode 100644
index 0000000..267d11f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java
@@ -0,0 +1,66 @@
+/**
+ * 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.azurebfs.contracts.services;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.TraceScope;
+
+/**
+ * Azure Blob File System tracing service.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface TracingService extends InjectableService {
+  /**
+   * Creates a {@link TraceScope} object with the provided description.
+   * @param description the trace description.
+   * @return created traceScope.
+   */
+  TraceScope traceBegin(String description);
+
+  /**
+   * Creates a {@link TraceScope} object with the provided description.
+   * @param description the trace description.
+   * @param parentSpanId the span id of the parent trace scope.
+   * @return create traceScope
+   */
+  TraceScope traceBegin(String description, SpanId parentSpanId);
+
+  /**
+   * Gets current thread latest generated traceScope id.
+   * @return current thread latest generated traceScope id.
+   */
+  SpanId getCurrentTraceScopeSpanId();
+
+  /**
+   * Appends the provided exception to the trace scope.
+   * @param traceScope the scope which exception needs to be attached to.
+   * @param azureBlobFileSystemException the exception to be attached to the scope.
+   */
+  void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException);
+
+  /**
+   * Ends the provided traceScope.
+   * @param traceScope the scope that needs to be ended.
+   */
+  void traceEnd(TraceScope traceScope);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java
new file mode 100644
index 0000000..8b8a597
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts.services;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java
new file mode 100644
index 0000000..69288c5
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java
@@ -0,0 +1,50 @@
+/**
+ * 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.azurebfs.diagnostics;
+
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.commons.codec.binary.Base64;
+
+/**
+* String Base64 configuration value Validator
+*/
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class Base64StringConfigurationBasicValidator extends ConfigurationBasicValidator<String> implements ConfigurationValidator{
+
+  public Base64StringConfigurationBasicValidator(final String configKey, final String defaultVal, final boolean throwIfInvalid){
+    super(configKey, defaultVal, throwIfInvalid);
+  }
+
+  public String validate(final String configValue) throws InvalidConfigurationValueException {
+    String result = super.validate((configValue));
+    if (result != null) {
+      return result;
+    }
+
+    if (!Base64.isBase64(configValue)) {
+      throw new InvalidConfigurationValueException(getConfigKey());
+    }
+    return configValue;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java
new file mode 100644
index 0000000..c9927ff
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * Boolean configuration value validator
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BooleanConfigurationBasicValidator extends ConfigurationBasicValidator<Boolean> {
+  private static final String TRUE = "true";
+  private static final String FALSE = "false";
+
+  public BooleanConfigurationBasicValidator(final String configKey, final boolean defaultVal, final boolean throwIfInvalid) {
+    super(configKey, defaultVal, throwIfInvalid);
+  }
+
+  public Boolean validate(final String configValue) throws InvalidConfigurationValueException {
+    Boolean result = super.validate(configValue);
+    if (result != null) {
+      return result;
+    }
+
+    if (configValue.equalsIgnoreCase(TRUE) || configValue.equalsIgnoreCase(FALSE)) {
+      return Boolean.valueOf(configValue);
+    }
+
+    throw new InvalidConfigurationValueException(getConfigKey());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java
new file mode 100644
index 0000000..7da809c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.diagnostics;
+
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * ConfigurationBasicValidator covers the base case of missing user defined configuration value
+ * @param <T> the type of the validated value
+ */
+abstract class ConfigurationBasicValidator<T> implements ConfigurationValidator {
+  private final T defaultVal;
+  private final String configKey;
+  private final boolean throwIfInvalid;
+
+  ConfigurationBasicValidator(final String configKey, final T defaultVal, final boolean throwIfInvalid) {
+    this.configKey = configKey;
+    this.defaultVal = defaultVal;
+    this.throwIfInvalid = throwIfInvalid;
+  }
+
+  /**
+   * This method handles the base case where the configValue is null, based on the throwIfInvalid it either throws or returns the defaultVal,
+   * otherwise it returns null indicating that the configValue needs to be validated further
+   * @param configValue the configuration value set by the user
+   * @return the defaultVal in case the configValue is null and not required to be set, null in case the configValue not null
+   * @throws InvalidConfigurationValueException in case the configValue is null and required to be set
+   */
+  public T validate(final String configValue) throws InvalidConfigurationValueException {
+    if (configValue == null) {
+      if (this.throwIfInvalid) {
+        throw new InvalidConfigurationValueException(this.configKey);
+      }
+      return this.defaultVal;
+    }
+    return null;
+  }
+
+  public T getDefaultVal() {
+    return this.defaultVal;
+  }
+
+  public String getConfigKey() {
+    return this.configKey;
+  }
+
+  public boolean getThrowIfInvalid() {
+    return this.throwIfInvalid;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java
new file mode 100644
index 0000000..ec38cd8
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * Integer configuration value Validator
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class IntegerConfigurationBasicValidator extends ConfigurationBasicValidator<Integer> implements ConfigurationValidator {
+  private final int min;
+  private final int max;
+
+  public IntegerConfigurationBasicValidator(final int min, final int max, final int defaultVal, final String configKey, final boolean throwIfInvalid) {
+    super(configKey, defaultVal, throwIfInvalid);
+    this.min = min;
+    this.max = max;
+  }
+
+  public Integer validate(final String configValue) throws InvalidConfigurationValueException {
+    Integer result = super.validate(configValue);
+    if (result != null) {
+      return result;
+    }
+
+    try {
+      result = Integer.parseInt(configValue);
+      // throw an exception if a 'within bounds' value is missing
+      if (getThrowIfInvalid() && (result < this.min || result > this.max)) {
+        throw new InvalidConfigurationValueException(getConfigKey());
+      }
+
+      // set the value to the nearest bound if it's out of bounds
+      if (result < this.min) {
+        return this.min;
+      }
+
+      if (result > this.max) {
+        return this.max;
+      }
+    } catch (NumberFormatException ex) {
+      throw new InvalidConfigurationValueException(getConfigKey(), ex);
+    }
+
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java
new file mode 100644
index 0000000..559dbc0
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azurebfs.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * Long configuration value Validator
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class LongConfigurationBasicValidator extends ConfigurationBasicValidator<Long> implements ConfigurationValidator {
+  private final long min;
+  private final long max;
+
+  public LongConfigurationBasicValidator(final long min, final long max, final long defaultVal, final String configKey, final boolean throwIfInvalid) {
+    super(configKey, defaultVal, throwIfInvalid);
+    this.min = min;
+    this.max = max;
+  }
+
+  public Long validate(final String configValue) throws InvalidConfigurationValueException {
+    Long result = super.validate(configValue);
+    if (result != null) {
+      return result;
+    }
+
+    try {
+      result = Long.parseLong(configValue);
+      // throw an exception if a 'within bounds' value is missing
+      if (getThrowIfInvalid() && (result < this.min || result > this.max)) {
+        throw new InvalidConfigurationValueException(getConfigKey());
+      }
+
+      // set the value to the nearest bound if it's out of bounds
+      if (result < this.min) {
+        return this.min;
+      } else if (result > this.max) {
+        return this.max;
+      }
+    } catch (NumberFormatException ex) {
+      throw new InvalidConfigurationValueException(getConfigKey(), ex);
+    }
+
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java
new file mode 100644
index 0000000..d6f9c59
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java
@@ -0,0 +1,45 @@
+/**
+ * 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.azurebfs.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * String configuration value Validator
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class StringConfigurationBasicValidator extends ConfigurationBasicValidator<String> implements ConfigurationValidator{
+
+  public StringConfigurationBasicValidator(final String configKey, final String defaultVal, final boolean throwIfInvalid){
+    super(configKey, defaultVal, throwIfInvalid);
+  }
+
+  public String validate(final String configValue) throws InvalidConfigurationValueException {
+    String result = super.validate((configValue));
+    if (result != null) {
+      return result;
+    }
+
+    return configValue;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java
new file mode 100644
index 0000000..c3434ac
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.diagnostics;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html
new file mode 100644
index 0000000..5333cec
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html
@@ -0,0 +1,31 @@
+<html>
+
+<!--
+   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.
+-->
+
+<body>
+
+<p>
+A distributed implementation of {@link
+org.apache.hadoop.fs.FileSystem} for reading and writing files on
+<a href="http://store.azure.com">Azure Storage</a>.
+This implementation stores files on Azure in their native form for
+interoperability with other Azure tools.
+</p>
+
+</body>
+</html>


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


[39/50] [abbrv] hadoop git commit: HADOOP-15659. Code changes for bug fix and new tests. Contributed by Da Zhou.

Posted by tm...@apache.org.
HADOOP-15659. Code changes for bug fix and new tests.
Contributed by Da Zhou.


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

Branch: refs/heads/HADOOP-15407
Commit: ee6866de626b898c9e2085afc71cbe90df946841
Parents: 75b184c
Author: Thomas Marquardt <tm...@microsoft.com>
Authored: Sat Aug 11 00:10:26 2018 +0000
Committer: Thomas Marquardt <tm...@microsoft.com>
Committed: Sat Aug 11 03:42:27 2018 +0000

----------------------------------------------------------------------
 hadoop-tools/hadoop-azure/pom.xml               |  26 +-
 .../hadoop/fs/azurebfs/AbfsConfiguration.java   | 356 +++++++++++++++++++
 .../hadoop/fs/azurebfs/AzureBlobFileSystem.java |  55 ++-
 .../fs/azurebfs/AzureBlobFileSystemStore.java   |  39 +-
 .../azurebfs/constants/ConfigurationKeys.java   |   6 +
 .../constants/FileSystemConfigurations.java     |   4 +-
 .../exceptions/KeyProviderException.java        |  42 +++
 .../services/AzureServiceErrorCode.java         |   1 +
 .../services/ListResultEntrySchema.java         |   2 +-
 .../contracts/services/ListResultSchema.java    |   2 +-
 .../hadoop/fs/azurebfs/services/AbfsClient.java |  26 +-
 .../fs/azurebfs/services/AbfsConfiguration.java | 297 ----------------
 .../fs/azurebfs/services/AbfsHttpOperation.java |  19 +-
 .../fs/azurebfs/services/AbfsInputStream.java   |   2 +-
 .../fs/azurebfs/services/AbfsOutputStream.java  |  25 +-
 .../fs/azurebfs/services/AbfsRestOperation.java |   2 +-
 .../azurebfs/services/AbfsUriQueryBuilder.java  |   8 +-
 .../fs/azurebfs/services/KeyProvider.java       |  42 +++
 .../services/ShellDecryptionKeyProvider.java    |  63 ++++
 .../fs/azurebfs/services/SimpleKeyProvider.java |  54 +++
 .../azurebfs/AbstractAbfsIntegrationTest.java   |  17 +-
 .../hadoop/fs/azurebfs/ITestAbfsClient.java     |  45 +++
 .../fs/azurebfs/ITestAbfsReadWriteAndSeek.java  |  89 +++++
 .../azurebfs/ITestAzureBlobFileSystemE2E.java   |   2 +-
 .../ITestAzureBlobFileSystemE2EScale.java       |   4 +-
 .../ITestAzureBlobFileSystemFinalize.java       |  60 ++++
 .../azurebfs/ITestAzureBlobFileSystemFlush.java | 136 ++++++-
 .../ITestAzureBlobFileSystemInitAndCreate.java  |   4 +-
 .../ITestAzureBlobFileSystemRename.java         |   3 +-
 .../fs/azurebfs/ITestFileSystemProperties.java  |   4 -
 .../TestAbfsConfigurationFieldsValidation.java  | 149 ++++++++
 .../contract/AbfsFileSystemContract.java        |   5 +-
 .../services/ITestAbfsReadWriteAndSeek.java     |  91 -----
 .../fs/azurebfs/services/TestAbfsClient.java    |  60 ++++
 .../TestAbfsConfigurationFieldsValidation.java  | 147 --------
 .../TestShellDecryptionKeyProvider.java         |  89 +++++
 36 files changed, 1344 insertions(+), 632 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml
index cbd4dfb..7d0406c 100644
--- a/hadoop-tools/hadoop-azure/pom.xml
+++ b/hadoop-tools/hadoop-azure/pom.xml
@@ -149,17 +149,6 @@
       <scope>provided</scope>
     </dependency>
 
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-      <scope>compile</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-      <scope>compile</scope>
-    </dependency>
     
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>
@@ -198,17 +187,24 @@
     </dependency>
 
     <dependency>
-      <groupId>joda-time</groupId>
-      <artifactId>joda-time</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util-ajax</artifactId>
       <scope>compile</scope>
     </dependency>
 
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-util-ajax</artifactId>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
       <scope>compile</scope>
     </dependency>
 
+
+
     <!-- dependencies use for test only -->
     <dependency>
       <groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
new file mode 100644
index 0000000..1fb5df9
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
@@ -0,0 +1,356 @@
+/**
+ * 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.azurebfs;
+
+import java.lang.reflect.Field;
+import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException;
+import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.services.KeyProvider;
+import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider;
+
+/**
+ * Configuration for Azure Blob FileSystem.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class AbfsConfiguration{
+  private final Configuration configuration;
+  private final boolean isSecure;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE,
+      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
+      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
+      DefaultValue = FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE)
+  private int writeBufferSize;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_READ_BUFFER_SIZE,
+      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
+      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
+      DefaultValue = FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE)
+  private int readBufferSize;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL,
+      DefaultValue = FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL)
+  private int minBackoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL,
+      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL)
+  private int maxBackoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BACKOFF_INTERVAL,
+      DefaultValue = FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL)
+  private int backoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_IO_RETRIES,
+      MinValue = 0,
+      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS)
+  private int maxIoRetries;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_SIZE_PROPERTY_NAME,
+      MinValue = 0,
+      MaxValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE,
+      DefaultValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE)
+  private long azureBlockSize;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
+      DefaultValue = FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT)
+  private String azureBlockLocationHost;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_OUT,
+      MinValue = 1,
+      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_WRITE_THREADS)
+  private int maxConcurrentWriteThreads;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_IN,
+      MinValue = 1,
+      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_READ_THREADS)
+  private int maxConcurrentReadThreads;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_TOLERATE_CONCURRENT_APPEND,
+      DefaultValue = FileSystemConfigurations.DEFAULT_READ_TOLERATE_CONCURRENT_APPEND)
+  private boolean tolerateOobAppends;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ATOMIC_RENAME_KEY,
+          DefaultValue = FileSystemConfigurations.DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES)
+  private String azureAtomicDirs;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
+      DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION)
+  private boolean createRemoteFileSystemDuringInitialization;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION,
+          DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION)
+  private boolean skipUserGroupMetadataDuringInitialization;
+
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH,
+      DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH)
+  private int readAheadQueueDepth;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ENABLE_FLUSH,
+          DefaultValue = FileSystemConfigurations.DEFAULT_ENABLE_FLUSH)
+  private boolean enableFlush;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY,
+          DefaultValue = "")
+  private String userAgentId;
+
+  private Map<String, String> storageAccountKeys;
+
+  public AbfsConfiguration(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException {
+    this.configuration = configuration;
+    this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false);
+
+    validateStorageAccountKeys();
+    Field[] fields = this.getClass().getDeclaredFields();
+    for (Field field : fields) {
+      field.setAccessible(true);
+      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateInt(field));
+      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateLong(field));
+      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateString(field));
+      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateBase64String(field));
+      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateBoolean(field));
+      }
+    }
+  }
+
+  public boolean isEmulator() {
+    return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
+  }
+
+  public boolean isSecureMode() {
+    return this.isSecure;
+  }
+
+  public String getStorageAccountKey(final String accountName) throws AzureBlobFileSystemException {
+    String key;
+    String keyProviderClass =
+            configuration.get(ConfigurationKeys.AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX + accountName);
+    KeyProvider keyProvider;
+
+    if (keyProviderClass == null) {
+      // No key provider was provided so use the provided key as is.
+      keyProvider = new SimpleKeyProvider();
+    } else {
+      // create an instance of the key provider class and verify it
+      // implements KeyProvider
+      Object keyProviderObject;
+      try {
+        Class<?> clazz = configuration.getClassByName(keyProviderClass);
+        keyProviderObject = clazz.newInstance();
+      } catch (Exception e) {
+        throw new KeyProviderException("Unable to load key provider class.", e);
+      }
+      if (!(keyProviderObject instanceof KeyProvider)) {
+        throw new KeyProviderException(keyProviderClass
+                + " specified in config is not a valid KeyProvider class.");
+      }
+      keyProvider = (KeyProvider) keyProviderObject;
+    }
+    key = keyProvider.getStorageAccountKey(accountName, configuration);
+
+    if (key == null) {
+      throw new ConfigurationPropertyNotFoundException(accountName);
+    }
+
+    return key;
+  }
+
+  public Configuration getConfiguration() {
+    return this.configuration;
+  }
+
+  public int getWriteBufferSize() {
+    return this.writeBufferSize;
+  }
+
+  public int getReadBufferSize() {
+    return this.readBufferSize;
+  }
+
+  public int getMinBackoffIntervalMilliseconds() {
+    return this.minBackoffInterval;
+  }
+
+  public int getMaxBackoffIntervalMilliseconds() {
+    return this.maxBackoffInterval;
+  }
+
+  public int getBackoffIntervalMilliseconds() {
+    return this.backoffInterval;
+  }
+
+  public int getMaxIoRetries() {
+    return this.maxIoRetries;
+  }
+
+  public long getAzureBlockSize() {
+    return this.azureBlockSize;
+  }
+
+  public String getAzureBlockLocationHost() {
+    return this.azureBlockLocationHost;
+  }
+
+  public int getMaxConcurrentWriteThreads() {
+    return this.maxConcurrentWriteThreads;
+  }
+
+  public int getMaxConcurrentReadThreads() {
+    return this.maxConcurrentReadThreads;
+  }
+
+  public boolean getTolerateOobAppends() {
+    return this.tolerateOobAppends;
+  }
+
+  public String getAzureAtomicRenameDirs() {
+    return this.azureAtomicDirs;
+  }
+
+  public boolean getCreateRemoteFileSystemDuringInitialization() {
+    return this.createRemoteFileSystemDuringInitialization;
+  }
+
+  public boolean getSkipUserGroupMetadataDuringInitialization() {
+    return this.skipUserGroupMetadataDuringInitialization;
+  }
+
+  public int getReadAheadQueueDepth() {
+    return this.readAheadQueueDepth;
+  }
+
+  public boolean isFlushEnabled() {
+    return this.enableFlush;
+  }
+
+  public String getCustomUserAgentPrefix() {
+    return this.userAgentId;
+  }
+
+  void validateStorageAccountKeys() throws InvalidConfigurationValueException {
+    Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator(
+        ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true);
+    this.storageAccountKeys = this.configuration.getValByRegex(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX);
+
+    for (Map.Entry<String, String> account : this.storageAccountKeys.entrySet()) {
+      validator.validate(account.getValue());
+    }
+  }
+
+  int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new IntegerConfigurationBasicValidator(
+        validator.MinValue(),
+        validator.MaxValue(),
+        validator.DefaultValue(),
+        validator.ConfigurationKey(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new LongConfigurationBasicValidator(
+        validator.MinValue(),
+        validator.MaxValue(),
+        validator.DefaultValue(),
+        validator.ConfigurationKey(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new StringConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class));
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new Base64StringConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new BooleanConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  @VisibleForTesting
+  void setReadBufferSize(int bufferSize) {
+    this.readBufferSize = bufferSize;
+  }
+
+  @VisibleForTesting
+  void setWriteBufferSize(int bufferSize) {
+    this.writeBufferSize = bufferSize;
+  }
+
+  @VisibleForTesting
+  void setEnableFlush(boolean enableFlush) {
+    this.enableFlush = enableFlush;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
index 9f58f6b..b0a30a0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
@@ -36,6 +36,7 @@ import java.util.concurrent.Future;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -90,7 +91,6 @@ public class AzureBlobFileSystem extends FileSystem {
     this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
     this.userGroupInformation = UserGroupInformation.getCurrentUser();
     this.user = userGroupInformation.getUserName();
-    this.primaryUserGroup = userGroupInformation.getPrimaryGroupName();
     this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation);
 
     LOG.debug("Initializing NativeAzureFileSystem for {}", uri);
@@ -98,7 +98,16 @@ public class AzureBlobFileSystem extends FileSystem {
     this.setWorkingDirectory(this.getHomeDirectory());
 
     if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) {
-      this.createFileSystem();
+      if (!this.fileSystemExists()) {
+        this.createFileSystem();
+      }
+    }
+
+    if (!abfsStore.getAbfsConfiguration().getSkipUserGroupMetadataDuringInitialization()) {
+      this.primaryUserGroup = userGroupInformation.getPrimaryGroupName();
+    } else {
+      //Provide a default group name
+      this.primaryUserGroup = this.user;
     }
   }
 
@@ -375,7 +384,7 @@ public class AzureBlobFileSystem extends FileSystem {
     if (file.getLen() < start) {
       return new BlockLocation[0];
     }
-    final String blobLocationHost = this.abfsStore.getAbfsConfiguration().getAzureBlockLocationHost();
+    final String blobLocationHost = abfsStore.getAbfsConfiguration().getAzureBlockLocationHost();
 
     final String[] name = { blobLocationHost };
     final String[] host = { blobLocationHost };
@@ -397,6 +406,13 @@ public class AzureBlobFileSystem extends FileSystem {
     return locations;
   }
 
+  @Override
+  protected void finalize() throws Throwable {
+    LOG.debug("finalize() called.");
+    close();
+    super.finalize();
+  }
+
   public String getOwnerUser() {
     return user;
   }
@@ -450,13 +466,31 @@ public class AzureBlobFileSystem extends FileSystem {
     }
   }
 
+  private boolean fileSystemExists() throws IOException {
+    LOG.debug(
+            "AzureBlobFileSystem.fileSystemExists uri: {}", uri);
+    try {
+      abfsStore.getFilesystemProperties();
+    } catch (AzureBlobFileSystemException ex) {
+      try {
+        checkException(null, ex);
+        // Because HEAD request won't contain message body,
+        // there is not way to get the storage error code
+        // workaround here is to check its status code.
+      } catch (FileNotFoundException e) {
+        return false;
+      }
+    }
+    return true;
+  }
+
   private void createFileSystem() throws IOException {
     LOG.debug(
         "AzureBlobFileSystem.createFileSystem uri: {}", uri);
     try {
-      this.abfsStore.createFilesystem();
+      abfsStore.createFilesystem();
     } catch (AzureBlobFileSystemException ex) {
-      checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS);
+      checkException(null, ex);
     }
   }
 
@@ -556,10 +590,10 @@ public class AzureBlobFileSystem extends FileSystem {
 
       //AbfsRestOperationException.getMessage() contains full error info including path/uri.
       if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) {
-        throw (IOException)new FileNotFoundException(ere.getMessage())
+        throw (IOException) new FileNotFoundException(ere.getMessage())
             .initCause(exception);
       } else if (statusCode == HttpURLConnection.HTTP_CONFLICT) {
-        throw (IOException)new FileAlreadyExistsException(ere.getMessage())
+        throw (IOException) new FileAlreadyExistsException(ere.getMessage())
             .initCause(exception);
       } else {
         throw ere;
@@ -615,6 +649,11 @@ public class AzureBlobFileSystem extends FileSystem {
 
   @VisibleForTesting
   AzureBlobFileSystemStore getAbfsStore() {
-    return this.abfsStore;
+    return abfsStore;
+  }
+
+  @VisibleForTesting
+  AbfsClient getAbfsClient() {
+    return abfsStore.getClient();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
index 8ac31ce..ba72149 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
@@ -31,8 +31,11 @@ import java.nio.charset.CharacterCodingException;
 import java.nio.charset.Charset;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CharsetEncoder;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Date;
 import java.util.HashSet;
 import java.util.Hashtable;
 import java.util.Map;
@@ -65,7 +68,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
-import org.apache.hadoop.fs.azurebfs.services.AbfsConfiguration;
 import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
@@ -75,8 +77,6 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.http.client.utils.URIBuilder;
-import org.joda.time.DateTime;
-import org.joda.time.format.DateTimeFormat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -223,7 +223,7 @@ public class AzureBlobFileSystemStore {
     final OutputStream outputStream;
     outputStream = new FSDataOutputStream(
             new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0,
-                    abfsConfiguration.getWriteBufferSize()), null);
+                    abfsConfiguration.getWriteBufferSize(), abfsConfiguration.isFlushEnabled()), null);
     return outputStream;
   }
 
@@ -287,7 +287,7 @@ public class AzureBlobFileSystemStore {
     final OutputStream outputStream;
     outputStream = new FSDataOutputStream(
             new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
-                    offset, abfsConfiguration.getWriteBufferSize()), null);
+                    offset, abfsConfiguration.getWriteBufferSize(), abfsConfiguration.isFlushEnabled()), null);
     return outputStream;
   }
 
@@ -366,7 +366,7 @@ public class AzureBlobFileSystemStore {
               true,
               1,
               blockSize,
-              parseLastModifiedTime(lastModified).getMillis(),
+              parseLastModifiedTime(lastModified),
               path,
               eTag);
     } else {
@@ -385,7 +385,7 @@ public class AzureBlobFileSystemStore {
               parseIsDirectory(resourceType),
               1,
               blockSize,
-              parseLastModifiedTime(lastModified).getMillis(),
+              parseLastModifiedTime(lastModified),
               path,
               eTag);
     }
@@ -419,10 +419,7 @@ public class AzureBlobFileSystemStore {
         long contentLength = entry.contentLength() == null ? 0 : entry.contentLength();
         boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory();
         if (entry.lastModified() != null && !entry.lastModified().isEmpty()) {
-          final DateTime dateTime = DateTime.parse(
-                  entry.lastModified(),
-                  DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
-          lastModifiedMillis = dateTime.getMillis();
+          lastModifiedMillis = parseLastModifiedTime(entry.lastModified());
         }
 
         Path entryPath = new Path(File.separator + entry.name());
@@ -534,10 +531,16 @@ public class AzureBlobFileSystemStore {
         && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
   }
 
-  private DateTime parseLastModifiedTime(final String lastModifiedTime) {
-    return DateTime.parse(
-            lastModifiedTime,
-            DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
+  private long parseLastModifiedTime(final String lastModifiedTime) {
+    long parsedTime = 0;
+    try {
+      Date utcDate = new SimpleDateFormat(DATE_TIME_PATTERN).parse(lastModifiedTime);
+      parsedTime = utcDate.getTime();
+    } catch (ParseException e) {
+      LOG.error("Failed to parse the date {0}", lastModifiedTime);
+    } finally {
+      return parsedTime;
+    }
   }
 
   private String convertXmsPropertiesToCommaSeparatedString(final Hashtable<String, String> properties) throws
@@ -663,7 +666,7 @@ public class AzureBlobFileSystemStore {
       }
 
       if (other instanceof VersionedFileStatus) {
-        return this.version.equals(((VersionedFileStatus)other).version);
+        return this.version.equals(((VersionedFileStatus) other).version);
       }
 
       return true;
@@ -702,5 +705,9 @@ public class AzureBlobFileSystemStore {
     }
   }
 
+  @VisibleForTesting
+  AbfsClient getClient() {
+    return this.client;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
index ead1003..9c805a2 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
@@ -49,9 +49,15 @@ public final class ConfigurationKeys {
   public static final String AZURE_CONCURRENT_CONNECTION_VALUE_IN = "fs.azure.concurrentRequestCount.in";
   public static final String AZURE_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append";
   public static final String AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = "fs.azure.createRemoteFileSystemDuringInitialization";
+  public static final String AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = "fs.azure.skipUserGroupMetadataDuringInitialization";
   public static final String FS_AZURE_AUTOTHROTTLING_ENABLE = "fs.azure.autothrottling.enable";
   public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key";
   public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth";
+  public static final String FS_AZURE_ENABLE_FLUSH = "fs.azure.enable.flush";
+  public static final String FS_AZURE_USER_AGENT_PREFIX_KEY = "fs.azure.user.agent.prefix";
+
+  public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX = "fs.azure.account.keyprovider.";
+  public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script";
 
   private ConfigurationKeys() {}
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
index 482158c..1655d04 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
@@ -39,7 +39,7 @@ public final class FileSystemConfigurations {
   private static final int ONE_MB = ONE_KB * ONE_KB;
 
   // Default upload and download buffer size
-  public static final int DEFAULT_WRITE_BUFFER_SIZE = 4 * ONE_MB;  // 4 MB
+  public static final int DEFAULT_WRITE_BUFFER_SIZE = 8 * ONE_MB;  // 8 MB
   public static final int DEFAULT_READ_BUFFER_SIZE = 4 * ONE_MB;  // 4 MB
   public static final int MIN_BUFFER_SIZE = 16 * ONE_KB;  // 16 KB
   public static final int MAX_BUFFER_SIZE = 100 * ONE_MB;  // 100 MB
@@ -50,10 +50,12 @@ public final class FileSystemConfigurations {
   public static final int MAX_CONCURRENT_WRITE_THREADS = 8;
   public static final boolean DEFAULT_READ_TOLERATE_CONCURRENT_APPEND = false;
   public static final boolean DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = false;
+  public static final boolean DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = false;
 
   public static final String DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES = "/hbase";
 
   public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1;
+  public static final boolean DEFAULT_ENABLE_FLUSH = true;
 
   private FileSystemConfigurations() {}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/KeyProviderException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/KeyProviderException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/KeyProviderException.java
new file mode 100644
index 0000000..6723d69
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/KeyProviderException.java
@@ -0,0 +1,42 @@
+/**
+ * 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.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Thrown if there is a problem instantiating a KeyProvider or retrieving a key
+ * using a KeyProvider object.
+ */
+@InterfaceAudience.Private
+public class KeyProviderException extends AzureBlobFileSystemException {
+  private static final long serialVersionUID = 1L;
+
+  public KeyProviderException(String message) {
+    super(message);
+  }
+
+  public KeyProviderException(String message, Throwable cause) {
+    super(message);
+  }
+
+  public KeyProviderException(Throwable t) {
+    super(t.getMessage());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
index 90e580f..a89f339 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
@@ -43,6 +43,7 @@ public enum AzureServiceErrorCode {
   INVALID_RENAME_SOURCE_PATH("InvalidRenameSourcePath", HttpURLConnection.HTTP_CONFLICT, null),
   INGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Ingress is over the account limit."),
   EGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Egress is over the account limit."),
+  INVALID_QUERY_PARAMETER_VALUE("InvalidQueryParameterValue", HttpURLConnection.HTTP_BAD_REQUEST, null),
   UNKNOWN(null, -1, null);
 
   private final String errorCode;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java
index 02a7ac9..903ff69 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.azurebfs.contracts.services;
 
-import com.fasterxml.jackson.annotation.JsonProperty;
+import org.codehaus.jackson.annotate.JsonProperty;
 
 import org.apache.hadoop.classification.InterfaceStability;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java
index baf06dc..3259742 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.fs.azurebfs.contracts.services;
 
 import java.util.List;
 
-import com.fasterxml.jackson.annotation.JsonProperty;
+import org.codehaus.jackson.annotate.JsonProperty;
 
 import org.apache.hadoop.classification.InterfaceStability;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
index 2b3ccc0..60369be 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -26,12 +26,13 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
-
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
 
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
@@ -44,7 +45,7 @@ public class AbfsClient {
   public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
   private final URL baseUrl;
   private final SharedKeyCredentials sharedKeyCredentials;
-  private final String xMsVersion = "2018-03-28";
+  private final String xMsVersion = "2018-06-17";
   private final ExponentialRetryPolicy retryPolicy;
   private final String filesystem;
   private final AbfsConfiguration abfsConfiguration;
@@ -59,7 +60,7 @@ public class AbfsClient {
     this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1);
     this.abfsConfiguration = abfsConfiguration;
     this.retryPolicy = exponentialRetryPolicy;
-    this.userAgent = initializeUserAgent();
+    this.userAgent = initializeUserAgent(abfsConfiguration);
   }
 
   public String getFileSystem() {
@@ -137,7 +138,7 @@ public class AbfsClient {
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
-    abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? "" : relativePath);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults));
@@ -380,8 +381,8 @@ public class AbfsClient {
     return url;
   }
 
-  private static String urlEncode(final String value) throws AzureBlobFileSystemException {
-    String encodedString = null;
+  public static String urlEncode(final String value) throws AzureBlobFileSystemException {
+    String encodedString;
     try {
       encodedString =  URLEncoder.encode(value, UTF_8)
           .replace(PLUS, PLUS_ENCODE)
@@ -393,14 +394,23 @@ public class AbfsClient {
     return encodedString;
   }
 
-  private String initializeUserAgent() {
+  @VisibleForTesting
+  String initializeUserAgent(final AbfsConfiguration abfsConfiguration) {
     final String userAgentComment = String.format(Locale.ROOT,
             "(JavaJRE %s; %s %s)",
             System.getProperty(JAVA_VERSION),
             System.getProperty(OS_NAME)
                     .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING),
             System.getProperty(OS_VERSION));
-
+    String customUserAgentId = abfsConfiguration.getCustomUserAgentPrefix();
+    if (customUserAgentId != null && !customUserAgentId.isEmpty()) {
+      return String.format(Locale.ROOT, CLIENT_VERSION + " %s %s", userAgentComment, customUserAgentId);
+    }
     return String.format(CLIENT_VERSION + " %s", userAgentComment);
   }
+
+  @VisibleForTesting
+  URL getBaseUrl() {
+    return baseUrl;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java
deleted file mode 100644
index 8def1bb..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java
+++ /dev/null
@@ -1,297 +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.fs.azurebfs.services;
-
-import java.lang.reflect.Field;
-import java.util.Map;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
-import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator;
-import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator;
-import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
-import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
-import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
-
-/**
- * Configuration for Azure Blob FileSystem.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class AbfsConfiguration{
-  private final Configuration configuration;
-  private final boolean isSecure;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE,
-      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
-      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
-      DefaultValue = FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE)
-  private int writeBufferSize;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_READ_BUFFER_SIZE,
-      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
-      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
-      DefaultValue = FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE)
-  private int readBufferSize;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL,
-      DefaultValue = FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL)
-  private int minBackoffInterval;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL,
-      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL)
-  private int maxBackoffInterval;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BACKOFF_INTERVAL,
-      DefaultValue = FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL)
-  private int backoffInterval;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_IO_RETRIES,
-      MinValue = 0,
-      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS)
-  private int maxIoRetries;
-
-  @LongConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_SIZE_PROPERTY_NAME,
-      MinValue = 0,
-      MaxValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE,
-      DefaultValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE)
-  private long azureBlockSize;
-
-  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
-      DefaultValue = FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT)
-  private String azureBlockLocationHost;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_OUT,
-      MinValue = 1,
-      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_WRITE_THREADS)
-  private int maxConcurrentWriteThreads;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_IN,
-      MinValue = 1,
-      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_READ_THREADS)
-  private int maxConcurrentReadThreads;
-
-  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_TOLERATE_CONCURRENT_APPEND,
-      DefaultValue = FileSystemConfigurations.DEFAULT_READ_TOLERATE_CONCURRENT_APPEND)
-  private boolean tolerateOobAppends;
-
-  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ATOMIC_RENAME_KEY,
-          DefaultValue = FileSystemConfigurations.DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES)
-  private String azureAtomicDirs;
-
-  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
-      DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION)
-  private boolean createRemoteFileSystemDuringInitialization;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH,
-      DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH)
-  private int readAheadQueueDepth;
-
-  private Map<String, String> storageAccountKeys;
-
-  public AbfsConfiguration(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException {
-    this.configuration = configuration;
-    this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false);
-
-    validateStorageAccountKeys();
-    Field[] fields = this.getClass().getDeclaredFields();
-    for (Field field : fields) {
-      field.setAccessible(true);
-      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateInt(field));
-      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateLong(field));
-      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateString(field));
-      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateBase64String(field));
-      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateBoolean(field));
-      }
-    }
-  }
-
-  public boolean isEmulator() {
-    return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
-  }
-
-  public boolean isSecureMode() {
-    return this.isSecure;
-  }
-
-  public String getStorageAccountKey(final String accountName) throws ConfigurationPropertyNotFoundException {
-    String accountKey = this.storageAccountKeys.get(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName);
-    if (accountKey == null) {
-      throw new ConfigurationPropertyNotFoundException(accountName);
-    }
-
-    return accountKey;
-  }
-
-  public Configuration getConfiguration() {
-    return this.configuration;
-  }
-
-  public int getWriteBufferSize() {
-    return this.writeBufferSize;
-  }
-
-  public int getReadBufferSize() {
-    return this.readBufferSize;
-  }
-
-  public int getMinBackoffIntervalMilliseconds() {
-    return this.minBackoffInterval;
-  }
-
-  public int getMaxBackoffIntervalMilliseconds() {
-    return this.maxBackoffInterval;
-  }
-
-  public int getBackoffIntervalMilliseconds() {
-    return this.backoffInterval;
-  }
-
-  public int getMaxIoRetries() {
-    return this.maxIoRetries;
-  }
-
-  public long getAzureBlockSize() {
-    return this.azureBlockSize;
-  }
-
-  public String getAzureBlockLocationHost() {
-    return this.azureBlockLocationHost;
-  }
-
-  public int getMaxConcurrentWriteThreads() {
-    return this.maxConcurrentWriteThreads;
-  }
-
-  public int getMaxConcurrentReadThreads() {
-    return this.maxConcurrentReadThreads;
-  }
-
-  public boolean getTolerateOobAppends() {
-    return this.tolerateOobAppends;
-  }
-
-  public String getAzureAtomicRenameDirs() {
-    return this.azureAtomicDirs;
-  }
-
-  public boolean getCreateRemoteFileSystemDuringInitialization() {
-    return this.createRemoteFileSystemDuringInitialization;
-  }
-
-  public int getReadAheadQueueDepth() {
-    return this.readAheadQueueDepth;
-  }
-
-  void validateStorageAccountKeys() throws InvalidConfigurationValueException {
-    Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator(
-        ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true);
-    this.storageAccountKeys = this.configuration.getValByRegex(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX);
-
-    for (Map.Entry<String, String> account : this.storageAccountKeys.entrySet()) {
-      validator.validate(account.getValue());
-    }
-  }
-
-  int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class);
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new IntegerConfigurationBasicValidator(
-        validator.MinValue(),
-        validator.MaxValue(),
-        validator.DefaultValue(),
-        validator.ConfigurationKey(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class);
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new LongConfigurationBasicValidator(
-        validator.MinValue(),
-        validator.MaxValue(),
-        validator.DefaultValue(),
-        validator.ConfigurationKey(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class);
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new StringConfigurationBasicValidator(
-        validator.ConfigurationKey(),
-        validator.DefaultValue(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class));
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new Base64StringConfigurationBasicValidator(
-        validator.ConfigurationKey(),
-        validator.DefaultValue(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class);
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new BooleanConfigurationBasicValidator(
-        validator.ConfigurationKey(),
-        validator.DefaultValue(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  @VisibleForTesting
-  void setReadBufferSize(int bufferSize) {
-    this.readBufferSize = bufferSize;
-  }
-
-  @VisibleForTesting
-  void setWriteBufferSize(int bufferSize) {
-    this.writeBufferSize = bufferSize;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
index 53f6900..2bfcff2 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
@@ -26,10 +26,11 @@ import java.net.URL;
 import java.util.List;
 import java.util.UUID;
 
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonToken;
-import com.fasterxml.jackson.databind.ObjectMapper;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import org.codehaus.jackson.map.ObjectMapper;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -167,7 +168,7 @@ public class AbfsHttpOperation {
    */
   public AbfsHttpOperation(final URL url, final String method, final List<AbfsHttpHeader> requestHeaders)
       throws IOException {
-    this.isTraceEnabled = this.LOG.isTraceEnabled();
+    this.isTraceEnabled = LOG.isTraceEnabled();
     this.url = url;
     this.method = method;
     this.clientRequestId = UUID.randomUUID().toString();
@@ -303,7 +304,7 @@ public class AbfsHttpOperation {
           }
         }
       } catch (IOException ex) {
-        this.LOG.error("UnexpectedError: ", ex);
+        LOG.error("UnexpectedError: ", ex);
         throw ex;
       } finally {
         if (this.isTraceEnabled) {
@@ -355,7 +356,7 @@ public class AbfsHttpOperation {
         return;
       }
       JsonFactory jf = new JsonFactory();
-      try (JsonParser jp = jf.createParser(stream)) {
+      try (JsonParser jp = jf.createJsonParser(stream)) {
         String fieldName, fieldValue;
         jp.nextToken();  // START_OBJECT - {
         jp.nextToken();  // FIELD_NAME - "error":
@@ -384,7 +385,7 @@ public class AbfsHttpOperation {
       // Ignore errors that occur while attempting to parse the storage
       // error, since the response may have been handled by the HTTP driver
       // or for other reasons have an unexpected
-      this.LOG.debug("ExpectedError: ", ex);
+      LOG.debug("ExpectedError: ", ex);
     }
   }
 
@@ -415,7 +416,7 @@ public class AbfsHttpOperation {
       final ObjectMapper objectMapper = new ObjectMapper();
       this.listResultSchema = objectMapper.readValue(stream, ListResultSchema.class);
     } catch (IOException ex) {
-      this.LOG.error("Unable to deserialize list results", ex);
+      LOG.error("Unable to deserialize list results", ex);
       throw ex;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
index 848ce8a..960579d 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
@@ -64,7 +64,7 @@ public class AbfsInputStream extends FSInputStream {
     this.path = path;
     this.contentLength = contentLength;
     this.bufferSize = bufferSize;
-    this.readAheadQueueDepth = (readAheadQueueDepth >= 0) ? readAheadQueueDepth : 2 * Runtime.getRuntime().availableProcessors();
+    this.readAheadQueueDepth = (readAheadQueueDepth >= 0) ? readAheadQueueDepth : Runtime.getRuntime().availableProcessors();
     this.eTag = eTag;
     this.tolerateOobAppends = false;
     this.readAheadEnabled = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
index 2dbcee5..b69ec83 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -43,6 +43,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
   private final String path;
   private long position;
   private boolean closed;
+  private boolean supportFlush;
   private volatile IOException lastError;
 
   private long lastFlushOffset;
@@ -61,11 +62,13 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
       final AbfsClient client,
       final String path,
       final long position,
-      final int bufferSize) {
+      final int bufferSize,
+      final boolean supportFlush) {
     this.client = client;
     this.path = path;
     this.position = position;
     this.closed = false;
+    this.supportFlush = supportFlush;
     this.lastError = null;
     this.lastFlushOffset = 0;
     this.bufferSize = bufferSize;
@@ -162,7 +165,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
    */
   @Override
   public void flush() throws IOException {
-    flushInternalAsync();
+    if (supportFlush) {
+      flushInternalAsync();
+    }
   }
 
   /** Similar to posix fsync, flush out the data in client's user buffer
@@ -171,7 +176,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
    */
   @Override
   public void hsync() throws IOException {
-    flushInternal();
+    if (supportFlush) {
+      flushInternal();
+    }
   }
 
   /** Flush out the data in client's user buffer. After the return of
@@ -180,7 +187,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
    */
   @Override
   public void hflush() throws IOException {
-    flushInternal();
+    if (supportFlush) {
+      flushInternal();
+    }
   }
 
   /**
@@ -262,7 +271,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
         writeOperation.task.get();
       } catch (Exception ex) {
         if (ex.getCause() instanceof AzureBlobFileSystemException) {
-          ex = (AzureBlobFileSystemException)ex.getCause();
+          ex = (AzureBlobFileSystemException) ex.getCause();
         }
         lastError = new IOException(ex);
         throw lastError;
@@ -277,8 +286,6 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
     if (this.lastTotalAppendOffset > this.lastFlushOffset) {
       this.flushWrittenBytesToServiceInternal(this.lastTotalAppendOffset, true);
     }
-
-    this.lastTotalAppendOffset = 0;
   }
 
   private synchronized void flushWrittenBytesToServiceInternal(final long offset,
@@ -304,7 +311,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
       }
     } catch (Exception e) {
       if (e.getCause() instanceof AzureBlobFileSystemException) {
-        lastError = (AzureBlobFileSystemException)e.getCause();
+        lastError = (AzureBlobFileSystemException) e.getCause();
       } else {
         lastError = new IOException(e);
       }
@@ -322,7 +329,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
       try {
         completionService.take();
       } catch (InterruptedException e) {
-        lastError = (IOException)new InterruptedIOException(e.toString()).initCause(e);
+        lastError = (IOException) new InterruptedIOException(e.toString()).initCause(e);
         throw lastError;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
index 6126398..6dd32fa 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
@@ -121,7 +121,7 @@ public class AbfsRestOperation {
       }
     }
 
-    if (result.getStatusCode() > HttpURLConnection.HTTP_BAD_REQUEST) {
+    if (result.getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST) {
       throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(),
           result.getStorageErrorMessage(), null, result);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
index 3624853..a200b40 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
@@ -22,6 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 
 /**
  * The UrlQueryBuilder for Rest AbfsClient.
@@ -51,7 +52,12 @@ public class AbfsUriQueryBuilder {
       } else {
         sb.append(AbfsHttpConstants.AND_MARK);
       }
-      sb.append(entry.getKey()).append(AbfsHttpConstants.EQUAL).append(entry.getValue());
+      try {
+        sb.append(entry.getKey()).append(AbfsHttpConstants.EQUAL).append(AbfsClient.urlEncode(entry.getValue()));
+      }
+      catch (AzureBlobFileSystemException ex) {
+        throw new IllegalArgumentException("Query string param is not encode-able: " + entry.getKey() + "=" + entry.getValue());
+      }
     }
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java
new file mode 100644
index 0000000..27f76f8
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java
@@ -0,0 +1,42 @@
+/**
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException;
+
+/**
+ * The interface that every Azure file system key provider must implement.
+ */
+public interface KeyProvider {
+  /**
+   * Key providers must implement this method. Given a list of configuration
+   * parameters for the specified Azure storage account, retrieve the plaintext
+   * storage account key.
+   *
+   * @param accountName
+   *          the storage account name
+   * @param conf
+   *          Hadoop configuration parameters
+   * @return the plaintext storage account key
+   * @throws KeyProviderException
+   */
+  String getStorageAccountKey(String accountName, Configuration conf)
+      throws KeyProviderException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java
new file mode 100644
index 0000000..3fc05ff
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java
@@ -0,0 +1,63 @@
+/**
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException;
+import org.apache.hadoop.util.Shell;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Shell decryption key provider which invokes an external script that will
+ * perform the key decryption.
+ */
+public class ShellDecryptionKeyProvider extends SimpleKeyProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(ShellDecryptionKeyProvider.class);
+
+  @Override
+  public String getStorageAccountKey(String accountName, Configuration conf)
+      throws KeyProviderException {
+    String envelope = super.getStorageAccountKey(accountName, conf);
+
+    final String command = conf.get(ConfigurationKeys.AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT);
+    if (command == null) {
+      throw new KeyProviderException(
+          "Script path is not specified via fs.azure.shellkeyprovider.script");
+    }
+
+    String[] cmd = command.split(" ");
+    String[] cmdWithEnvelope = Arrays.copyOf(cmd, cmd.length + 1);
+    cmdWithEnvelope[cmdWithEnvelope.length - 1] = envelope;
+
+    String decryptedKey = null;
+    try {
+      decryptedKey = Shell.execCommand(cmdWithEnvelope);
+    } catch (IOException ex) {
+      throw new KeyProviderException(ex);
+    }
+
+    // trim any whitespace
+    return decryptedKey.trim();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java
new file mode 100644
index 0000000..cedae57
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java
@@ -0,0 +1,54 @@
+/**
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException;
+import org.apache.hadoop.security.ProviderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Key provider that simply returns the storage account key from the
+ * configuration as plaintext.
+ */
+public class SimpleKeyProvider implements KeyProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(SimpleKeyProvider.class);
+
+  @Override
+  public String getStorageAccountKey(String accountName, Configuration conf)
+      throws KeyProviderException {
+    String key = null;
+    try {
+      Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
+          conf, AzureBlobFileSystem.class);
+      char[] keyChars = c.getPassword(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName);
+      if (keyChars != null) {
+        key = new String(keyChars);
+      }
+    } catch(IOException ioe) {
+      LOG.warn("Unable to get key from credential providers. {}", ioe);
+    }
+    return key;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
index 106fa09..b1f1485 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
@@ -27,10 +27,6 @@ import java.util.concurrent.Callable;
 import com.google.common.base.Preconditions;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.rules.TestName;
-import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,9 +37,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.AbstractWasbTestWithTimeout;
 import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
-import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
-import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
@@ -175,6 +169,17 @@ public abstract class AbstractAbfsIntegrationTest extends
     return abfs;
   }
 
+  public AzureBlobFileSystem getFileSystem(Configuration configuration) throws Exception{
+    final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration);
+    return fs;
+  }
+
+  public AzureBlobFileSystem getFileSystem(String abfsUri) throws Exception {
+    configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, abfsUri);
+    final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration);
+    return fs;
+  }
+
   /**
    * Creates the filesystem; updates the {@link #abfs} field.
    * @return the created filesystem.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java
new file mode 100644
index 0000000..9c369bb
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java
@@ -0,0 +1,45 @@
+/**
+ * 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.azurebfs;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test continuation token which has equal sign.
+ */
+public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
+  private static final int LIST_MAX_RESULTS = 5000;
+  @Test
+  public void testContinuationTokenHavingEqualSign() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    AbfsClient abfsClient =  fs.getAbfsClient();
+
+    try {
+      AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, "===========");
+      Assert.assertTrue(false);
+    } catch (AbfsRestOperationException ex) {
+      Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java
new file mode 100644
index 0000000..f62ea6e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java
@@ -0,0 +1,89 @@
+/**
+ * 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.azurebfs;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
+
+/**
+ * Test read, write and seek.
+ * Uses package-private methods in AbfsConfiguration, which is why it is in
+ * this package.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
+  private static final Path TEST_PATH = new Path("/testfile");
+
+  @Parameterized.Parameters(name = "Size={0}")
+  public static Iterable<Object[]> sizes() {
+    return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE},
+        {DEFAULT_READ_BUFFER_SIZE},
+        {MAX_BUFFER_SIZE}});
+  }
+
+  private final int size;
+
+  public ITestAbfsReadWriteAndSeek(final int size) {
+    this.size = size;
+  }
+
+  @Test
+  public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception {
+    testReadWriteAndSeek(size);
+  }
+
+  private void testReadWriteAndSeek(int bufferSize) throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(getConfiguration());
+
+    abfsConfiguration.setWriteBufferSize(bufferSize);
+    abfsConfiguration.setReadBufferSize(bufferSize);
+
+
+    final byte[] b = new byte[2 * bufferSize];
+    new Random().nextBytes(b);
+    try (FSDataOutputStream stream = fs.create(TEST_PATH)) {
+      stream.write(b);
+    }
+
+    final byte[] readBuffer = new byte[2 * bufferSize];
+    int result;
+    try (FSDataInputStream inputStream = fs.open(TEST_PATH)) {
+      inputStream.seek(bufferSize);
+      result = inputStream.read(readBuffer, bufferSize, bufferSize);
+      assertNotEquals(-1, result);
+      inputStream.seek(0);
+      result = inputStream.read(readBuffer, 0, bufferSize);
+    }
+    assertNotEquals("data read in final read()", -1, result);
+    assertArrayEquals(readBuffer, b);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
index 057dfa0..f1800c0 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
@@ -108,7 +108,7 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
 
     final byte[] b = new byte[1024 * 1000];
     new Random().nextBytes(b);
-    try(final FSDataOutputStream stream = fs.create(TEST_FILE)) {
+    try (FSDataOutputStream stream = fs.create(TEST_FILE)) {
       stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
index 04690de..522b635 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
@@ -91,7 +91,7 @@ public class ITestAzureBlobFileSystemE2EScale extends
     final FileSystem.Statistics abfsStatistics;
     int testBufferSize;
     final byte[] sourceData;
-    try(final FSDataOutputStream stream = fs.create(TEST_FILE)) {
+    try (FSDataOutputStream stream = fs.create(TEST_FILE)) {
       abfsStatistics = fs.getFsStatistics();
       abfsStatistics.reset();
 
@@ -112,7 +112,7 @@ public class ITestAzureBlobFileSystemE2EScale extends
         remoteData.length, abfsStatistics.getBytesRead());
     assertEquals("bytes written in " + stats,
         sourceData.length, abfsStatistics.getBytesWritten());
-    assertEquals("bytesRead from read() call", testBufferSize, bytesRead );
+    assertEquals("bytesRead from read() call", testBufferSize, bytesRead);
     assertArrayEquals("round tripped data", sourceData, remoteData);
 
   }


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


[27/50] [abbrv] hadoop git commit: YARN-8521. NPE in AllocationTagsManager when a container is removed more than once. Contributed by Weiwei Yang.

Posted by tm...@apache.org.
YARN-8521. NPE in AllocationTagsManager when a container is removed more than once. 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/08d50606
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/08d50606
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/08d50606

Branch: refs/heads/HADOOP-15407
Commit: 08d5060605af81a3d6048044176dc656c0dad56c
Parents: f5dbbfe
Author: Weiwei Yang <ww...@apache.org>
Authored: Fri Aug 10 08:32:02 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Fri Aug 10 08:32:02 2018 +0800

----------------------------------------------------------------------
 .../constraint/AllocationTagsManager.java       |  5 ++
 .../constraint/TestAllocationTagsManager.java   | 37 ++++++++++++++
 .../TestPlacementConstraintsUtil.java           | 51 ++++++++++----------
 3 files changed, 68 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d50606/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsManager.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/constraint/AllocationTagsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsManager.java
index a690767..6f160b6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsManager.java
@@ -115,6 +115,11 @@ public class AllocationTagsManager {
 
     private void removeTagFromInnerMap(Map<String, Long> innerMap, String tag) {
       Long count = innerMap.get(tag);
+      if (count == null) {
+        LOG.warn("Trying to remove tags, however the tag " + tag
+            + " no longer exists on this node/rack.");
+        return;
+      }
       if (count > 1) {
         innerMap.put(tag, count - 1);
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d50606/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsManager.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/constraint/TestAllocationTagsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsManager.java
index 3f2aaed..9095ac1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsManager.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
@@ -38,6 +39,7 @@ import org.junit.Test;
 import org.mockito.Mockito;
 
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
@@ -61,6 +63,41 @@ public class TestAllocationTagsManager {
   }
 
   @Test
+  public void testMultipleAddRemoveContainer() {
+    AllocationTagsManager atm = new AllocationTagsManager(rmContext);
+
+    NodeId nodeId = NodeId.fromString("host1:123");
+    ContainerId cid1 = TestUtils.getMockContainerId(1, 1);
+    ContainerId cid2 = TestUtils.getMockContainerId(1, 2);
+    ContainerId cid3 = TestUtils.getMockContainerId(1, 3);
+    Set<String> tags1 = ImmutableSet.of("mapper", "reducer");
+    Set<String> tags2 = ImmutableSet.of("mapper");
+    Set<String> tags3 = ImmutableSet.of("zk");
+
+    // node - mapper : 2
+    //      - reduce : 1
+    atm.addContainer(nodeId, cid1, tags1);
+    atm.addContainer(nodeId, cid2, tags2);
+    atm.addContainer(nodeId, cid3, tags3);
+    Assert.assertEquals(2L,
+        (long) atm.getAllocationTagsWithCount(nodeId).get("mapper"));
+    Assert.assertEquals(1L,
+        (long) atm.getAllocationTagsWithCount(nodeId).get("reducer"));
+
+    // remove container1
+    atm.removeContainer(nodeId, cid1, tags1);
+    Assert.assertEquals(1L,
+        (long) atm.getAllocationTagsWithCount(nodeId).get("mapper"));
+    Assert.assertNull(atm.getAllocationTagsWithCount(nodeId).get("reducer"));
+
+    // remove the same container again, the reducer no longer exists,
+    // make sure there is no NPE here
+    atm.removeContainer(nodeId, cid1, tags1);
+    Assert.assertNull(atm.getAllocationTagsWithCount(nodeId).get("mapper"));
+    Assert.assertNull(atm.getAllocationTagsWithCount(nodeId).get("reducer"));
+  }
+
+  @Test
   public void testAllocationTagsManagerSimpleCases()
       throws InvalidAllocationTagsQueryException {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d50606/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintsUtil.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/constraint/TestPlacementConstraintsUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintsUtil.java
index dc61981..5dbdc8a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintsUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintsUtil.java
@@ -163,6 +163,11 @@ public class TestPlacementConstraintsUtil {
         ApplicationAttemptId.newInstance(appId, 0), 0);
   }
 
+  private ContainerId newContainerId(ApplicationId appId, int containerId) {
+    return ContainerId.newContainerId(
+        ApplicationAttemptId.newInstance(appId, 0), containerId);
+  }
+
   private SchedulerNode newSchedulerNode(String hostname, String rackName,
       NodeId nodeId) {
     SchedulerNode node = mock(SchedulerNode.class);
@@ -271,12 +276,10 @@ public class TestPlacementConstraintsUtil {
     SchedulerNode schedulerNode3 =newSchedulerNode(n3_r2.getHostName(),
         n3_r2.getRackName(), n3_r2.getNodeID());
 
-    ContainerId ca = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(appId1, 0), 0);
+    ContainerId ca = newContainerId(appId1, 0);
     tm.addContainer(n0_r1.getNodeID(), ca, ImmutableSet.of("A"));
 
-    ContainerId cb = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(appId1, 0), 0);
+    ContainerId cb = newContainerId(appId1, 1);
     tm.addContainer(n1_r1.getNodeID(), cb, ImmutableSet.of("B"));
 
     // n0 and n1 has A/B so they cannot satisfy the PC
@@ -297,11 +300,9 @@ public class TestPlacementConstraintsUtil {
      * n2: A(1), B(1)
      * n3:
      */
-    ContainerId ca1 = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(appId1, 0), 0);
+    ContainerId ca1 = newContainerId(appId1, 2);
     tm.addContainer(n2_r2.getNodeID(), ca1, ImmutableSet.of("A"));
-    ContainerId cb1 = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(appId1, 0), 0);
+    ContainerId cb1 = newContainerId(appId1, 3);
     tm.addContainer(n2_r2.getNodeID(), cb1, ImmutableSet.of("B"));
 
     // Only n2 has both A and B so only it can satisfy the PC
@@ -468,9 +469,9 @@ public class TestPlacementConstraintsUtil {
      *  n3: ""
      */
     tm.addContainer(n0r1.getNodeID(),
-        newContainerId(appId1), ImmutableSet.of("hbase-m"));
+        newContainerId(appId1, 1), ImmutableSet.of("hbase-m"));
     tm.addContainer(n2r2.getNodeID(),
-        newContainerId(appId1), ImmutableSet.of("hbase-rs"));
+        newContainerId(appId1, 2), ImmutableSet.of("hbase-rs"));
     Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID())
         .get("hbase-m").longValue());
     Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
@@ -504,7 +505,7 @@ public class TestPlacementConstraintsUtil {
      *  n3: hbase-rs(1)
      */
     tm.addContainer(n3r2.getNodeID(),
-        newContainerId(appId1), ImmutableSet.of("hbase-rs"));
+        newContainerId(appId1, 2), ImmutableSet.of("hbase-rs"));
     // n3 is qualified now because it is allocated with hbase-rs tag
     Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
@@ -518,7 +519,7 @@ public class TestPlacementConstraintsUtil {
      */
     // Place
     tm.addContainer(n2r2.getNodeID(),
-        newContainerId(appId1), ImmutableSet.of("spark"));
+        newContainerId(appId1, 3), ImmutableSet.of("spark"));
     // According to constraint, "zk" is allowed to be placed on a node
     // has "hbase-m" tag OR a node has both "hbase-rs" and "spark" tags.
     Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
@@ -552,9 +553,9 @@ public class TestPlacementConstraintsUtil {
      *  n3: ""
      */
     tm.addContainer(n0r1.getNodeID(),
-        newContainerId(appId1), ImmutableSet.of("hbase-m"));
+        newContainerId(appId1, 0), ImmutableSet.of("hbase-m"));
     tm.addContainer(n2r2.getNodeID(),
-        newContainerId(appId1), ImmutableSet.of("hbase-m"));
+        newContainerId(appId1, 1), ImmutableSet.of("hbase-m"));
     Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID())
         .get("hbase-m").longValue());
     Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
@@ -589,7 +590,7 @@ public class TestPlacementConstraintsUtil {
      */
     for (int i=0; i<4; i++) {
       tm.addContainer(n1r1.getNodeID(),
-          newContainerId(appId1), ImmutableSet.of("spark"));
+          newContainerId(appId1, i+2), ImmutableSet.of("spark"));
     }
     Assert.assertEquals(4L, tm.getAllocationTagsWithCount(n1r1.getNodeID())
         .get("spark").longValue());
@@ -633,19 +634,19 @@ public class TestPlacementConstraintsUtil {
      *  n3: ""
      */
     tm.addContainer(n0r1.getNodeID(),
-        newContainerId(application1), ImmutableSet.of("A"));
+        newContainerId(application1, 0), ImmutableSet.of("A"));
     tm.addContainer(n0r1.getNodeID(),
-        newContainerId(application2), ImmutableSet.of("A"));
+        newContainerId(application2, 1), ImmutableSet.of("A"));
     tm.addContainer(n1r1.getNodeID(),
-        newContainerId(application3), ImmutableSet.of("A"));
+        newContainerId(application3, 2), ImmutableSet.of("A"));
     tm.addContainer(n1r1.getNodeID(),
-        newContainerId(application3), ImmutableSet.of("A"));
+        newContainerId(application3, 3), ImmutableSet.of("A"));
     tm.addContainer(n1r1.getNodeID(),
-        newContainerId(application3), ImmutableSet.of("A"));
+        newContainerId(application3, 4), ImmutableSet.of("A"));
     tm.addContainer(n2r2.getNodeID(),
-        newContainerId(application1), ImmutableSet.of("A"));
+        newContainerId(application1, 5), ImmutableSet.of("A"));
     tm.addContainer(n2r2.getNodeID(),
-        newContainerId(application1), ImmutableSet.of("A"));
+        newContainerId(application1, 6), ImmutableSet.of("A"));
 
     SchedulerNode schedulerNode0 = newSchedulerNode(n0r1.getHostName(),
         n0r1.getRackName(), n0r1.getNodeID());
@@ -888,9 +889,9 @@ public class TestPlacementConstraintsUtil {
      *  n3: ""
      */
     tm.addContainer(n0r1.getNodeID(),
-        newContainerId(application1), ImmutableSet.of("hbase-m"));
+        newContainerId(application1, 0), ImmutableSet.of("hbase-m"));
     tm.addContainer(n2r2.getNodeID(),
-        newContainerId(application1), ImmutableSet.of("hbase-m"));
+        newContainerId(application1, 1), ImmutableSet.of("hbase-m"));
     Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID())
         .get("hbase-m").longValue());
     Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
@@ -958,7 +959,7 @@ public class TestPlacementConstraintsUtil {
      *  n3: ""
      */
     tm.addContainer(n0r1.getNodeID(),
-        newContainerId(application3), ImmutableSet.of("hbase-m"));
+        newContainerId(application3, 0), ImmutableSet.of("hbase-m"));
 
     // Anti-affinity to self/hbase-m
     Assert.assertFalse(PlacementConstraintsUtil


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


[38/50] [abbrv] hadoop git commit: HADOOP-15659. Code changes for bug fix and new tests. Contributed by Da Zhou.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java
new file mode 100644
index 0000000..e4acbae
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java
@@ -0,0 +1,60 @@
+/**
+ * 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.azurebfs;
+
+import java.lang.ref.WeakReference;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Test finalize() method when "fs.abfs.impl.disable.cache" is enabled.
+ */
+public class ITestAzureBlobFileSystemFinalize extends AbstractAbfsScaleTest{
+  static final String DISABLE_CACHE_KEY = "fs.abfs.impl.disable.cache";
+
+  public ITestAzureBlobFileSystemFinalize() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testFinalize() throws Exception {
+    // Disable the cache for filesystem to make sure there is no reference.
+    Configuration configuration = this.getConfiguration();
+    configuration.setBoolean(this.DISABLE_CACHE_KEY, true);
+
+    AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration);
+
+    WeakReference<Object> ref = new WeakReference<Object>(fs);
+    fs = null;
+
+    int i = 0;
+    int maxTries = 1000;
+    while (ref.get() != null && i < maxTries) {
+      System.gc();
+      System.runFinalization();
+      i++;
+    }
+
+    Assert.assertTrue("testFinalizer didn't get cleaned up within maxTries", ref.get() == null);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
index d90f018..2f40b64 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
@@ -20,12 +20,20 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.EnumSet;
 import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.io.IOException;
 
+import com.microsoft.azure.storage.blob.BlockEntry;
+import com.microsoft.azure.storage.blob.BlockListingFilter;
+import com.microsoft.azure.storage.blob.CloudBlockBlob;
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+import org.hamcrest.core.IsEqual;
+import org.hamcrest.core.IsNot;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -46,6 +54,8 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
   private static final int THREAD_SLEEP_TIME = 6000;
 
   private static final Path TEST_FILE_PATH = new Path("/testfile");
+  private static final int TEST_FILE_LENGTH = 1024 * 1024 * 8;
+  private static final int WAITING_TIME = 4000;
 
   public ITestAzureBlobFileSystemFlush() {
     super();
@@ -55,7 +65,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
   public void testAbfsOutputStreamAsyncFlushWithRetainUncommittedData() throws Exception {
     final AzureBlobFileSystem fs = getFileSystem();
     final byte[] b;
-    try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
+    try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
       b = new byte[TEST_BUFFER_SIZE];
       new Random().nextBytes(b);
 
@@ -70,7 +80,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
     }
 
     final byte[] r = new byte[TEST_BUFFER_SIZE];
-    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) {
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) {
       while (inputStream.available() != 0) {
         int result = inputStream.read(r);
 
@@ -84,7 +94,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
   public void testAbfsOutputStreamSyncFlush() throws Exception {
     final AzureBlobFileSystem fs = getFileSystem();
     final byte[] b;
-    try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
+    try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
       b = new byte[TEST_BUFFER_SIZE];
       new Random().nextBytes(b);
       stream.write(b);
@@ -97,7 +107,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
     }
 
     final byte[] r = new byte[TEST_BUFFER_SIZE];
-    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) {
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) {
       int result = inputStream.read(r);
 
       assertNotEquals(-1, result);
@@ -111,7 +121,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
     final AzureBlobFileSystem fs = getFileSystem();
     final FileSystem.Statistics abfsStatistics;
     ExecutorService es;
-    try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
+    try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
       abfsStatistics = fs.getFsStatistics();
       abfsStatistics.reset();
 
@@ -160,7 +170,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
   public void testWriteHeavyBytesToFileAsyncFlush() throws Exception {
     final AzureBlobFileSystem fs = getFileSystem();
     ExecutorService es = Executors.newFixedThreadPool(10);
-    try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
+    try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
 
       final byte[] b = new byte[TEST_BUFFER_SIZE];
       new Random().nextBytes(b);
@@ -196,4 +206,118 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
     FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
     assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen());
   }
+
+  @Test
+  public void testFlushWithFlushEnabled() throws Exception {
+    AzureBlobStorageTestAccount testAccount = createWasbTestAccount();
+    String wasbUrl = testAccount.getFileSystem().getName();
+    String abfsUrl = wasbUrlToAbfsUrl(wasbUrl);
+    final AzureBlobFileSystem fs = this.getFileSystem(abfsUrl);
+    byte[] buffer = getRandomBytesArray();
+    CloudBlockBlob blob = testAccount.getBlobReference(TEST_FILE_PATH.toString().substring(1));
+    try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) {
+      // Wait for write request to be executed
+      Thread.sleep(WAITING_TIME);
+      stream.flush();
+      ArrayList<BlockEntry> blockList = blob.downloadBlockList(
+              BlockListingFilter.COMMITTED, null, null, null);
+      // verify block has been committed
+      assertEquals(1, blockList.size());
+    }
+  }
+
+  @Test
+  public void testFlushWithFlushDisabled() throws Exception {
+    AzureBlobStorageTestAccount testAccount = createWasbTestAccount();
+    String wasbUrl = testAccount.getFileSystem().getName();
+    String abfsUrl = wasbUrlToAbfsUrl(wasbUrl);
+    final AzureBlobFileSystem fs = this.getFileSystem(abfsUrl);
+    byte[] buffer = getRandomBytesArray();
+    CloudBlockBlob blob = testAccount.getBlobReference(TEST_FILE_PATH.toString().substring(1));
+    try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) {
+      // Wait for write request to be executed
+      Thread.sleep(WAITING_TIME);
+      stream.flush();
+      ArrayList<BlockEntry> blockList = blob.downloadBlockList(
+              BlockListingFilter.COMMITTED, null, null, null);
+      // verify block has not been committed
+      assertEquals(0, blockList.size());
+    }
+  }
+
+  @Test
+  public void testHflushWithFlushEnabled() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    byte[] buffer = getRandomBytesArray();
+    try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) {
+      stream.hflush();
+      validate(fs, TEST_FILE_PATH, buffer, true);
+    }
+  }
+
+  @Test
+  public void testHflushWithFlushDisabled() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    byte[] buffer = getRandomBytesArray();
+    try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) {
+      stream.hflush();
+      validate(fs, TEST_FILE_PATH, buffer, false);
+    }
+  }
+
+  @Test
+  public void testHsyncWithFlushEnabled() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    byte[] buffer = getRandomBytesArray();
+    try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) {
+      stream.hsync();
+      validate(fs, TEST_FILE_PATH, buffer, true);
+    }
+  }
+
+  @Test
+  public void testHsyncWithFlushDisabled() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    byte[] buffer = getRandomBytesArray();
+    try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) {
+      stream.hsync();
+      validate(fs, TEST_FILE_PATH, buffer, false);
+    }
+  }
+
+  private byte[] getRandomBytesArray() {
+    final byte[] b = new byte[TEST_FILE_LENGTH];
+    new Random().nextBytes(b);
+    return b;
+  }
+
+  private FSDataOutputStream getStreamAfterWrite(AzureBlobFileSystem fs, Path path, byte[] buffer, boolean enableFlush) throws IOException {
+    fs.getAbfsStore().getAbfsConfiguration().setEnableFlush(enableFlush);
+    FSDataOutputStream stream = fs.create(path);
+    stream.write(buffer);
+    return stream;
+  }
+
+  private AzureBlobStorageTestAccount createWasbTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create("", EnumSet.of(AzureBlobStorageTestAccount.CreateOptions.CreateContainer),
+            this.getConfiguration());
+  }
+
+  private void validate(FileSystem fs, Path path, byte[] writeBuffer, boolean isEqual) throws IOException {
+    String filePath = path.toUri().toString();
+    try (FSDataInputStream inputStream = fs.open(path)) {
+      byte[] readBuffer = new byte[TEST_FILE_LENGTH];
+      int numBytesRead = inputStream.read(readBuffer, 0, readBuffer.length);
+      if (isEqual) {
+        assertArrayEquals(
+                String.format("Bytes read do not match bytes written to %1$s", filePath), writeBuffer, readBuffer);
+      } else {
+        assertThat(
+                String.format("Bytes read unexpectedly match bytes written to %1$s",
+                        filePath),
+                readBuffer,
+                IsNot.not(IsEqual.equalTo(writeBuffer)));
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
index 5a6e46d..874a8a3 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
@@ -20,10 +20,10 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.io.FileNotFoundException;
 
-import org.junit.Test;
-
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
index 1a0edaf..07426c4 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
@@ -30,7 +30,6 @@ import org.junit.Test;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
-import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
@@ -86,7 +85,7 @@ public class ITestAzureBlobFileSystemRename extends
 
     assertRenameOutcome(fs, test1,
         new Path("testDir/test10"), true);
-    assertPathDoesNotExist(fs, "rename source dir", test1 );
+    assertPathDoesNotExist(fs, "rename source dir", test1);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
index 1c71125..7a7e327 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.util.Hashtable;
 
-import org.junit.Ignore;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -28,8 +27,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
-import static org.junit.Assert.assertEquals;
-
 /**
  * Test FileSystemProperties.
  */
@@ -62,7 +59,6 @@ public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest {
   }
 
   @Test
-  @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant")
   public void testBase64FileSystemProperties() throws Exception {
     final AzureBlobFileSystem fs = getFileSystem();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java
new file mode 100644
index 0000000..fb667dd
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java
@@ -0,0 +1,149 @@
+/**
+ * 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.azurebfs;
+
+import java.lang.reflect.Field;
+
+import org.apache.commons.codec.Charsets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT;
+
+import org.apache.commons.codec.binary.Base64;
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+/**
+ * Test ConfigurationServiceFieldsValidation.
+ */
+public class TestAbfsConfigurationFieldsValidation {
+  private AbfsConfiguration abfsConfiguration;
+
+  private static final String INT_KEY= "intKey";
+  private static final String LONG_KEY= "longKey";
+  private static final String STRING_KEY= "stringKey";
+  private static final String BASE64_KEY= "base64Key";
+  private static final String BOOLEAN_KEY= "booleanKey";
+  private static final int DEFAULT_INT = 4194304;
+  private static final int DEFAULT_LONG = 4194304;
+
+  private static final int TEST_INT = 1234565;
+  private static final int TEST_LONG = 4194304;
+
+  private final String encodedString;
+  private final String encodedAccountKey;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = INT_KEY,
+      MinValue = Integer.MIN_VALUE,
+      MaxValue = Integer.MAX_VALUE,
+      DefaultValue = DEFAULT_INT)
+  private int intField;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey = LONG_KEY,
+      MinValue = Long.MIN_VALUE,
+      MaxValue = Long.MAX_VALUE,
+      DefaultValue = DEFAULT_LONG)
+  private int longField;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = STRING_KEY,
+  DefaultValue = "default")
+  private String stringField;
+
+  @Base64StringConfigurationValidatorAnnotation(ConfigurationKey = BASE64_KEY,
+  DefaultValue = "base64")
+  private String base64Field;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = BOOLEAN_KEY,
+  DefaultValue = false)
+  private boolean boolField;
+
+  public TestAbfsConfigurationFieldsValidation() throws Exception {
+    super();
+    Base64 base64 = new Base64();
+    this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
+    this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
+    Configuration configuration = new Configuration();
+    configuration.addResource("azure-bfs-test.xml");
+    configuration.set(INT_KEY, "1234565");
+    configuration.set(LONG_KEY, "4194304");
+    configuration.set(STRING_KEY, "stringValue");
+    configuration.set(BASE64_KEY, encodedString);
+    configuration.set(BOOLEAN_KEY, "true");
+    configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "testaccount1.blob.core.windows.net", this.encodedAccountKey);
+    abfsConfiguration = new AbfsConfiguration(configuration);
+  }
+
+  @Test
+  public void testValidateFunctionsInConfigServiceImpl() throws Exception {
+    Field[] fields = this.getClass().getDeclaredFields();
+    for (Field field : fields) {
+      field.setAccessible(true);
+      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
+        assertEquals(TEST_INT, abfsConfiguration.validateInt(field));
+      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
+        assertEquals(DEFAULT_LONG, abfsConfiguration.validateLong(field));
+      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
+        assertEquals("stringValue", abfsConfiguration.validateString(field));
+      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
+        assertEquals(this.encodedString, abfsConfiguration.validateBase64String(field));
+      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
+        assertEquals(true, abfsConfiguration.validateBoolean(field));
+      }
+    }
+  }
+
+  @Test
+  public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception {
+    // test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor
+    assertEquals(DEFAULT_WRITE_BUFFER_SIZE, abfsConfiguration.getWriteBufferSize());
+    assertEquals(DEFAULT_READ_BUFFER_SIZE, abfsConfiguration.getReadBufferSize());
+    assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, abfsConfiguration.getMinBackoffIntervalMilliseconds());
+    assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, abfsConfiguration.getMaxBackoffIntervalMilliseconds());
+    assertEquals(DEFAULT_BACKOFF_INTERVAL, abfsConfiguration.getBackoffIntervalMilliseconds());
+    assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, abfsConfiguration.getMaxIoRetries());
+    assertEquals(MAX_AZURE_BLOCK_SIZE, abfsConfiguration.getAzureBlockSize());
+    assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, abfsConfiguration.getAzureBlockLocationHost());
+  }
+
+  @Test
+  public void testGetAccountKey() throws Exception {
+    String accountKey = abfsConfiguration.getStorageAccountKey("testaccount1.blob.core.windows.net");
+    assertEquals(this.encodedAccountKey, accountKey);
+  }
+
+  @Test (expected = ConfigurationPropertyNotFoundException.class)
+  public void testGetAccountKeyWithNonExistingAccountName() throws Exception {
+    abfsConfiguration.getStorageAccountKey("bogusAccountName");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
index d365e6e..c0c5f91 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
@@ -43,9 +43,8 @@ public class AbfsFileSystemContract extends AbstractBondedFSContract {
 
   @Override
   public String getScheme() {
-    return isSecure ?
-        FileSystemUriSchemes.ABFS_SECURE_SCHEME
-        : FileSystemUriSchemes.ABFS_SCHEME;
+    return isSecure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME
+            : FileSystemUriSchemes.ABFS_SCHEME;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java
deleted file mode 100644
index dd06fe3..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java
+++ /dev/null
@@ -1,91 +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.fs.azurebfs.services;
-
-import java.util.Arrays;
-import java.util.Random;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.AbstractAbfsScaleTest;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
-
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
-
-/**
- * Test read, write and seek.
- * Uses package-private methods in AbfsConfiguration, which is why it is in
- * this package.
- */
-@RunWith(Parameterized.class)
-public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
-  private static final Path TEST_PATH = new Path("/testfile");
-
-  @Parameterized.Parameters(name = "Size={0}")
-  public static Iterable<Object[]> sizes() {
-    return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE},
-        {DEFAULT_READ_BUFFER_SIZE},
-        {MAX_BUFFER_SIZE}});
-  }
-
-  private final int size;
-
-  public ITestAbfsReadWriteAndSeek(final int size) {
-    this.size = size;
-  }
-
-  @Test
-  public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception {
-    testReadWriteAndSeek(size);
-  }
-
-  private void testReadWriteAndSeek(int bufferSize) throws Exception {
-    final AzureBlobFileSystem fs = getFileSystem();
-    final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(getConfiguration());
-
-    abfsConfiguration.setWriteBufferSize(bufferSize);
-    abfsConfiguration.setReadBufferSize(bufferSize);
-
-
-    final byte[] b = new byte[2 * bufferSize];
-    new Random().nextBytes(b);
-    try(final FSDataOutputStream stream = fs.create(TEST_PATH)) {
-      stream.write(b);
-    }
-
-    final byte[] readBuffer = new byte[2 * bufferSize];
-    int result;
-    try(final FSDataInputStream inputStream = fs.open(TEST_PATH)) {
-      inputStream.seek(bufferSize);
-      result = inputStream.read(readBuffer, bufferSize, bufferSize);
-      assertNotEquals(-1, result);
-      inputStream.seek(0);
-      result = inputStream.read(readBuffer, 0, bufferSize);
-    }
-    assertNotEquals("data read in final read()", -1, result);
-    assertArrayEquals(readBuffer, b);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
new file mode 100644
index 0000000..0b335a5
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
@@ -0,0 +1,60 @@
+/**
+ * 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.azurebfs.services;
+
+import java.net.URL;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+
+/**
+ * Test useragent of abfs client.
+ *
+ */
+public final class TestAbfsClient {
+
+  @Test
+  public void verifyUnknownUserAgent() throws Exception {
+    String expectedUserAgentPattern = "Azure Blob FS\\/1.0 \\(JavaJRE ([^\\)]+)\\)";
+    final Configuration configuration = new Configuration();
+    configuration.unset(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY);
+    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration);
+    AbfsClient abfsClient = new AbfsClient(new URL("http://azure.com"), null, abfsConfiguration, null);
+    String userAgent = abfsClient.initializeUserAgent(abfsConfiguration);
+    Pattern pattern = Pattern.compile(expectedUserAgentPattern);
+    Assert.assertTrue(pattern.matcher(userAgent).matches());
+  }
+
+  @Test
+  public void verifyUserAgent() throws Exception {
+    String expectedUserAgentPattern = "Azure Blob FS\\/1.0 \\(JavaJRE ([^\\)]+)\\) Partner Service";
+    final Configuration configuration = new Configuration();
+    configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service");
+    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration);
+    AbfsClient abfsClient = new AbfsClient(new URL("http://azure.com"), null, abfsConfiguration, null);
+    String userAgent = abfsClient.initializeUserAgent(abfsConfiguration);
+    Pattern pattern = Pattern.compile(expectedUserAgentPattern);
+    Assert.assertTrue(pattern.matcher(userAgent).matches());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
deleted file mode 100644
index ebaafa4..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
+++ /dev/null
@@ -1,147 +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.fs.azurebfs.services;
-
-import java.lang.reflect.Field;
-
-import org.apache.commons.codec.Charsets;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
-
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT;
-
-import org.apache.commons.codec.binary.Base64;
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-/**
- * Test ConfigurationServiceFieldsValidation.
- */
-public class TestAbfsConfigurationFieldsValidation {
-  private final AbfsConfiguration abfsConfiguration;
-
-  private static final String INT_KEY= "intKey";
-  private static final String LONG_KEY= "longKey";
-  private static final String STRING_KEY= "stringKey";
-  private static final String BASE64_KEY= "base64Key";
-  private static final String BOOLEAN_KEY= "booleanKey";
-  private static final int DEFAULT_INT = 4194304;
-  private static final int DEFAULT_LONG = 4194304;
-
-  private static final int TEST_INT = 1234565;
-  private static final int TEST_LONG = 4194304;
-
-  private final String encodedString;
-  private final String encodedAccountKey;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = INT_KEY,
-      MinValue = Integer.MIN_VALUE,
-      MaxValue = Integer.MAX_VALUE,
-      DefaultValue = DEFAULT_INT)
-  private int intField;
-
-  @LongConfigurationValidatorAnnotation(ConfigurationKey = LONG_KEY,
-      MinValue = Long.MIN_VALUE,
-      MaxValue = Long.MAX_VALUE,
-      DefaultValue = DEFAULT_LONG)
-  private int longField;
-
-  @StringConfigurationValidatorAnnotation(ConfigurationKey = STRING_KEY,
-  DefaultValue = "default")
-  private String stringField;
-
-  @Base64StringConfigurationValidatorAnnotation(ConfigurationKey = BASE64_KEY,
-  DefaultValue = "base64")
-  private String base64Field;
-
-  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = BOOLEAN_KEY,
-  DefaultValue = false)
-  private boolean boolField;
-
-  public TestAbfsConfigurationFieldsValidation() throws Exception {
-    Base64 base64 = new Base64();
-    this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
-    this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
-    Configuration configuration = new Configuration(false);
-    configuration.set(INT_KEY, "1234565");
-    configuration.set(LONG_KEY, "4194304");
-    configuration.set(STRING_KEY, "stringValue");
-    configuration.set(BASE64_KEY, encodedString);
-    configuration.set(BOOLEAN_KEY, "true");
-    configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "testaccount1.blob.core.windows.net", this.encodedAccountKey);
-    abfsConfiguration = new AbfsConfiguration(configuration);
-  }
-
-  @Test
-  public void testValidateFunctionsInConfigServiceImpl() throws Exception {
-    Field[] fields = this.getClass().getDeclaredFields();
-    for (Field field : fields) {
-      field.setAccessible(true);
-      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
-        assertEquals(TEST_INT, abfsConfiguration.validateInt(field));
-      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
-        assertEquals(DEFAULT_LONG, abfsConfiguration.validateLong(field));
-      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
-        assertEquals("stringValue", abfsConfiguration.validateString(field));
-      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
-        assertEquals(this.encodedString, abfsConfiguration.validateBase64String(field));
-      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
-        assertEquals(true, abfsConfiguration.validateBoolean(field));
-      }
-    }
-  }
-
-  @Test
-  public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception {
-    // test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor
-    assertEquals(DEFAULT_WRITE_BUFFER_SIZE, abfsConfiguration.getWriteBufferSize());
-    assertEquals(DEFAULT_READ_BUFFER_SIZE, abfsConfiguration.getReadBufferSize());
-    assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, abfsConfiguration.getMinBackoffIntervalMilliseconds());
-    assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, abfsConfiguration.getMaxBackoffIntervalMilliseconds());
-    assertEquals(DEFAULT_BACKOFF_INTERVAL, abfsConfiguration.getBackoffIntervalMilliseconds());
-    assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, abfsConfiguration.getMaxIoRetries());
-    assertEquals(MAX_AZURE_BLOCK_SIZE, abfsConfiguration.getAzureBlockSize());
-    assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, abfsConfiguration.getAzureBlockLocationHost());
-  }
-
-  @Test
-  public void testGetAccountKey() throws Exception {
-    String accountKey = abfsConfiguration.getStorageAccountKey("testaccount1.blob.core.windows.net");
-    assertEquals(this.encodedAccountKey, accountKey);
-  }
-
-  @Test (expected = ConfigurationPropertyNotFoundException.class)
-  public void testGetAccountKeyWithNonExistingAccountName() throws Exception {
-    abfsConfiguration.getStorageAccountKey("bogusAccountName");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6866de/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java
new file mode 100644
index 0000000..d17e767
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java
@@ -0,0 +1,89 @@
+/**
+ * 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.azurebfs.services;
+
+import java.io.File;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException;
+import org.apache.hadoop.util.Shell;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test ShellDecryptionKeyProvider.
+ *
+ */
+public class TestShellDecryptionKeyProvider {
+  public static final Log LOG = LogFactory
+      .getLog(TestShellDecryptionKeyProvider.class);
+  private static final File TEST_ROOT_DIR = new File(System.getProperty(
+      "test.build.data", "/tmp"), "TestShellDecryptionKeyProvider");
+
+  @Test
+  public void testScriptPathNotSpecified() throws Exception {
+    if (!Shell.WINDOWS) {
+      return;
+    }
+    ShellDecryptionKeyProvider provider = new ShellDecryptionKeyProvider();
+    Configuration conf = new Configuration();
+    String account = "testacct";
+    String key = "key";
+
+    conf.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + account, key);
+    try {
+      provider.getStorageAccountKey(account, conf);
+      Assert
+          .fail("fs.azure.shellkeyprovider.script is not specified, we should throw");
+    } catch (KeyProviderException e) {
+      LOG.info("Received an expected exception: " + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testValidScript() throws Exception {
+    if (!Shell.WINDOWS) {
+      return;
+    }
+    String expectedResult = "decretedKey";
+
+    // Create a simple script which echoes the given key plus the given
+    // expected result (so that we validate both script input and output)
+    File scriptFile = new File(TEST_ROOT_DIR, "testScript.cmd");
+    FileUtils.writeStringToFile(scriptFile, "@echo %1 " + expectedResult);
+
+    ShellDecryptionKeyProvider provider = new ShellDecryptionKeyProvider();
+    Configuration conf = new Configuration();
+    String account = "testacct";
+    String key = "key1";
+    conf.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME  + account, key);
+    conf.set(ConfigurationKeys.AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT,
+        "cmd /c " + scriptFile.getAbsolutePath());
+
+    String result = provider.getStorageAccountKey(account, conf);
+    assertEquals(key + " " + expectedResult, result);
+  }
+}


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


[29/50] [abbrv] hadoop git commit: HDFS-13795. Fix potential NPE in InMemoryLevelDBAliasMapServer.

Posted by tm...@apache.org.
HDFS-13795. Fix potential NPE in InMemoryLevelDBAliasMapServer.


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

Branch: refs/heads/HADOOP-15407
Commit: 15241c6349a5245761ed43bd0d38b25f783cc96b
Parents: 0a71bf1
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Aug 10 09:38:40 2018 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Aug 10 09:38:40 2018 -0700

----------------------------------------------------------------------
 ...yAliasMapProtocolClientSideTranslatorPB.java |  6 +++
 .../aliasmap/InMemoryLevelDBAliasMapServer.java |  8 +++-
 .../impl/TestInMemoryLevelDBAliasMapClient.java | 39 ++++++++++++++++++++
 3 files changed, 51 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/15241c63/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
index 2025c16..d9e984b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
@@ -167,6 +167,9 @@ public class InMemoryAliasMapProtocolClientSideTranslatorPB
   public Optional<ProvidedStorageLocation> read(@Nonnull Block block)
       throws IOException {
 
+    if (block == null) {
+      throw new IOException("Block cannot be null");
+    }
     ReadRequestProto request =
         ReadRequestProto
             .newBuilder()
@@ -191,6 +194,9 @@ public class InMemoryAliasMapProtocolClientSideTranslatorPB
   public void write(@Nonnull Block block,
       @Nonnull ProvidedStorageLocation providedStorageLocation)
       throws IOException {
+    if (block == null || providedStorageLocation == null) {
+      throw new IOException("Provided block and location cannot be null");
+    }
     WriteRequestProto request =
         WriteRequestProto
             .newBuilder()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15241c63/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
index f201bfd..5c56736 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
@@ -150,11 +150,15 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
   public void close() {
     LOG.info("Stopping InMemoryLevelDBAliasMapServer");
     try {
-      aliasMap.close();
+      if (aliasMap != null) {
+        aliasMap.close();
+      }
     } catch (IOException e) {
       LOG.error(e.getMessage());
     }
-    aliasMapServer.stop();
+    if (aliasMapServer != null) {
+      aliasMapServer.stop();
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15241c63/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
index f062633..fccb6f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
@@ -28,14 +28,19 @@ import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
 import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
@@ -59,6 +64,9 @@ public class TestInMemoryLevelDBAliasMapClient {
   private Configuration conf;
   private final static String BPID = "BPID-0";
 
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
   @Before
   public void setUp() throws IOException {
     conf = new Configuration();
@@ -348,4 +356,35 @@ public class TestInMemoryLevelDBAliasMapClient {
     conf.set(DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY, "0.0.0.0");
     writeRead();
   }
+
+  @Test
+  public void testNonExistentFile() throws Exception {
+    // delete alias map location
+    FileUtils.deleteDirectory(tempDir);
+    // expect a RuntimeException when the aliasmap is started.
+    exception.expect(RuntimeException.class);
+    levelDBAliasMapServer.setConf(conf);
+  }
+
+  @Test
+  public void testNonExistentBlock() throws Exception {
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    Block block1 = new Block(100, 43, 44);
+    ProvidedStorageLocation providedStorageLocation1 = null;
+    BlockAliasMap.Writer<FileRegion> writer1 =
+        inMemoryLevelDBAliasMapClient.getWriter(null, BPID);
+    try {
+      writer1.store(new FileRegion(block1, providedStorageLocation1));
+      fail("Should fail on writing a region with null ProvidedLocation");
+    } catch (IOException | IllegalArgumentException e) {
+      assertTrue(e.getMessage().contains("not be null"));
+    }
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null, BPID);
+    LambdaTestUtils.assertOptionalUnset("Expected empty BlockAlias",
+        reader.resolve(block1));
+  }
 }
\ 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


[48/50] [abbrv] hadoop git commit: HADOOP-15560. ABFS: removed dependency injection and unnecessary dependencies. Contributed by Da Zhou.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java
deleted file mode 100644
index 9e4c27b..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java
+++ /dev/null
@@ -1,116 +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.fs.azurebfs.services;
-
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.net.URL;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
-import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
-import org.apache.http.client.utils.URIBuilder;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
-
-@Singleton
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-class AbfsHttpClientFactoryImpl implements AbfsHttpClientFactory {
-  private final ConfigurationService configurationService;
-
-  @Inject
-  AbfsHttpClientFactoryImpl(
-      final ConfigurationService configurationService) {
-
-    Preconditions.checkNotNull(configurationService, "configurationService");
-
-    this.configurationService = configurationService;
-  }
-
-  @VisibleForTesting
-  URIBuilder getURIBuilder(final String hostName, final FileSystem fs) {
-    final AzureBlobFileSystem abfs = (AzureBlobFileSystem) fs;
-
-    String scheme = FileSystemUriSchemes.HTTP_SCHEME;
-
-    if (abfs.isSecure()) {
-      scheme = FileSystemUriSchemes.HTTPS_SCHEME;
-    }
-
-    final URIBuilder uriBuilder = new URIBuilder();
-    uriBuilder.setScheme(scheme);
-    uriBuilder.setHost(hostName);
-
-    return uriBuilder;
-  }
-
-  public AbfsClient create(final AzureBlobFileSystem fs) throws AzureBlobFileSystemException {
-    final URI uri = fs.getUri();
-    final String authority = uri.getRawAuthority();
-    if (null == authority) {
-      throw new InvalidUriAuthorityException(uri.toString());
-    }
-
-    if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) {
-      throw new InvalidUriAuthorityException(uri.toString());
-    }
-
-    final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
-
-    if (authorityParts.length < 2 || "".equals(authorityParts[0])) {
-      final String errMsg = String
-          .format("URI '%s' has a malformed authority, expected container name. "
-                  + "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
-              uri.toString());
-      throw new InvalidUriException(errMsg);
-    }
-
-    final String fileSystemName = authorityParts[0];
-    final String accountName = authorityParts[1];
-
-    final URIBuilder uriBuilder = getURIBuilder(accountName, fs);
-
-    final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName;
-
-    URL baseUrl;
-    try {
-      baseUrl = new URL(url);
-    } catch (MalformedURLException e) {
-      throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString()));
-    }
-
-    SharedKeyCredentials creds =
-        new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)),
-                this.configurationService.getStorageAccountKey(accountName));
-
-    return new AbfsClient(baseUrl, creds, configurationService, new ExponentialRetryPolicy());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java
deleted file mode 100644
index 06e1a8a..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java
+++ /dev/null
@@ -1,693 +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.fs.azurebfs.services;
-
-import javax.xml.bind.DatatypeConverter;
-import java.io.File;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.ByteBuffer;
-import java.nio.CharBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.nio.charset.Charset;
-import java.nio.charset.CharsetDecoder;
-import java.nio.charset.CharsetEncoder;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.Hashtable;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import com.google.common.base.Preconditions;
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
-import org.joda.time.DateTime;
-import org.joda.time.format.DateTimeFormat;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-
-import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
-import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
-import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.hadoop.util.Time.now;
-
-@Singleton
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-final class AbfsHttpServiceImpl implements AbfsHttpService {
-  public static final Logger LOG = LoggerFactory.getLogger(AbfsHttpService.class);
-  private static final String DATE_TIME_PATTERN = "E, dd MMM yyyy HH:mm:ss 'GMT'";
-  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
-  private static final int LIST_MAX_RESULTS = 5000;
-  private static final int DELETE_DIRECTORY_TIMEOUT_MILISECONDS = 180000;
-  private static final int RENAME_TIMEOUT_MILISECONDS = 180000;
-
-  private final AbfsHttpClientFactory abfsHttpClientFactory;
-  private final ConcurrentHashMap<AzureBlobFileSystem, AbfsClient> clientCache;
-  private final ConfigurationService configurationService;
-  private final Set<String> azureAtomicRenameDirSet;
-
-  @Inject
-  AbfsHttpServiceImpl(
-      final ConfigurationService configurationService,
-      final AbfsHttpClientFactory abfsHttpClientFactory,
-      final TracingService tracingService) {
-    Preconditions.checkNotNull(abfsHttpClientFactory, "abfsHttpClientFactory");
-    Preconditions.checkNotNull(configurationService, "configurationService");
-    Preconditions.checkNotNull(tracingService, "tracingService");
-
-    this.configurationService = configurationService;
-    this.clientCache = new ConcurrentHashMap<>();
-    this.abfsHttpClientFactory = abfsHttpClientFactory;
-    this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(configurationService.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
-  }
-
-  @Override
-  public Hashtable<String, String> getFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem)
-      throws AzureBlobFileSystemException{
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "getFilesystemProperties for filesystem: {}",
-        client.getFileSystem());
-
-    final Hashtable<String, String> parsedXmsProperties;
-
-    final AbfsRestOperation op = client.getFilesystemProperties();
-    final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
-
-    parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
-
-    return parsedXmsProperties;
-  }
-
-  @Override
-  public void setFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem, final Hashtable<String, String> properties) throws
-      AzureBlobFileSystemException {
-    if (properties == null || properties.size() == 0) {
-      return;
-    }
-
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "setFilesystemProperties for filesystem: {} with properties: {}",
-        client.getFileSystem(),
-        properties);
-
-    final String commaSeparatedProperties;
-    try {
-      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
-    } catch (CharacterCodingException ex) {
-      throw new InvalidAbfsRestOperationException(ex);
-    }
-    client.setFilesystemProperties(commaSeparatedProperties);
-  }
-
-  @Override
-  public Hashtable<String, String> getPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws
-      AzureBlobFileSystemException {
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "getPathProperties for filesystem: {} path: {}",
-        client.getFileSystem(),
-        path.toString());
-
-    final Hashtable<String, String> parsedXmsProperties;
-    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
-
-    final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
-
-    parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
-
-    return parsedXmsProperties;
-  }
-
-  @Override
-  public void setPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final Hashtable<String,
-      String> properties) throws
-      AzureBlobFileSystemException {
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "setFilesystemProperties for filesystem: {} path: {} with properties: {}",
-        client.getFileSystem(),
-        path.toString(),
-        properties);
-
-    final String commaSeparatedProperties;
-    try {
-      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
-    } catch (CharacterCodingException ex) {
-      throw new InvalidAbfsRestOperationException(ex);
-    }
-    client.setPathProperties("/" + getRelativePath(path), commaSeparatedProperties);
-  }
-
-  @Override
-  public void createFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "createFilesystem for filesystem: {}",
-        client.getFileSystem());
-
-    client.createFilesystem();
-  }
-
-  @Override
-  public void deleteFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "deleteFilesystem for filesystem: {}",
-        client.getFileSystem());
-
-    client.deleteFilesystem();
-  }
-
-  @Override
-  public OutputStream createFile(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws
-      AzureBlobFileSystemException {
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "createFile filesystem: {} path: {} overwrite: {}",
-        client.getFileSystem(),
-        path.toString(),
-        overwrite);
-
-    client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite);
-
-    final OutputStream outputStream;
-    outputStream = new FSDataOutputStream(
-        new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0,
-            configurationService.getWriteBufferSize()), null);
-    return outputStream;
-  }
-
-  @Override
-  public Void createDirectory(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "createDirectory filesystem: {} path: {} overwrite: {}",
-        client.getFileSystem(),
-        path.toString());
-
-    client.createPath("/" + getRelativePath(path), false, true);
-
-    return null;
-  }
-
-  @Override
-  public InputStream openFileForRead(final AzureBlobFileSystem azureBlobFileSystem, final Path path,
-      final FileSystem.Statistics statistics) throws AzureBlobFileSystemException {
-    final AbfsClient client = getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "openFileForRead filesystem: {} path: {}",
-        client.getFileSystem(),
-        path.toString());
-
-    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
-
-    final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
-    final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
-    final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
-
-    if (parseIsDirectory(resourceType)) {
-      throw new AbfsRestOperationException(
-          AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
-          AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
-          "openFileForRead must be used with files and not directories",
-          null);
-    }
-
-    // Add statistics for InputStream
-    return new FSDataInputStream(
-        new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
-            configurationService.getReadBufferSize(), configurationService.getReadAheadQueueDepth(), eTag));
-  }
-
-  @Override
-  public OutputStream openFileForWrite(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws
-      AzureBlobFileSystemException {
-    final AbfsClient client = getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "openFileForWrite filesystem: {} path: {} overwrite: {}",
-        client.getFileSystem(),
-        path.toString(),
-        overwrite);
-
-    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
-
-    final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
-    final Long contentLength = Long.valueOf(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
-
-    if (parseIsDirectory(resourceType)) {
-      throw new AbfsRestOperationException(
-          AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
-          AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
-          "openFileForRead must be used with files and not directories",
-          null);
-    }
-
-    final long offset = overwrite ? 0 : contentLength;
-
-    final OutputStream outputStream;
-    outputStream = new FSDataOutputStream(
-        new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
-            offset, configurationService.getWriteBufferSize()), null);
-    return outputStream;
-  }
-
-  @Override
-  public void rename(final AzureBlobFileSystem azureBlobFileSystem, final Path source, final Path destination) throws
-      AzureBlobFileSystemException {
-
-    if (isAtomicRenameKey(source.getName())) {
-      this.LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename,"
-          +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account.");
-    }
-
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "renameAsync filesystem: {} source: {} destination: {}",
-        client.getFileSystem(),
-        source.toString(),
-        destination.toString());
-
-    String continuation = null;
-    long deadline = now() + RENAME_TIMEOUT_MILISECONDS;
-
-    do {
-      if (now() > deadline) {
-        LOG.debug(
-            "Rename {} to {} timed out.",
-            source,
-            destination);
-
-        throw new TimeoutException("Rename timed out.");
-      }
-
-      AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source),
-          AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation);
-      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
-
-    } while (continuation != null && !continuation.isEmpty());
-  }
-
-  @Override
-  public void delete(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean recursive) throws
-      AzureBlobFileSystemException {
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "delete filesystem: {} path: {} recursive: {}",
-        client.getFileSystem(),
-        path.toString(),
-        String.valueOf(recursive));
-
-    String continuation = null;
-    long deadline = now() + DELETE_DIRECTORY_TIMEOUT_MILISECONDS;
-
-    do {
-      if (now() > deadline) {
-        this.LOG.debug(
-            "Delete directory {} timed out.", path);
-
-        throw new TimeoutException("Delete directory timed out.");
-      }
-
-      AbfsRestOperation op = client.deletePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation);
-      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
-
-    } while (continuation != null && !continuation.isEmpty());
-  }
-
-  @Override
-  public FileStatus getFileStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "getFileStatus filesystem: {} path: {}",
-        client.getFileSystem(),
-        path.toString());
-
-    if (path.isRoot()) {
-      AbfsRestOperation op = client.getFilesystemProperties();
-      final long blockSize = configurationService.getAzureBlockSize();
-      final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
-      final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
-      return new VersionedFileStatus(
-          azureBlobFileSystem.getOwnerUser(),
-          azureBlobFileSystem.getOwnerUserPrimaryGroup(),
-          0,
-          true,
-          1,
-          blockSize,
-          parseLastModifiedTime(lastModified).getMillis(),
-          path,
-          eTag);
-    } else {
-      AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
-
-      final long blockSize = configurationService.getAzureBlockSize();
-      final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
-      final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
-      final String contentLength = op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH);
-      final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
-
-      return new VersionedFileStatus(
-          azureBlobFileSystem.getOwnerUser(),
-          azureBlobFileSystem.getOwnerUserPrimaryGroup(),
-          parseContentLength(contentLength),
-          parseIsDirectory(resourceType),
-          1,
-          blockSize,
-          parseLastModifiedTime(lastModified).getMillis(),
-          path,
-          eTag);
-    }
-  }
-
-  @Override
-  public FileStatus[] listStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
-
-    this.LOG.debug(
-        "listStatus filesystem: {} path: {}",
-        client.getFileSystem(),
-        path.toString());
-
-    String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path);
-    String continuation = null;
-    ArrayList<FileStatus> fileStatuses = new ArrayList<>();
-
-    do {
-      AbfsRestOperation op = client.listPath(relativePath, false, LIST_MAX_RESULTS, continuation);
-      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
-      ListResultSchema retrievedSchema = op.getResult().getListResultSchema();
-      if (retrievedSchema == null) {
-        throw new AbfsRestOperationException(
-            AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
-            AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
-            "listStatusAsync path not found",
-            null, op.getResult());
-      }
-
-      long blockSize = configurationService.getAzureBlockSize();
-
-      for (ListResultEntrySchema entry : retrievedSchema.paths()) {
-        long lastModifiedMillis = 0;
-        long contentLength = entry.contentLength() == null ? 0 : entry.contentLength();
-        boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory();
-        if (entry.lastModified() != null && !entry.lastModified().isEmpty()) {
-          final DateTime dateTime = DateTime.parse(
-              entry.lastModified(),
-              DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
-          lastModifiedMillis = dateTime.getMillis();
-        }
-
-        fileStatuses.add(
-            new VersionedFileStatus(
-                azureBlobFileSystem.getOwnerUser(),
-                azureBlobFileSystem.getOwnerUserPrimaryGroup(),
-                contentLength,
-                isDirectory,
-                1,
-                blockSize,
-                lastModifiedMillis,
-                azureBlobFileSystem.makeQualified(new Path(File.separator + entry.name())),
-                entry.eTag()));
-      }
-
-    } while (continuation != null && !continuation.isEmpty());
-
-    return fileStatuses.toArray(new FileStatus[0]);
-  }
-
-  @Override
-  public synchronized void closeFileSystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
-    this.clientCache.remove(azureBlobFileSystem);
-  }
-
-  @Override
-  public boolean isAtomicRenameKey(String key) {
-    return isKeyForDirectorySet(key, azureAtomicRenameDirSet);
-  }
-
-  private String getRelativePath(final Path path) {
-    Preconditions.checkNotNull(path, "path");
-    final String relativePath = path.toUri().getPath();
-
-    if (relativePath.length() == 0) {
-      return relativePath;
-    }
-
-    if (relativePath.charAt(0) == Path.SEPARATOR_CHAR) {
-      if (relativePath.length() == 1) {
-        return AbfsHttpConstants.EMPTY_STRING;
-      }
-
-      return relativePath.substring(1);
-    }
-
-    return relativePath;
-  }
-
-  private synchronized AbfsClient getOrCreateClient(final AzureBlobFileSystem azureBlobFileSystem) throws
-      AzureBlobFileSystemException {
-    Preconditions.checkNotNull(azureBlobFileSystem, "azureBlobFileSystem");
-
-    AbfsClient client = this.clientCache.get(azureBlobFileSystem);
-
-    if (client != null) {
-      return client;
-    }
-
-    client = abfsHttpClientFactory.create(azureBlobFileSystem);
-    this.clientCache.put(
-        azureBlobFileSystem,
-        client);
-    return client;
-  }
-
-  private long parseContentLength(final String contentLength) {
-    if (contentLength == null) {
-      return -1;
-    }
-
-    return Long.parseLong(contentLength);
-  }
-
-  private boolean parseIsDirectory(final String resourceType) {
-    return resourceType == null ? false : resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
-  }
-
-  private DateTime parseLastModifiedTime(final String lastModifiedTime) {
-    return DateTime.parse(
-        lastModifiedTime,
-        DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
-  }
-
-  private String convertXmsPropertiesToCommaSeparatedString(final Hashtable<String, String> properties) throws
-      CharacterCodingException {
-    StringBuilder commaSeparatedProperties = new StringBuilder();
-
-    final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING).newEncoder();
-
-    for (Map.Entry<String, String> propertyEntry : properties.entrySet()) {
-      String key = propertyEntry.getKey();
-      String value = propertyEntry.getValue();
-
-      Boolean canEncodeValue = encoder.canEncode(value);
-      if (!canEncodeValue) {
-        throw new CharacterCodingException();
-      }
-
-      String encodedPropertyValue = DatatypeConverter.printBase64Binary(encoder.encode(CharBuffer.wrap(value)).array());
-      commaSeparatedProperties.append(key)
-              .append(AbfsHttpConstants.EQUAL)
-              .append(encodedPropertyValue);
-
-      commaSeparatedProperties.append(AbfsHttpConstants.COMMA);
-    }
-
-    if (commaSeparatedProperties.length() != 0) {
-      commaSeparatedProperties.deleteCharAt(commaSeparatedProperties.length() - 1);
-    }
-
-    return commaSeparatedProperties.toString();
-  }
-
-  private Hashtable<String, String> parseCommaSeparatedXmsProperties(String xMsProperties) throws
-      InvalidFileSystemPropertyException, InvalidAbfsRestOperationException {
-    Hashtable<String, String> properties = new Hashtable<>();
-
-    final CharsetDecoder decoder = Charset.forName(XMS_PROPERTIES_ENCODING).newDecoder();
-
-    if (xMsProperties != null && !xMsProperties.isEmpty()) {
-      String[] userProperties = xMsProperties.split(AbfsHttpConstants.COMMA);
-
-      if (userProperties.length == 0) {
-        return properties;
-      }
-
-      for (String property : userProperties) {
-        if (property.isEmpty()) {
-          throw new InvalidFileSystemPropertyException(xMsProperties);
-        }
-
-        String[] nameValue = property.split(AbfsHttpConstants.EQUAL, 2);
-        if (nameValue.length != 2) {
-          throw new InvalidFileSystemPropertyException(xMsProperties);
-        }
-
-        byte[] decodedValue = DatatypeConverter.parseBase64Binary(nameValue[1]);
-
-        final String value;
-        try {
-          value = decoder.decode(ByteBuffer.wrap(decodedValue)).toString();
-        } catch (CharacterCodingException ex) {
-          throw new InvalidAbfsRestOperationException(ex);
-        }
-        properties.put(nameValue[0], value);
-      }
-    }
-
-    return properties;
-  }
-
-  private boolean isKeyForDirectorySet(String key, Set<String> dirSet) {
-    for (String dir : dirSet) {
-      if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) {
-        return true;
-      }
-
-      try {
-        URI uri = new URI(dir);
-        if (null == uri.getAuthority()) {
-          if (key.startsWith(dir + "/")){
-            return true;
-          }
-        }
-      } catch (URISyntaxException e) {
-        this.LOG.info("URI syntax error creating URI for {}", dir);
-      }
-    }
-
-    return false;
-  }
-
-  private static class VersionedFileStatus extends FileStatus {
-    private final String version;
-
-    VersionedFileStatus(
-        final String owner, final String group,
-        final long length, final boolean isdir, final int blockReplication,
-        final long blocksize, final long modificationTime, final Path path,
-        String version) {
-      super(length, isdir, blockReplication, blocksize, modificationTime, 0,
-          new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL),
-          owner,
-          group,
-          path);
-
-      this.version = version;
-    }
-
-    /** Compare if this object is equal to another object.
-     * @param   obj the object to be compared.
-     * @return  true if two file status has the same path name; false if not.
-     */
-    @Override
-    public boolean equals(Object obj) {
-      if (obj == this) {
-        return true;
-      }
-
-      if (obj == null) {
-        return false;
-      }
-
-      if (this.getClass() == obj.getClass()) {
-        VersionedFileStatus other = (VersionedFileStatus) obj;
-        return this.getPath().equals(other.getPath()) && this.version.equals(other.version);
-      }
-
-      return false;
-    }
-
-    /**
-     * Returns a hash code value for the object, which is defined as
-     * the hash code of the path name.
-     *
-     * @return  a hash code value for the path name and version
-     */
-    @Override
-    public int hashCode() {
-      int hash = getPath().hashCode();
-      hash = 89 * hash + (this.version != null ? this.version.hashCode() : 0);
-      return hash;
-    }
-
-    /**
-     * Returns the version of this FileStatus
-     *
-     * @return  a string value for the FileStatus version
-     */
-    public String getVersion() {
-      return this.version;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java
deleted file mode 100644
index 1cbf6b5..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java
+++ /dev/null
@@ -1,81 +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.fs.azurebfs.services;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import com.google.inject.AbstractModule;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
-import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
-
-/**
- * This class is responsible to configure all the services used by Azure Blob File System.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-class AbfsServiceInjectorImpl extends AbstractModule {
-  private final Configuration configuration;
-  private final Map<Class, Class> providers;
-  private final Map<Class, Object> instances;
-
-  AbfsServiceInjectorImpl(Configuration configuration) {
-    this.providers = new HashMap<>();
-    this.instances = new HashMap<>();
-    this.configuration = configuration;
-
-    this.instances.put(Configuration.class, this.configuration);
-
-    this.providers.put(ConfigurationService.class, ConfigurationServiceImpl.class);
-
-    this.providers.put(AbfsHttpService.class, AbfsHttpServiceImpl.class);
-    this.providers.put(AbfsHttpClientFactory.class, AbfsHttpClientFactoryImpl.class);
-
-    this.providers.put(TracingService.class, TracingServiceImpl.class);
-  }
-
-  @Override
-  protected void configure() {
-    for (Map.Entry<Class, Object> entrySet : this.instances.entrySet()) {
-      bind(entrySet.getKey()).toInstance(entrySet.getValue());
-    }
-
-    for (Map.Entry<Class, Class> entrySet : this.providers.entrySet()) {
-      bind(entrySet.getKey()).to(entrySet.getValue());
-    }
-  }
-
-  protected Configuration getConfiguration() {
-    return this.configuration;
-  }
-
-  protected Map<Class, Class> getProviders() {
-    return this.providers;
-  }
-
-  protected Map<Class, Object> getInstances() {
-    return this.instances;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java
deleted file mode 100644
index 8560620..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java
+++ /dev/null
@@ -1,96 +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.fs.azurebfs.services;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider;
-import org.apache.hadoop.fs.azurebfs.contracts.services.InjectableService;
-
-/**
- * Dependency injected Azure Storage services provider.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class AbfsServiceProviderImpl implements AbfsServiceProvider {
-  private static AbfsServiceProviderImpl abfsServiceProvider;
-  private final Injector abfsServiceInjector;
-
-  private AbfsServiceProviderImpl(final Configuration configuration) {
-    this.abfsServiceInjector = Guice.createInjector(new AbfsServiceInjectorImpl(Preconditions.checkNotNull(configuration, "configuration")));
-  }
-
-  @VisibleForTesting
-  private AbfsServiceProviderImpl(final Injector abfsServiceInjector) {
-    Preconditions.checkNotNull(abfsServiceInjector, "abfsServiceInjector");
-    this.abfsServiceInjector = abfsServiceInjector;
-  }
-
-  /**
-   * Create an instance or returns existing instance of service provider.
-   * This method must be marked as synchronized to ensure thread-safety.
-   * @param configuration hadoop configuration.
-   * @return AbfsServiceProvider the service provider instance.
-   */
-  public static synchronized AbfsServiceProvider create(final Configuration configuration) {
-    if (abfsServiceProvider == null) {
-      abfsServiceProvider = new AbfsServiceProviderImpl(configuration);
-    }
-
-    return abfsServiceProvider;
-  }
-
-  /**
-   * Returns current instance of service provider.
-   * @return AbfsServiceProvider the service provider instance.
-   */
-  public static AbfsServiceProvider instance() {
-    return abfsServiceProvider;
-  }
-
-  @VisibleForTesting
-  static synchronized AbfsServiceProvider create(Injector serviceInjector) {
-    abfsServiceProvider = new AbfsServiceProviderImpl(serviceInjector);
-    return abfsServiceProvider;
-  }
-
-  /**
-   * Returns an instance of resolved injectable service by class name.
-   * The injectable service must be configured first to be resolvable.
-   * @param clazz the injectable service which is expected to be returned.
-   * @param <T> The type of injectable service.
-   * @return T instance
-   * @throws ServiceResolutionException if the service is not resolvable.
-   */
-  @Override
-  public <T extends InjectableService> T get(final Class<T> clazz) throws ServiceResolutionException {
-    try {
-      return this.abfsServiceInjector.getInstance(clazz);
-    } catch (Exception ex) {
-      throw new ServiceResolutionException(clazz.getSimpleName(), ex);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java
deleted file mode 100644
index 568ee5d..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java
+++ /dev/null
@@ -1,317 +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.fs.azurebfs.services;
-
-import java.lang.reflect.Field;
-import java.util.Map;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
-import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator;
-import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator;
-import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
-import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
-import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
-
-@Singleton
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-class ConfigurationServiceImpl implements ConfigurationService {
-  private final Configuration configuration;
-  private final boolean isSecure;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE,
-      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
-      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
-      DefaultValue = FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE)
-  private int writeBufferSize;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_READ_BUFFER_SIZE,
-      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
-      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
-      DefaultValue = FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE)
-  private int readBufferSize;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL,
-      DefaultValue = FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL)
-  private int minBackoffInterval;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL,
-      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL)
-  private int maxBackoffInterval;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BACKOFF_INTERVAL,
-      DefaultValue = FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL)
-  private int backoffInterval;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_IO_RETRIES,
-      MinValue = 0,
-      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS)
-  private int maxIoRetries;
-
-  @LongConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_SIZE_PROPERTY_NAME,
-      MinValue = 0,
-      MaxValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE,
-      DefaultValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE)
-  private long azureBlockSize;
-
-  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
-      DefaultValue = FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT)
-  private String azureBlockLocationHost;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_OUT,
-      MinValue = 1,
-      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_WRITE_THREADS)
-  private int maxConcurrentWriteThreads;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_IN,
-      MinValue = 1,
-      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_READ_THREADS)
-  private int maxConcurrentReadThreads;
-
-  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_TOLERATE_CONCURRENT_APPEND,
-      DefaultValue = FileSystemConfigurations.DEFAULT_READ_TOLERATE_CONCURRENT_APPEND)
-  private boolean tolerateOobAppends;
-
-  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ATOMIC_RENAME_KEY,
-          DefaultValue = FileSystemConfigurations.DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES)
-  private String azureAtomicDirs;
-
-  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
-      DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION)
-  private boolean createRemoteFileSystemDuringInitialization;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH,
-      DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH)
-  private int readAheadQueueDepth;
-
-  private Map<String, String> storageAccountKeys;
-
-  @Inject
-  ConfigurationServiceImpl(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException {
-    this.configuration = configuration;
-    this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false);
-
-    validateStorageAccountKeys();
-    Field[] fields = this.getClass().getDeclaredFields();
-    for (Field field : fields) {
-      field.setAccessible(true);
-      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateInt(field));
-      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateLong(field));
-      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateString(field));
-      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateBase64String(field));
-      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
-        field.set(this, validateBoolean(field));
-      }
-    }
-  }
-
-  @Override
-  public boolean isEmulator() {
-    return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
-  }
-
-  @Override
-  public boolean isSecureMode() {
-    return this.isSecure;
-  }
-
-  @Override
-  public String getStorageAccountKey(final String accountName) throws ConfigurationPropertyNotFoundException {
-    String accountKey = this.storageAccountKeys.get(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName);
-    if (accountKey == null) {
-      throw new ConfigurationPropertyNotFoundException(accountName);
-    }
-
-    return accountKey;
-  }
-
-  @Override
-  public Configuration getConfiguration() {
-    return this.configuration;
-  }
-
-  @Override
-  public int getWriteBufferSize() {
-    return this.writeBufferSize;
-  }
-
-  @Override
-  public int getReadBufferSize() {
-    return this.readBufferSize;
-  }
-
-  @Override
-  public int getMinBackoffIntervalMilliseconds() {
-    return this.minBackoffInterval;
-  }
-
-  @Override
-  public int getMaxBackoffIntervalMilliseconds() {
-    return this.maxBackoffInterval;
-  }
-
-  @Override
-  public int getBackoffIntervalMilliseconds() {
-    return this.backoffInterval;
-  }
-
-  @Override
-  public int getMaxIoRetries() {
-    return this.maxIoRetries;
-  }
-
-  @Override
-  public long getAzureBlockSize() {
-    return this.azureBlockSize;
-  }
-
-  @Override
-  public String getAzureBlockLocationHost() {
-    return this.azureBlockLocationHost;
-  }
-
-  @Override
-  public int getMaxConcurrentWriteThreads() {
-    return this.maxConcurrentWriteThreads;
-  }
-
-  @Override
-  public int getMaxConcurrentReadThreads() {
-    return this.maxConcurrentReadThreads;
-  }
-
-  @Override
-  public boolean getTolerateOobAppends() {
-    return this.tolerateOobAppends;
-  }
-
-  @Override
-  public String getAzureAtomicRenameDirs() {
-    return this.azureAtomicDirs;
-  }
-
-  @Override
-  public boolean getCreateRemoteFileSystemDuringInitialization() {
-    return this.createRemoteFileSystemDuringInitialization;
-  }
-
-  @Override
-  public int getReadAheadQueueDepth() {
-    return this.readAheadQueueDepth;
-  }
-
-  void validateStorageAccountKeys() throws InvalidConfigurationValueException {
-    Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator(
-        ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true);
-    this.storageAccountKeys = this.configuration.getValByRegex(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX);
-
-    for (Map.Entry<String, String> account : this.storageAccountKeys.entrySet()) {
-      validator.validate(account.getValue());
-    }
-  }
-
-  int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class);
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new IntegerConfigurationBasicValidator(
-        validator.MinValue(),
-        validator.MaxValue(),
-        validator.DefaultValue(),
-        validator.ConfigurationKey(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class);
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new LongConfigurationBasicValidator(
-        validator.MinValue(),
-        validator.MaxValue(),
-        validator.DefaultValue(),
-        validator.ConfigurationKey(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class);
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new StringConfigurationBasicValidator(
-        validator.ConfigurationKey(),
-        validator.DefaultValue(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class));
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new Base64StringConfigurationBasicValidator(
-        validator.ConfigurationKey(),
-        validator.DefaultValue(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
-    BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class);
-    String value = this.configuration.get(validator.ConfigurationKey());
-
-    // validate
-    return new BooleanConfigurationBasicValidator(
-        validator.ConfigurationKey(),
-        validator.DefaultValue(),
-        validator.ThrowIfInvalid()).validate(value);
-  }
-
-  @VisibleForTesting
-  void setReadBufferSize(int bufferSize) {
-    this.readBufferSize = bufferSize;
-  }
-
-  @VisibleForTesting
-  void setWriteBufferSize(int bufferSize) {
-    this.writeBufferSize = bufferSize;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
index 0c92612..54aa1ab 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
@@ -21,7 +21,10 @@ package org.apache.hadoop.fs.azurebfs.services;
 import java.util.Random;
 import java.net.HttpURLConnection;
 
-class ExponentialRetryPolicy {
+/**
+ * Retry policy used by AbfsClient.
+ * */
+public class ExponentialRetryPolicy {
   /**
    * Represents the default number of retry attempts.
    */
@@ -83,7 +86,7 @@ class ExponentialRetryPolicy {
   /**
    * Initializes a new instance of the {@link ExponentialRetryPolicy} class.
    */
-  ExponentialRetryPolicy() {
+  public ExponentialRetryPolicy() {
     this(DEFAULT_CLIENT_RETRY_COUNT, DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_CLIENT_BACKOFF);
   }
 
@@ -96,7 +99,7 @@ class ExponentialRetryPolicy {
    * @param deltaBackoff The value that will be used to calculate a random delta in the exponential delay
    *                     between retries.
    */
-  ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) {
+  public ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) {
     this.retryCount = retryCount;
     this.minBackoff = minBackoff;
     this.maxBackoff = maxBackoff;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java
deleted file mode 100644
index 99190e6..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java
+++ /dev/null
@@ -1,74 +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.fs.azurebfs.services;
-
-import java.io.IOException;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
-import org.apache.htrace.core.HTraceConfiguration;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.SpanReceiver;
-import org.apache.htrace.fasterxml.jackson.core.JsonProcessingException;
-import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.htrace.fasterxml.jackson.databind.ObjectWriter;
-import org.apache.htrace.fasterxml.jackson.databind.SerializationFeature;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * LoggerSpanReceiver is a layer between HTrace and log4j only used for {@link org.apache.hadoop.fs.azurebfs.contracts.services.TracingService}
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class LoggerSpanReceiver extends SpanReceiver {
-  private static final ObjectWriter JSON_WRITER =
-      new ObjectMapper()
-          .configure(SerializationFeature.INDENT_OUTPUT, true)
-          .configure(SerializationFeature.WRITE_BIGDECIMAL_AS_PLAIN, true)
-          .configure(SerializationFeature.WRITE_EMPTY_JSON_ARRAYS, false)
-          .configure(SerializationFeature.USE_EQUALITY_FOR_OBJECT_ID, false)
-          .writer();
-
-  public LoggerSpanReceiver(HTraceConfiguration hTraceConfiguration) {
-    Preconditions.checkNotNull(hTraceConfiguration, "hTraceConfiguration");
-  }
-
-  @Override
-  public void receiveSpan(final Span span) {
-    String jsonValue;
-
-    Logger logger = LoggerFactory.getLogger(AzureBlobFileSystem.class);
-
-    try {
-      jsonValue = JSON_WRITER.writeValueAsString(span);
-      logger.trace(jsonValue);
-    } catch (JsonProcessingException e) {
-      logger.error("Json processing error: " + e.getMessage());
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    // No-Op
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java
deleted file mode 100644
index 57b6463..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java
+++ /dev/null
@@ -1,134 +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.fs.azurebfs.services;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.Objects;
-
-import com.google.common.base.Preconditions;
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
-import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
-import org.apache.htrace.core.HTraceConfiguration;
-import org.apache.htrace.core.Sampler;
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Singleton
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-final class TracingServiceImpl implements TracingService {
-  private static final Logger LOG = LoggerFactory.getLogger(TracingService.class);
-
-  private final Tracer tracer;
-  private final ThreadLocal<SpanId> currentScopeId;
-
-  @Inject
-  TracingServiceImpl(
-      final Configuration configuration) {
-    Preconditions.checkNotNull(configuration, "configuration");
-
-    this.currentScopeId = new ThreadLocal<>();
-
-    this.tracer = new Tracer.Builder(TracingService.class.getSimpleName()).
-        conf(new HTraceConfiguration() {
-          @Override
-          public String get(String key) {
-            if (Objects.equals(key, Tracer.SPAN_RECEIVER_CLASSES_KEY)) {
-              return LoggerSpanReceiver.class.getName();
-            }
-            return null;
-          }
-
-          @Override
-          public String get(String key, String defaultValue) {
-            String value = get(key);
-            if (value != null) {
-              return value;
-            }
-            return defaultValue;
-          }
-        }).
-        build();
-
-    this.tracer.addSampler(Sampler.ALWAYS);
-  }
-
-  @Override
-  public TraceScope traceBegin(String description) {
-    if (this.LOG.isTraceEnabled()) {
-      TraceScope traceScope = this.tracer.newScope(description);
-      this.currentScopeId.set(traceScope.getSpanId());
-      return traceScope;
-    }
-
-    return null;
-  }
-
-  @Override
-  public TraceScope traceBegin(String description, SpanId parentSpanId) {
-    if (this.LOG.isTraceEnabled()) {
-      TraceScope traceScope = this.tracer.newScope(description, parentSpanId);
-      this.currentScopeId.set(traceScope.getSpanId());
-      return traceScope;
-    }
-
-    return null;
-  }
-
-  @Override
-  public void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException) {
-    if (this.LOG.isTraceEnabled()) {
-      Preconditions.checkNotNull(traceScope, "traceScope");
-      Preconditions.checkNotNull(azureBlobFileSystemException, "azureBlobFileSystemException");
-
-      StringWriter stringWriter = new StringWriter();
-      PrintWriter printWriter = new PrintWriter(stringWriter);
-      azureBlobFileSystemException.printStackTrace(printWriter);
-      printWriter.flush();
-
-      traceScope.addKVAnnotation("Exception", stringWriter.toString());
-    }
-  }
-
-  @Override
-  public SpanId getCurrentTraceScopeSpanId() {
-    return this.currentScopeId.get();
-  }
-
-  @Override
-  public void traceEnd(TraceScope traceScope) {
-    if (this.LOG.isTraceEnabled()) {
-      Preconditions.checkNotNull(traceScope, "traceScope");
-
-      SpanId[] parents = traceScope.getSpan().getParents();
-      this.currentScopeId.set(parents != null && parents.length > 0 ? parents[parents.length - 1] : null);
-      traceScope.close();
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
index 5ec1e2e..74a530c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
@@ -23,11 +23,9 @@ import java.util.UUID;
 import java.util.concurrent.Callable;
 
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
-import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.mockito.internal.util.MockUtil;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -38,12 +36,6 @@ import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
-import org.apache.hadoop.fs.azurebfs.services.MockAbfsHttpClientFactoryImpl;
-import org.apache.hadoop.fs.azurebfs.services.MockAbfsServiceInjectorImpl;
-import org.apache.hadoop.fs.azurebfs.services.MockServiceProviderImpl;
 
 import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -54,7 +46,6 @@ import static org.junit.Assume.assumeNotNull;
  * Provide dependencies for AzureBlobFileSystem tests.
  */
 public abstract class DependencyInjectedTest {
-  private final MockAbfsServiceInjectorImpl mockServiceInjector;
   private final boolean isEmulator;
   private NativeAzureFileSystem wasb;
   private String abfsScheme;
@@ -64,21 +55,19 @@ public abstract class DependencyInjectedTest {
   private String accountName;
   private String testUrl;
 
+  public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-";
+
   public DependencyInjectedTest(final boolean secure) {
     this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME);
   }
 
-  public MockAbfsServiceInjectorImpl getMockServiceInjector() {
-    return this.mockServiceInjector;
-  }
-
   protected DependencyInjectedTest() {
     this(FileSystemUriSchemes.ABFS_SCHEME);
   }
 
   private DependencyInjectedTest(final String scheme) {
     abfsScheme = scheme;
-    fileSystemName = UUID.randomUUID().toString();
+    fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString();
     configuration = new Configuration();
     configuration.addResource("azure-bfs-test.xml");
 
@@ -98,18 +87,14 @@ public abstract class DependencyInjectedTest {
     this.testUrl = defaultUri.toString();
     configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
     configuration.setBoolean(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
-    this.mockServiceInjector = new MockAbfsServiceInjectorImpl(configuration);
     this.isEmulator = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
     this.accountName = this.configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
   }
 
   @Before
   public void initialize() throws Exception {
-    if (this.isEmulator) {
-      this.mockServiceInjector.replaceProvider(AbfsHttpClientFactory.class, MockAbfsHttpClientFactoryImpl.class);
-    }
-
-    MockServiceProviderImpl.create(this.mockServiceInjector);
+    //Create filesystem first to make sure getWasbFileSystem() can return an existed filesystem.
+    this.getFileSystem();
 
     if (!this.isEmulator) {
       final URI wasbUri = new URI(abfsUrlToWasbUrl(this.getTestUrl()));
@@ -133,28 +118,24 @@ public abstract class DependencyInjectedTest {
     FileSystem.closeAll();
 
     final AzureBlobFileSystem fs = this.getFileSystem();
-    final AbfsHttpService abfsHttpService = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class);
-    abfsHttpService.deleteFilesystem(fs);
-
-    if (!(new MockUtil().isMock(abfsHttpService))) {
-      AbfsRestOperationException ex = intercept(
-          AbfsRestOperationException.class,
-          new Callable<Void>() {
-            @Override
-            public Void call() throws Exception {
-              abfsHttpService.getFilesystemProperties(fs);
-              return null;
-            }
-          });
-
-      assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode());
-    }
+    final AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
+    abfsStore.deleteFilesystem();
+
+    AbfsRestOperationException ex = intercept(
+            AbfsRestOperationException.class,
+            new Callable<Void>() {
+              @Override
+              public Void call() throws Exception {
+                fs.getAbfsStore().getFilesystemProperties();
+                return null;
+              }
+            });
+
+    assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode());
   }
 
   public AzureBlobFileSystem getFileSystem() throws Exception {
-    final Configuration configuration = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getConfiguration();
-    final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration);
-    return fs;
+    return (AzureBlobFileSystem) FileSystem.get(this.configuration);
   }
 
   protected NativeAzureFileSystem getWasbFileSystem() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
index 4985f58..ad22f99 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
@@ -22,15 +22,12 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Random;
 
-
-import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 
@@ -52,8 +49,6 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
     super();
     Configuration configuration = this.getConfiguration();
     configuration.set(ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, "0");
-    this.getMockServiceInjector().replaceInstance(Configuration.class, configuration);
-
   }
 
   @Test
@@ -82,7 +77,7 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
   @Test (expected = IOException.class)
   public void testOOBWrites() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
-    int readBufferSize = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getReadBufferSize();
+    int readBufferSize = fs.getAbfsStore().getAbfsConfiguration().getReadBufferSize();
 
     fs.create(TEST_FILE);
     FSDataOutputStream writeStream = fs.create(TEST_FILE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
index 9477587..8b96c69 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
@@ -535,17 +535,16 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
       character = (character == 'z') ? 'a' : (char) ((int) character + 1);
     }
 
-    System.out.println(("Creating test file {} of size: {} " + TEST_FILE_PATH
-            + TEST_FILE_SIZE));
+    System.out.println(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE));
     ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
 
-    try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
+    try (FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
       int bytesWritten = 0;
       while (bytesWritten < TEST_FILE_SIZE) {
         outputStream.write(buffer);
         bytesWritten += buffer.length;
       }
-      System.out.println("Closing stream {}" +  outputStream);
+      System.out.println(String.format("Closing stream %s", outputStream));
       ContractTestUtils.NanoTimer closeTimer
               = new ContractTestUtils.NanoTimer();
       outputStream.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
index aa30a85..29af1b8 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
@@ -22,18 +22,10 @@ import java.net.URI;
 
 import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Mockito;
 
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
-import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
-
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Mockito.doReturn;
 
 /**
  * Test AzureBlobFileSystem initialization.
@@ -41,18 +33,11 @@ import static org.mockito.Mockito.doReturn;
 public class ITestFileSystemInitialization extends DependencyInjectedTest {
   public ITestFileSystemInitialization() {
     super();
-
-    this.getMockServiceInjector().removeProvider(AbfsHttpService.class);
-    this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class));
   }
 
   @Test
   public void ensureAzureBlobFileSystemIsInitialized() throws Exception {
-    doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
-        .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
-        .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
-
-    final FileSystem fs = FileSystem.get(this.getConfiguration());
+    final FileSystem fs = this.getFileSystem();
     final String accountName = this.getAccountName();
     final String filesystem = this.getFileSystemName();
 
@@ -62,16 +47,12 @@ public class ITestFileSystemInitialization extends DependencyInjectedTest {
 
   @Test
   public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception {
-    doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
-        .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
-        .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
-
     final String accountName = this.getAccountName();
     final String filesystem = this.getFileSystemName();
     final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null);
     this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
 
-    final FileSystem fs = FileSystem.get(this.getConfiguration());
+    final FileSystem fs = this.getFileSystem();
     Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null));
     Assert.assertNotNull(fs.getWorkingDirectory());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
new file mode 100644
index 0000000..62d967e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
@@ -0,0 +1,126 @@
+/**
+ * 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.azurebfs;
+
+import java.util.Hashtable;
+
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test FileSystemProperties.
+ */
+public class ITestFileSystemProperties extends DependencyInjectedTest {
+  private static final int TEST_DATA = 100;
+  private static final Path TEST_PATH = new Path("/testfile");
+  public ITestFileSystemProperties() {
+    super();
+  }
+
+  @Test
+  public void testReadWriteBytesToFileAndEnsureThreadPoolCleanup() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    testWriteOneByteToFileAndEnsureThreadPoolCleanup();
+
+    FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024);
+    int i = inputStream.read();
+
+    assertEquals(TEST_DATA, i);
+  }
+
+  @Test
+  public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    FSDataOutputStream stream = fs.create(TEST_PATH);
+
+    stream.write(TEST_DATA);
+    stream.close();
+
+    FileStatus fileStatus = fs.getFileStatus(TEST_PATH);
+    assertEquals(1, fileStatus.getLen());
+  }
+
+  @Test
+  @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant")
+  public void testBase64FileSystemProperties() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "{ value: value }");
+    fs.getAbfsStore().setFilesystemProperties(properties);
+    Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
+
+    Assert.assertEquals(properties, fetchedProperties);
+  }
+
+  @Test
+  public void testBase64PathProperties() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "{ value: valueTest }");
+    fs.create(TEST_PATH);
+    fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
+    Hashtable<String, String> fetchedProperties =
+            fs.getAbfsStore().getPathProperties(TEST_PATH);
+
+    Assert.assertEquals(properties, fetchedProperties);
+  }
+
+  @Test (expected = Exception.class)
+  public void testBase64InvalidFileSystemProperties() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "{ value: value歲 }");
+    fs.getAbfsStore().setFilesystemProperties(properties);
+    Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
+
+    Assert.assertEquals(properties, fetchedProperties);
+  }
+
+  @Test (expected = Exception.class)
+  public void testBase64InvalidPathProperties() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "{ value: valueTest兩 }");
+    fs.create(TEST_PATH);
+    fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
+    Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getPathProperties(TEST_PATH);
+
+    Assert.assertEquals(properties, fetchedProperties);
+  }
+
+  @Test
+  public void testSetFileSystemProperties() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("containerForDevTest", "true");
+    fs.getAbfsStore().setFilesystemProperties(properties);
+    Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
+
+    Assert.assertEquals(properties, fetchedProperties);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
index a55599b..ef61e52 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
@@ -20,22 +20,14 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.net.URI;
 
-import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Mockito;
 
 import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
-
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Mockito.doReturn;
 
 /**
  * Test AzureBlobFileSystem registration.
@@ -43,17 +35,10 @@ import static org.mockito.Mockito.doReturn;
 public class ITestFileSystemRegistration extends DependencyInjectedTest {
   public ITestFileSystemRegistration() throws Exception {
     super();
-
-    this.getMockServiceInjector().removeProvider(AbfsHttpService.class);
-    this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class));
   }
 
   @Test
   public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
-    doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
-        .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
-        .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
-
     FileSystem fs = FileSystem.get(this.getConfiguration());
     Assert.assertTrue(fs instanceof AzureBlobFileSystem);
 
@@ -63,14 +48,10 @@ public class ITestFileSystemRegistration extends DependencyInjectedTest {
 
   @Test
   public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
-    doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
-        .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
-        .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
-
     final String accountName = this.getAccountName();
-    final String filesystem = this.getFileSystemName();
+    final String fileSystemName = this.getFileSystemName();
 
-    final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null);
+    final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, fileSystemName + "@" + accountName, null, null, null);
     this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
 
     FileSystem fs = FileSystem.get(this.getConfiguration());


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


[08/50] [abbrv] hadoop git commit: Make 3.1.1 awared by other branches - adding missing files

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/3214cd75/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_3.1.1.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_3.1.1.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_3.1.1.xml
new file mode 100644
index 0000000..d55be8d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_3.1.1.xml
@@ -0,0 +1,3327 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Thu Aug 02 05:11:24 UTC 2018 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop YARN Common 3.1.1"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-annotations.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/jdiff.jar -verbose -classpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/classes:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-3.1.1.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/commons-net/commons-net/3.6/commons-net-3.6.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/maven/org/eclipse/jetty/jetty-server/9.3.19.v20170502/jetty-server-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-http/9.3.19.v20170502/jetty-http-9.3.19.v20170
 502.jar:/maven/org/eclipse/jetty/jetty-io/9.3.19.v20170502/jetty-io-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-servlet/9.3.19.v20170502/jetty-servlet-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-security/9.3.19.v20170502/jetty-security-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-webapp/9.3.19.v20170502/jetty-webapp-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-xml/9.3.19.v20170502/jetty-xml-9.3.19.v20170502.jar:/maven/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/com/sun/jersey/jersey-servlet/1.19/jersey-servlet-1.19.jar:/maven/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar:/maven/org/apache/commons/commons-configuration2/2.1.1/commons-configuration2-2.1.1.jar:/maven/org/apache/commons/commons-lang3/3.4/commons-lang3-3.4.jar:/maven/org/slf4j/slf4j-log4j12/1.7.25/slf4j-log4j12-1.7.25.jar:/maven/org/apache/avro/avro/1.7.7/avro-1.7.7.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy
 /snappy-java/1.0.5/snappy-java-1.0.5.jar:/maven/com/google/re2j/re2j/1.1/re2j-1.1.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.12.0/curator-client-2.12.0.jar:/maven/org/apache/curator/curator-recipes/2.12.0/curator-recipes-2.12.0.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/htrace/htrace-core4/4.1.0-incubating/htrace-core4-4.1.0-incubating.jar:/maven/org/apache/zookeeper/zookeeper/3.4.9/zookeeper-3.4.9.jar:/maven/io/netty/netty/3.10.5.Final/netty-3.10.5.Final.jar:/maven/org/apache/kerby/kerb-simplekdc/1.0.1/kerb-simplekdc-1.0.1.jar:/maven/org/apache/kerby/kerb-client/1.0.1/kerb-client-1.0.1.jar:/maven/org/apache/kerby/kerby-config/1.0.1/kerby-config-1.0.1.jar:/maven/org/apache/kerby/kerb-core/1.0.1/kerb-core-1.0.1.jar:/maven/org/apache/kerby/kerby-pkix/1.0.1/kerby-pkix-1.0.1.jar:/maven/org/apache/kerby/kerby-asn1/1.0.1/kerby-asn1-1.0.1.jar:/mav
 en/org/apache/kerby/kerby-util/1.0.1/kerby-util-1.0.1.jar:/maven/org/apache/kerby/kerb-common/1.0.1/kerb-common-1.0.1.jar:/maven/org/apache/kerby/kerb-crypto/1.0.1/kerb-crypto-1.0.1.jar:/maven/org/apache/kerby/kerb-util/1.0.1/kerb-util-1.0.1.jar:/maven/org/apache/kerby/token-provider/1.0.1/token-provider-1.0.1.jar:/maven/org/apache/kerby/kerb-admin/1.0.1/kerb-admin-1.0.1.jar:/maven/org/apache/kerby/kerb-server/1.0.1/kerb-server-1.0.1.jar:/maven/org/apache/kerby/kerb-identity/1.0.1/kerb-identity-1.0.1.jar:/maven/org/apache/kerby/kerby-xdr/1.0.1/kerby-xdr-1.0.1.jar:/maven/org/codehaus/woodstox/stax2-api/3.1.4/stax2-api-3.1.4.jar:/maven/com/fasterxml/woodstox/woodstox-core/5.0.3/woodstox-core-5.0.3.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-3.1.1.jar:/maven/com/fasterxml/jackson/core/jackson-annotations/2.7.8/jackson-annotations-2.7.8.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-3.1.1.jar:/maven/com/nimbusds/nimbus
 -jose-jwt/4.41.1/nimbus-jose-jwt-4.41.1.jar:/maven/com/github/stephenc/jcip/jcip-annotations/1.0-1/jcip-annotations-1.0-1.jar:/maven/net/minidev/json-smart/2.3/json-smart-2.3.jar:/maven/net/minidev/accessors-smart/1.2/accessors-smart-1.2.jar:/maven/org/ow2/asm/asm/5.0.4/asm-5.0.4.jar:/maven/org/apache/curator/curator-framework/2.12.0/curator-framework-2.12.0.jar:/maven/javax/xml/bind/jaxb-api/2.2.11/jaxb-api-2.2.11.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/maven/commons-codec/commons-codec/1.11/commons-codec-1.11.jar:/maven/org/eclipse/jetty/jetty-util/9.3.19.v20170502/jetty-util-9.3.19.v20170502.jar:/maven/com/sun/jersey/jersey-core/1.19/jersey-core-1.19.jar:/maven/javax/ws/rs/jsr311-api/1.1.1/jsr311-api-1.1.1.jar:/maven/com/sun/jersey/jersey-client/1.19/jersey-client-1.19.jar:/
 maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/org/slf4j/slf4j-api/1.7.25/slf4j-api-1.7.25.jar:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-3.1.1.jar:/usr/lib/jvm/java-8-openjdk-amd64/lib/tools.jar:/maven/com/google/inject/extensions/guice-servlet/4.0/guice-servlet-4.0.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/commons-io/commons-io/2.5/commons-io-2.5.jar:/maven/com/google/inject/guice/4.0/guice-4.0.jar:/maven/javax/inject/javax.inject/1/javax.inject-1.jar:/maven/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/maven/com/sun/jersey/jersey-server/1.19/jersey-server-1.19.jar:/maven/com/sun/jersey/jersey-json/1.19/jersey-json-1.19.jar:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/maven/org/codehaus/
 jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/com/sun/jersey/contribs/jersey-guice/1.19/jersey-guice-1.19.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/maven/com/fasterxml/jackson/core/jackson-core/2.7.8/jackson-core-2.7.8.jar:/maven/com/fasterxml/jackson/core/jackson-databind/2.7.8/jackson-databind-2.7.8.jar:/maven/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.7.8/jackson-module-jaxb-annotations-2.7.8.jar:/maven/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.7.8/jackson-jaxrs-json-provider-2.7.8.jar:/maven/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.7.8/jackson-jaxrs-base-2.7.8.jar:/maven/xerces/xercesImpl/2.11.0/xercesImpl-2.11.0.jar:/maven/xml-apis/xml-apis/1.4.01/xml-apis-1.4.01.jar -sourcepath /build/
 source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-annotations.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/jdiff.jar -apidir /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/site/jdiff/xml -apiname Apache Hadoop YARN Common 3.1.1 -->
+<package name="org.apache.hadoop.yarn">
+  <!-- start class org.apache.hadoop.yarn.ContainerLogAppender -->
+  <class name="ContainerLogAppender" extends="org.apache.log4j.FileAppender"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="ContainerLogAppender"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="activateOptions"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="append"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.log4j.spi.LoggingEvent"/>
+    </method>
+    <method name="flush"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerLogDir" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Getter/Setter methods for log4j.]]>
+      </doc>
+    </method>
+    <method name="setContainerLogDir"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogDir" type="java.lang.String"/>
+    </method>
+    <method name="getContainerLogFile" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainerLogFile"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogFile" type="java.lang.String"/>
+    </method>
+    <method name="getTotalLogFileSize" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setTotalLogFileSize"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="logSize" type="long"/>
+      <doc>
+      <![CDATA[Setter so that log4j can configure it from the
+  configuration(log4j.properties).]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[A simple log4j-appender for container's logs.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.ContainerLogAppender -->
+  <!-- start class org.apache.hadoop.yarn.ContainerRollingLogAppender -->
+  <class name="ContainerRollingLogAppender" extends="org.apache.log4j.RollingFileAppender"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="ContainerRollingLogAppender"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="activateOptions"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="flush"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerLogDir" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Getter/Setter methods for log4j.]]>
+      </doc>
+    </method>
+    <method name="setContainerLogDir"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogDir" type="java.lang.String"/>
+    </method>
+    <method name="getContainerLogFile" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainerLogFile"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogFile" type="java.lang.String"/>
+    </method>
+    <doc>
+    <![CDATA[A simple log4j-appender for container's logs.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.ContainerRollingLogAppender -->
+  <!-- start class org.apache.hadoop.yarn.YarnUncaughtExceptionHandler -->
+  <class name="YarnUncaughtExceptionHandler" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.lang.Thread.UncaughtExceptionHandler"/>
+    <constructor name="YarnUncaughtExceptionHandler"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="uncaughtException"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="t" type="java.lang.Thread"/>
+      <param name="e" type="java.lang.Throwable"/>
+    </method>
+    <doc>
+    <![CDATA[This class is intended to be installed by calling 
+ {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)}
+ In the main entry point.  It is intended to try and cleanly shut down
+ programs using the YARN Event framework.
+ 
+ Note: Right now it only will shut down the program if a Error is caught, but
+ not any other exception.  Anything else is just logged.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.YarnUncaughtExceptionHandler -->
+</package>
+<package name="org.apache.hadoop.yarn.api">
+</package>
+<package name="org.apache.hadoop.yarn.client">
+  <!-- start class org.apache.hadoop.yarn.client.AHSProxy -->
+  <class name="AHSProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AHSProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAHSProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="ahsAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="rmAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.AHSProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.ClientRMProxy -->
+  <class name="ClientRMProxy" extends="org.apache.hadoop.yarn.client.RMProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="createRMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="configuration" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Create a proxy to the ResourceManager for the specified protocol.
+ @param configuration Configuration with all the required information.
+ @param protocol Client protocol for which proxy is being requested.
+ @param <T> Type of proxy.
+ @return Proxy to the ResourceManager for the specified client protocol.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRMDelegationTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Get the token service name to be used for RMDelegationToken. Depending
+ on whether HA is enabled or not, this method generates the appropriate
+ service name as a comma-separated list of service addresses.
+
+ @param conf Configuration corresponding to the cluster we need the
+             RMDelegationToken for
+ @return - Service name for RMDelegationToken]]>
+      </doc>
+    </method>
+    <method name="getAMRMTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="address" type="java.lang.String"/>
+      <param name="defaultAddr" type="java.lang.String"/>
+      <param name="defaultPort" type="int"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.ClientRMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.NMProxy -->
+  <class name="NMProxy" extends="org.apache.hadoop.yarn.client.ServerProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createNMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="ugi" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="rpc" type="org.apache.hadoop.yarn.ipc.YarnRPC"/>
+      <param name="serverAddress" type="java.net.InetSocketAddress"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.NMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.RMProxy -->
+  <class name="RMProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMProxy"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createRetryPolicy" return="org.apache.hadoop.io.retry.RetryPolicy"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="retryTime" type="long"/>
+      <param name="retryInterval" type="long"/>
+      <param name="isHAEnabled" type="boolean"/>
+      <doc>
+      <![CDATA[Fetch retry policy from Configuration and create the
+ retry policy with specified retryTime and retry interval.]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.RMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.ServerProxy -->
+  <class name="ServerProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ServerProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createRetryPolicy" return="org.apache.hadoop.io.retry.RetryPolicy"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="maxWaitTimeStr" type="java.lang.String"/>
+      <param name="defMaxWaitTime" type="long"/>
+      <param name="connectRetryIntervalStr" type="java.lang.String"/>
+      <param name="defRetryInterval" type="long"/>
+    </method>
+    <method name="createRetriableProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="user" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="rpc" type="org.apache.hadoop.yarn.ipc.YarnRPC"/>
+      <param name="serverAddress" type="java.net.InetSocketAddress"/>
+      <param name="retryPolicy" type="org.apache.hadoop.io.retry.RetryPolicy"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.ServerProxy -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api">
+  <!-- start class org.apache.hadoop.yarn.client.api.AppAdminClient -->
+  <class name="AppAdminClient" extends="org.apache.hadoop.service.CompositeService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppAdminClient"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAppAdminClient" return="org.apache.hadoop.yarn.client.api.AppAdminClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appType" type="java.lang.String"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[<p>
+ Create a new instance of AppAdminClient.
+ </p>
+
+ @param appType application type
+ @param conf configuration
+ @return app admin client]]>
+      </doc>
+    </method>
+    <method name="actionLaunch" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="fileName" type="java.lang.String"/>
+      <param name="appName" type="java.lang.String"/>
+      <param name="lifetime" type="java.lang.Long"/>
+      <param name="queue" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Launch a new YARN application.
+ </p>
+
+ @param fileName specification of application
+ @param appName name of the application
+ @param lifetime lifetime of the application
+ @param queue queue of the application
+ @return exit code
+ @throws IOException IOException
+ @throws YarnException exception in client or server]]>
+      </doc>
+    </method>
+    <method name="actionStop" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appName" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Stop a YARN application (attempt to stop gracefully before killing the
+ application). In the case of a long-running service, the service may be
+ restarted later.
+ </p>
+
+ @param appName the name of the application
+ @return exit code
+ @throws IOException IOException
+ @throws YarnException exception in client or server]]>
+      </doc>
+    </method>
+    <method name="actionStart" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appName" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Start a YARN application from a previously saved specification. In the
+ case of a long-running service, the service must have been previously
+ launched/started and then stopped, or previously saved but not started.
+ </p>
+
+ @param appName the name of the application
+ @return exit code
+ @throws IOException IOException
+ @throws YarnException exception in client or server]]>
+      </doc>
+    </method>
+    <method name="actionSave" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="fileName" type="java.lang.String"/>
+      <param name="appName" type="java.lang.String"/>
+      <param name="lifetime" type="java.lang.Long"/>
+      <param name="queue" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Save the specification for a YARN application / long-running service.
+ The application may be started later.
+ </p>
+
+ @param fileName specification of application to save
+ @param appName name of the application
+ @param lifetime lifetime of the application
+ @param queue queue of the application
+ @return exit code
+ @throws IOException IOException
+ @throws YarnException exception in client or server]]>
+      </doc>
+    </method>
+    <method name="actionDestroy" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appName" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Remove the specification and all application data for a YARN application.
+ The application cannot be running.
+ </p>
+
+ @param appName the name of the application
+ @return exit code
+ @throws IOException IOException
+ @throws YarnException exception in client or server]]>
+      </doc>
+    </method>
+    <method name="actionFlex" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appName" type="java.lang.String"/>
+      <param name="componentCounts" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Change the number of running containers for a component of a YARN
+ application / long-running service.
+ </p>
+
+ @param appName the name of the application
+ @param componentCounts map of component name to new component count or
+                        amount to change existing component count (e.g.
+                        5, +5, -5)
+ @return exit code
+ @throws IOException IOException
+ @throws YarnException exception in client or server]]>
+      </doc>
+    </method>
+    <method name="enableFastLaunch" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="destinationFolder" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Upload AM dependencies to HDFS. This makes future application launches
+ faster since the dependencies do not have to be uploaded on each launch.
+ </p>
+
+ @param destinationFolder
+          an optional HDFS folder where dependency tarball will be uploaded
+ @return exit code
+ @throws IOException
+           IOException
+ @throws YarnException
+           exception in client or server]]>
+      </doc>
+    </method>
+    <method name="getStatusString" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appIdOrName" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Get detailed app specific status string for a YARN application.
+ </p>
+
+ @param appIdOrName appId or appName
+ @return status string
+ @throws IOException IOException
+ @throws YarnException exception in client or server]]>
+      </doc>
+    </method>
+    <method name="initiateUpgrade" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appName" type="java.lang.String"/>
+      <param name="fileName" type="java.lang.String"/>
+      <param name="autoFinalize" type="boolean"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[Initiate upgrade of a long running service.
+
+ @param appName      the name of the application.
+ @param fileName     specification of application upgrade to save.
+ @param autoFinalize when true, finalization of upgrade will be done
+                     automatically.
+ @return exit code
+ @throws IOException   IOException
+ @throws YarnException exception in client or server]]>
+      </doc>
+    </method>
+    <method name="actionUpgradeInstances" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appName" type="java.lang.String"/>
+      <param name="componentInstances" type="java.util.List"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[Upgrade component instances of a long running service.
+
+ @param appName            the name of the application.
+ @param componentInstances the name of the component instances.]]>
+      </doc>
+    </method>
+    <method name="actionUpgradeComponents" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appName" type="java.lang.String"/>
+      <param name="components" type="java.util.List"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[Upgrade components of a long running service.
+
+ @param appName    the name of the application.
+ @param components the name of the components.]]>
+      </doc>
+    </method>
+    <method name="actionCleanUp" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appName" type="java.lang.String"/>
+      <param name="userName" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[Operation to be performed by the RM after an application has completed.
+
+ @param appName  the name of the application.
+ @param userName the name of the user.
+ @return exit code]]>
+      </doc>
+    </method>
+    <field name="YARN_APP_ADMIN_CLIENT_PREFIX" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="DEFAULT_TYPE" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="DEFAULT_CLASS_NAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="UNIT_TEST_TYPE" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="UNIT_TEST_CLASS_NAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Client for managing applications.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AppAdminClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.TimelineClient -->
+  <class name="TimelineClient" extends="org.apache.hadoop.service.CompositeService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="TimelineClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createTimelineClient" return="org.apache.hadoop.yarn.client.api.TimelineClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Creates an instance of the timeline v.1.x client.
+ The current UGI when the user initialize the client will be used to do the
+ put and the delegation token operations. The current user may use
+ {@link UserGroupInformation#doAs} another user to construct and initialize
+ a timeline client if the following operations are supposed to be conducted
+ by that user.
+
+ @return the created timeline client instance]]>
+      </doc>
+    </method>
+    <method name="putEntities" return="org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="entities" type="org.apache.hadoop.yarn.api.records.timeline.TimelineEntity[]"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a number of conceptual entities to the timeline
+ server. It is a blocking API. The method will not return until it gets the
+ response from the timeline server.
+ </p>
+ 
+ @param entities
+          the collection of {@link TimelineEntity}
+ @return the error information if the sent entities are not correctly stored
+ @throws IOException if there are I/O errors
+ @throws YarnException if entities are incomplete/invalid]]>
+      </doc>
+    </method>
+    <method name="putEntities" return="org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <param name="groupId" type="org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId"/>
+      <param name="entities" type="org.apache.hadoop.yarn.api.records.timeline.TimelineEntity[]"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a number of conceptual entities to the timeline
+ server. It is a blocking API. The method will not return until it gets the
+ response from the timeline server.
+
+ This API is only for timeline service v1.5
+ </p>
+
+ @param appAttemptId {@link ApplicationAttemptId}
+ @param groupId {@link TimelineEntityGroupId}
+ @param entities
+          the collection of {@link TimelineEntity}
+ @return the error information if the sent entities are not correctly stored
+ @throws IOException if there are I/O errors
+ @throws YarnException if entities are incomplete/invalid]]>
+      </doc>
+    </method>
+    <method name="putDomain"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="domain" type="org.apache.hadoop.yarn.api.records.timeline.TimelineDomain"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a domain to the timeline server. It is a
+ blocking API. The method will not return until it gets the response from
+ the timeline server.
+ </p>
+ 
+ @param domain
+          an {@link TimelineDomain} object
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="putDomain"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <param name="domain" type="org.apache.hadoop.yarn.api.records.timeline.TimelineDomain"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a domain to the timeline server. It is a
+ blocking API. The method will not return until it gets the response from
+ the timeline server.
+
+ This API is only for timeline service v1.5
+ </p>
+
+ @param domain
+          an {@link TimelineDomain} object
+ @param appAttemptId {@link ApplicationAttemptId}
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="getDelegationToken" return="org.apache.hadoop.security.token.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="renewer" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a delegation token so as to be able to talk to the timeline server in a
+ secure way.
+ </p>
+ 
+ @param renewer
+          Address of the renewer who can renew these tokens when needed by
+          securely talking to the timeline server
+ @return a delegation token ({@link Token}) that can be used to talk to the
+         timeline server
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="renewDelegationToken" return="long"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="timelineDT" type="org.apache.hadoop.security.token.Token"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Renew a timeline delegation token.
+ </p>
+ 
+ @param timelineDT
+          the delegation token to renew
+ @return the new expiration time
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="cancelDelegationToken"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="timelineDT" type="org.apache.hadoop.security.token.Token"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Cancel a timeline delegation token.
+ </p>
+ 
+ @param timelineDT
+          the delegation token to cancel
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[A client library that can be used to post some information in terms of a
+ number of conceptual entities. This client library needs to be used along
+ with Timeline V.1.x server versions.
+ Refer {@link TimelineV2Client} for ATS V2 interface.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.TimelineClient -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api.impl">
+</package>
+<package name="org.apache.hadoop.yarn.event">
+  <!-- start class org.apache.hadoop.yarn.event.AbstractEvent -->
+  <class name="AbstractEvent" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.event.Event"/>
+    <constructor name="AbstractEvent" type="TYPE"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AbstractEvent" type="TYPE, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getTimestamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getType" return="TYPE"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Parent class of all the events. All events extend this class.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.event.AbstractEvent -->
+  <!-- start class org.apache.hadoop.yarn.event.AsyncDispatcher -->
+  <class name="AsyncDispatcher" extends="org.apache.hadoop.service.AbstractService"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.event.Dispatcher"/>
+    <constructor name="AsyncDispatcher"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AsyncDispatcher" type="java.util.concurrent.BlockingQueue"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AsyncDispatcher" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Set a name for this dispatcher thread.
+ @param dispatcherName name of the dispatcher thread]]>
+      </doc>
+    </constructor>
+    <method name="disableExitOnDispatchException"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="serviceStart"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="setDrainEventsOnStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="serviceStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="dispatch"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.hadoop.yarn.event.Event"/>
+    </method>
+    <method name="register"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="eventType" type="java.lang.Class"/>
+      <param name="handler" type="org.apache.hadoop.yarn.event.EventHandler"/>
+    </method>
+    <method name="getEventHandler" return="org.apache.hadoop.yarn.event.EventHandler"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="isEventThreadWaiting" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <method name="isDrained" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <method name="isStopped" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <field name="eventDispatchers" type="java.util.Map"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Dispatches {@link Event}s in a separate thread. Currently only single thread
+ does that. Potentially there could be multiple channels for each event type
+ class and a thread pool can be used to dispatch the events.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.event.AsyncDispatcher -->
+  <!-- start interface org.apache.hadoop.yarn.event.Dispatcher -->
+  <interface name="Dispatcher"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getEventHandler" return="org.apache.hadoop.yarn.event.EventHandler"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="register"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="eventType" type="java.lang.Class"/>
+      <param name="handler" type="org.apache.hadoop.yarn.event.EventHandler"/>
+    </method>
+    <doc>
+    <![CDATA[Event Dispatcher interface. It dispatches events to registered 
+ event handlers based on event types.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.Dispatcher -->
+  <!-- start interface org.apache.hadoop.yarn.event.Event -->
+  <interface name="Event"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getType" return="TYPE"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getTimestamp" return="long"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Interface defining events api.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.Event -->
+  <!-- start interface org.apache.hadoop.yarn.event.EventHandler -->
+  <interface name="EventHandler"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="handle"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="event" type="T"/>
+    </method>
+    <doc>
+    <![CDATA[Interface for handling events of type T
+
+ @param <T> parameterized event of type T]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.EventHandler -->
+</package>
+<package name="org.apache.hadoop.yarn.factories">
+</package>
+<package name="org.apache.hadoop.yarn.factory.providers">
+</package>
+<package name="org.apache.hadoop.yarn.logaggregation">
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat -->
+  <class name="AggregatedLogFormat" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AggregatedLogFormat"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey -->
+  <class name="AggregatedLogFormat.LogKey" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.io.Writable"/>
+    <constructor name="LogKey"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="LogKey" type="org.apache.hadoop.yarn.api.records.ContainerId"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="LogKey" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="obj" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader -->
+  <class name="AggregatedLogFormat.LogReader" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="LogReader" type="org.apache.hadoop.conf.Configuration, org.apache.hadoop.fs.Path"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </constructor>
+    <method name="getApplicationOwner" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns the owner of the application.
+
+ @return the application owner.
+ @throws IOException if we can not get the application owner.]]>
+      </doc>
+    </method>
+    <method name="getApplicationAcls" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns ACLs for the application. An empty map is returned if no ACLs are
+ found.
+
+ @return a map of the Application ACLs.
+ @throws IOException if we can not get the application acls.]]>
+      </doc>
+    </method>
+    <method name="next" return="java.io.DataInputStream"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Read the next key and return the value-stream.
+ 
+ @param key the log key
+ @return the valueStream if there are more keys or null otherwise
+ @throws IOException if we can not get the dataInputStream
+ for the next key]]>
+      </doc>
+    </method>
+    <method name="readAcontainerLogs"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="writer" type="java.io.Writer"/>
+      <param name="logUploadedTime" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes all logs for a single container to the provided writer.
+ @param valueStream the valueStream
+ @param writer the log writer
+ @param logUploadedTime the time stamp
+ @throws IOException if we can not read the container logs.]]>
+      </doc>
+    </method>
+    <method name="readAcontainerLogs"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="writer" type="java.io.Writer"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes all logs for a single container to the provided writer.
+ @param valueStream the value stream
+ @param writer the log writer
+ @throws IOException if we can not read the container logs.]]>
+      </doc>
+    </method>
+    <method name="readAContainerLogsForALogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <param name="logUploadedTime" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ all types for a single container.
+ 
+ @param valueStream the value stream
+ @param out the print stream
+ @param logUploadedTime the time stamp
+ @throws IOException if we can not read the container log by specifying
+ the container log type.]]>
+      </doc>
+    </method>
+    <method name="readAContainerLogsForALogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <param name="logUploadedTime" type="long"/>
+      <param name="bytes" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ all types for a single container for the specific bytes.
+
+ @param valueStream the value stream
+ @param out the output print stream
+ @param logUploadedTime the log upload time stamp
+ @param bytes the output size of the log
+ @throws IOException if we can not read the container log]]>
+      </doc>
+    </method>
+    <method name="readAContainerLogsForALogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ all types for a single container.
+ 
+ @param valueStream the value stream
+ @param out the output print stream
+ @throws IOException if we can not read the container log]]>
+      </doc>
+    </method>
+    <method name="readContainerLogsForALogType" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <param name="logUploadedTime" type="long"/>
+      <param name="logType" type="java.util.List"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ the specific types for a single container.
+ @param valueStream the value stream
+ @param out the output print stream
+ @param logUploadedTime the log uploaded time stamp
+ @param logType the given log type
+ @throws IOException if we can not read the container logs]]>
+      </doc>
+    </method>
+    <method name="readContainerLogsForALogType" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <param name="logUploadedTime" type="long"/>
+      <param name="logType" type="java.util.List"/>
+      <param name="bytes" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ the specific types for a single container.
+ @param valueStream the value stream
+ @param out the output print stream
+ @param logUploadedTime the log uploaded time stamp
+ @param logType the given log type
+ @throws IOException if we can not read the container logs]]>
+      </doc>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader -->
+</package>
+<package name="org.apache.hadoop.yarn.logaggregation.filecontroller">
+</package>
+<package name="org.apache.hadoop.yarn.logaggregation.filecontroller.ifile">
+</package>
+<package name="org.apache.hadoop.yarn.logaggregation.filecontroller.tfile">
+</package>
+<package name="org.apache.hadoop.yarn.nodelabels">
+</package>
+<package name="org.apache.hadoop.yarn.nodelabels.event">
+</package>
+<package name="org.apache.hadoop.yarn.security">
+  <!-- start class org.apache.hadoop.yarn.security.AccessRequest -->
+  <class name="AccessRequest" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AccessRequest" type="org.apache.hadoop.yarn.security.PrivilegedEntity, org.apache.hadoop.security.UserGroupInformation, org.apache.hadoop.yarn.security.AccessType, java.lang.String, java.lang.String, java.lang.String, java.util.List"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAccessType" return="org.apache.hadoop.yarn.security.AccessType"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAppId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAppName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getEntity" return="org.apache.hadoop.yarn.security.PrivilegedEntity"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getForwardedAddresses" return="java.util.List"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRemoteAddress" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[This request object contains all the context information to determine whether
+ a user has permission to access the target entity.
+ user       : the user who's currently accessing
+ accessType : the access type against the entity.
+ entity     : the target object user is accessing.
+ appId      : the associated app Id for current access. This could be null
+              if no app is associated.
+ appName    : the associated app name for current access. This could be null if
+              no app is associated.
+ remoteAddress : The caller's remote ip address.
+ forwardedAddresses : In case this is an http request, this contains the
+                    originating IP address of a client connecting to a web
+                    server through an HTTP proxy or load balancer. This
+                    parameter is null, if it's a RPC request.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.AccessRequest -->
+  <!-- start class org.apache.hadoop.yarn.security.AMRMTokenIdentifier -->
+  <class name="AMRMTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AMRMTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId, int"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.AMRMTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND_NAME" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[AMRMTokenIdentifier is the TokenIdentifier to be used by
+ ApplicationMasters to authenticate to the ResourceManager.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.AMRMTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.AMRMTokenSelector -->
+  <class name="AMRMTokenSelector" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.security.token.TokenSelector"/>
+    <constructor name="AMRMTokenSelector"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="selectToken" return="org.apache.hadoop.security.token.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="service" type="org.apache.hadoop.io.Text"/>
+      <param name="tokens" type="java.util.Collection"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.AMRMTokenSelector -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo -->
+  <class name="ContainerManagerSecurityInfo" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerManagerSecurityInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerTokenIdentifier -->
+  <class name="ContainerTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext"
+      static="false" final="false" visibility="public"
+      deprecated="Use one of the other constructors instead.">
+      <doc>
+      <![CDATA[Creates a instance.
+
+ @param appSubmitter appSubmitter
+ @param containerID container ID
+ @param creationTime creation time
+ @param expiryTimeStamp expiry timestamp
+ @param hostName hostname
+ @param logAggregationContext log aggregation context
+ @param masterKeyId master key ID
+ @param priority priority
+ @param r resource needed by the container
+ @param rmIdentifier ResourceManager identifier
+ @deprecated Use one of the other constructors instead.]]>
+      </doc>
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext, java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext, java.lang.String, org.apache.hadoop.yarn.server.api.ContainerType"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, int, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext, java.lang.String, org.apache.hadoop.yarn.server.api.ContainerType, org.apache.hadoop.yarn.api.records.ExecutionType"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, int, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext, java.lang.String, org.apache.hadoop.yarn.server.api.ContainerType, org.apache.hadoop.yarn.api.records.ExecutionType, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Convenience Constructor for existing clients.
+
+ @param containerID containerID
+ @param containerVersion containerVersion
+ @param hostName hostName
+ @param appSubmitter appSubmitter
+ @param r resource
+ @param expiryTimeStamp expiryTimeStamp
+ @param masterKeyId masterKeyId
+ @param rmIdentifier rmIdentifier
+ @param priority priority
+ @param creationTime creationTime
+ @param logAggregationContext logAggregationContext
+ @param nodeLabelExpression nodeLabelExpression
+ @param containerType containerType
+ @param executionType executionType
+ @param allocationRequestId allocationRequestId]]>
+      </doc>
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, int, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext, java.lang.String, org.apache.hadoop.yarn.server.api.ContainerType, org.apache.hadoop.yarn.api.records.ExecutionType, long, java.util.Set"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a Container Token Identifier.
+
+ @param containerID containerID
+ @param containerVersion containerVersion
+ @param hostName hostName
+ @param appSubmitter appSubmitter
+ @param r resource
+ @param expiryTimeStamp expiryTimeStamp
+ @param masterKeyId masterKeyId
+ @param rmIdentifier rmIdentifier
+ @param priority priority
+ @param creationTime creationTime
+ @param logAggregationContext logAggregationContext
+ @param nodeLabelExpression nodeLabelExpression
+ @param containerType containerType
+ @param executionType executionType
+ @param allocationRequestId allocationRequestId
+ @param allocationTags Set of allocation Tags.]]>
+      </doc>
+    </constructor>
+    <constructor name="ContainerTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Default constructor needed by RPC layer/SecretManager.]]>
+      </doc>
+    </constructor>
+    <method name="getContainerID" return="org.apache.hadoop.yarn.api.records.ContainerId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmitter" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNmHostAddress" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getResource" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getExpiryTimeStamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getMasterKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getPriority" return="org.apache.hadoop.yarn.api.records.Priority"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getCreationTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRMIdentifier" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the RMIdentifier of RM in which containers are allocated.
+ @return RMIdentifier]]>
+      </doc>
+    </method>
+    <method name="getContainerType" return="org.apache.hadoop.yarn.server.api.ContainerType"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the ContainerType of container to allocate.
+ @return ContainerType]]>
+      </doc>
+    </method>
+    <method name="getExecutionType" return="org.apache.hadoop.yarn.api.records.ExecutionType"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the ExecutionType of container to allocate
+ @return ExecutionType]]>
+      </doc>
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.ContainerTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getLogAggregationContext" return="org.apache.hadoop.yarn.api.records.LogAggregationContext"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAllocationRequestId" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getVersion" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the Container version
+ @return container version]]>
+      </doc>
+    </method>
+    <method name="getNodeLabelExpression" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the node-label-expression in the original ResourceRequest]]>
+      </doc>
+    </method>
+    <method name="getAllcationTags" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[TokenIdentifier for a container. Encodes {@link ContainerId},
+ {@link Resource} needed by the container and the target NMs host-address.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerTokenSelector -->
+  <class name="ContainerTokenSelector" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.security.token.TokenSelector"/>
+    <constructor name="ContainerTokenSelector"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="selectToken" return="org.apache.hadoop.security.token.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="service" type="org.apache.hadoop.io.Text"/>
+      <param name="tokens" type="java.util.Collection"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerTokenSelector -->
+  <!-- start class org.apache.hadoop.yarn.security.NMTokenIdentifier -->
+  <class name="NMTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId, org.apache.hadoop.yarn.api.records.NodeId, java.lang.String, int"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="NMTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Default constructor needed by RPC/Secret manager]]>
+      </doc>
+    </constructor>
+    <method name="getApplicationAttemptId" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNodeId" return="org.apache.hadoop.yarn.api.records.NodeId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmitter" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.NMTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.NMTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.SchedulerSecurityInfo -->
+  <class name="SchedulerSecurityInfo" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"


<TRUNCATED>

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


[30/50] [abbrv] hadoop git commit: HDDS-339. Add block length and blockId in PutKeyResponse. Contributed by Shashikant Banerjee.

Posted by tm...@apache.org.
HDDS-339. Add block length and blockId in PutKeyResponse. Contributed by Shashikant Banerjee.


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

Branch: refs/heads/HADOOP-15407
Commit: 398d89554398a38ffa1347524286cd437f94f3ae
Parents: 15241c6
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Fri Aug 10 23:45:56 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Fri Aug 10 23:45:56 2018 +0530

----------------------------------------------------------------------
 .../main/proto/DatanodeContainerProtocol.proto  |   1 +
 .../container/keyvalue/KeyValueHandler.java     |  18 +-
 .../container/keyvalue/helpers/KeyUtils.java    |  50 +++-
 .../container/keyvalue/impl/KeyManagerImpl.java |   4 +-
 .../keyvalue/interfaces/KeyManager.java         |   3 +-
 .../ozone/scm/TestCommittedBlockLengthAPI.java  | 216 ----------------
 .../TestGetCommittedBlockLengthAndPutKey.java   | 254 +++++++++++++++++++
 7 files changed, 313 insertions(+), 233 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/398d8955/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
index af06346..930f314 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -308,6 +308,7 @@ message  PutKeyRequestProto {
 }
 
 message  PutKeyResponseProto {
+  required GetCommittedBlockLengthResponseProto committedBlockLength = 1;
 }
 
 message  GetKeyRequestProto  {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/398d8955/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index f4699dd..8364a77 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -421,6 +421,7 @@ public class KeyValueHandler extends Handler {
   ContainerCommandResponseProto handlePutKey(
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
 
+    long blockLength;
     if (!request.hasPutKey()) {
       LOG.debug("Malformed Put Key request. trace ID: {}",
           request.getTraceID());
@@ -433,7 +434,7 @@ public class KeyValueHandler extends Handler {
       KeyData keyData = KeyData.getFromProtoBuf(
           request.getPutKey().getKeyData());
       long numBytes = keyData.getProtoBufMessage().toByteArray().length;
-      commitKey(keyData, kvContainer);
+      blockLength = commitKey(keyData, kvContainer);
       metrics.incContainerBytesStats(Type.PutKey, numBytes);
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
@@ -443,7 +444,7 @@ public class KeyValueHandler extends Handler {
           request);
     }
 
-    return KeyUtils.getKeyResponseSuccess(request);
+    return KeyUtils.putKeyResponseSuccess(request, blockLength);
   }
 
   private void commitPendingKeys(KeyValueContainer kvContainer)
@@ -456,12 +457,13 @@ public class KeyValueHandler extends Handler {
     }
   }
 
-  private void commitKey(KeyData keyData, KeyValueContainer kvContainer)
+  private long commitKey(KeyData keyData, KeyValueContainer kvContainer)
       throws IOException {
     Preconditions.checkNotNull(keyData);
-    keyManager.putKey(kvContainer, keyData);
+    long length = keyManager.putKey(kvContainer, keyData);
     //update the open key Map in containerManager
     this.openContainerBlockMap.removeFromKeyMap(keyData.getBlockID());
+    return length;
   }
   /**
    * Handle Get Key operation. Calls KeyManager to process the request.
@@ -662,8 +664,12 @@ public class KeyValueHandler extends Handler {
           request.getWriteChunk().getStage() == Stage.COMBINED) {
         metrics.incContainerBytesStats(Type.WriteChunk, request.getWriteChunk()
             .getChunkData().getLen());
-        // the openContainerBlockMap should be updated only while writing data
-        // not during COMMIT_STAGE of handling write chunk request.
+      }
+
+      if (request.getWriteChunk().getStage() == Stage.COMMIT_DATA
+          || request.getWriteChunk().getStage() == Stage.COMBINED) {
+        // the openContainerBlockMap should be updated only during
+        // COMMIT_STAGE of handling write chunk request.
         openContainerBlockMap.addChunk(blockID, chunkInfoProto);
       }
     } catch (StorageContainerException ex) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/398d8955/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
index 2be966d..a83d298 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
@@ -27,6 +27,10 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .GetKeyResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
+    GetCommittedBlockLengthResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
+    PutKeyResponseProto;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
@@ -123,6 +127,26 @@ public final class KeyUtils {
   }
 
   /**
+   * Returns putKey response success.
+   * @param msg - Request.
+   * @return Response.
+   */
+  public static ContainerCommandResponseProto putKeyResponseSuccess(
+      ContainerCommandRequestProto msg, long blockLength) {
+    GetCommittedBlockLengthResponseProto.Builder
+        committedBlockLengthResponseBuilder =
+        getCommittedBlockLengthResponseBuilder(blockLength,
+            msg.getPutKey().getKeyData().getBlockID());
+    PutKeyResponseProto.Builder putKeyResponse =
+        PutKeyResponseProto.newBuilder();
+    putKeyResponse
+        .setCommittedBlockLength(committedBlockLengthResponseBuilder);
+    ContainerProtos.ContainerCommandResponseProto.Builder builder =
+        ContainerUtils.getSuccessResponseBuilder(msg);
+    builder.setPutKey(putKeyResponse);
+    return builder.build();
+  }
+  /**
    * Returns successful keyResponse.
    * @param msg - Request.
    * @return Response.
@@ -150,18 +174,26 @@ public final class KeyUtils {
    * @param msg - Request.
    * @return Response.
    */
-  public static ContainerProtos.ContainerCommandResponseProto
-  getBlockLengthResponse(ContainerProtos.
-      ContainerCommandRequestProto msg, long blockLength) {
+  public static ContainerCommandResponseProto getBlockLengthResponse(
+          ContainerCommandRequestProto msg, long blockLength) {
+    GetCommittedBlockLengthResponseProto.Builder
+        committedBlockLengthResponseBuilder =
+        getCommittedBlockLengthResponseBuilder(blockLength,
+            msg.getGetCommittedBlockLength().getBlockID());
+    ContainerProtos.ContainerCommandResponseProto.Builder builder =
+        ContainerUtils.getSuccessResponseBuilder(msg);
+    builder.setGetCommittedBlockLength(committedBlockLengthResponseBuilder);
+    return builder.build();
+  }
+
+  private static GetCommittedBlockLengthResponseProto.Builder
+  getCommittedBlockLengthResponseBuilder(
+      long blockLength, ContainerProtos.DatanodeBlockID blockID) {
     ContainerProtos.GetCommittedBlockLengthResponseProto.Builder
         getCommittedBlockLengthResponseBuilder = ContainerProtos.
         GetCommittedBlockLengthResponseProto.newBuilder();
     getCommittedBlockLengthResponseBuilder.setBlockLength(blockLength);
-    getCommittedBlockLengthResponseBuilder
-        .setBlockID(msg.getGetCommittedBlockLength().getBlockID());
-    ContainerProtos.ContainerCommandResponseProto.Builder builder =
-        ContainerUtils.getSuccessResponseBuilder(msg);
-    builder.setGetCommittedBlockLength(getCommittedBlockLengthResponseBuilder);
-    return  builder.build();
+    getCommittedBlockLengthResponseBuilder.setBlockID(blockID);
+    return getCommittedBlockLengthResponseBuilder;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/398d8955/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
index 58bf1f8..6370f8e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
@@ -67,9 +67,10 @@ public class KeyManagerImpl implements KeyManager {
    *
    * @param container - Container for which key need to be added.
    * @param data     - Key Data.
+   * @return length of the key.
    * @throws IOException
    */
-  public void putKey(Container container, KeyData data) throws IOException {
+  public long putKey(Container container, KeyData data) throws IOException {
     Preconditions.checkNotNull(data, "KeyData cannot be null for put " +
         "operation.");
     Preconditions.checkState(data.getContainerID() >= 0, "Container Id " +
@@ -87,6 +88,7 @@ public class KeyManagerImpl implements KeyManager {
 
     // Increment keycount here
     container.getContainerData().incrKeyCount();
+    return data.getSize();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/398d8955/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
index dad688e..37871be 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
@@ -35,9 +35,10 @@ public interface KeyManager {
    *
    * @param container - Container for which key need to be added.
    * @param data     - Key Data.
+   * @return length of the Key.
    * @throws IOException
    */
-  void putKey(Container container, KeyData data) throws IOException;
+  long putKey(Container container, KeyData data) throws IOException;
 
   /**
    * Gets an existing key.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/398d8955/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
deleted file mode 100644
index 3c6479f..0000000
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
+++ /dev/null
@@ -1,216 +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.ozone.scm;
-
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.commons.lang3.RandomUtils;
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.XceiverClientSpi;
-import org.apache.hadoop.hdds.scm.container.common.helpers.
-    ContainerWithPipeline;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.hdds.scm.container.common.helpers.
-    StorageContainerException;
-import org.apache.hadoop.hdds.scm.container.placement.algorithms.
-    ContainerPlacementPolicy;
-import org.apache.hadoop.hdds.scm.container.placement.algorithms.
-    SCMContainerPlacementCapacity;
-import org.apache.hadoop.hdds.scm.protocolPB.
-    StorageContainerLocationProtocolClientSideTranslatorPB;
-import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.container.ContainerTestHelper;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-
-import java.util.UUID;
-
-/**
- * Test Container calls.
- */
-public class TestCommittedBlockLengthAPI {
-
-  private static MiniOzoneCluster cluster;
-  private static OzoneConfiguration ozoneConfig;
-  private static StorageContainerLocationProtocolClientSideTranslatorPB
-      storageContainerLocationClient;
-  private static XceiverClientManager xceiverClientManager;
-  private static String containerOwner = "OZONE";
-
-  @BeforeClass
-  public static void init() throws Exception {
-    ozoneConfig = new OzoneConfiguration();
-    ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
-        SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
-    cluster =
-        MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(1).build();
-    cluster.waitForClusterToBeReady();
-    storageContainerLocationClient =
-        cluster.getStorageContainerLocationClient();
-    xceiverClientManager = new XceiverClientManager(ozoneConfig);
-  }
-
-  @AfterClass
-  public static void shutdown() throws InterruptedException {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-    IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
-  }
-
-  @Test
-  public void tesGetCommittedBlockLength() throws Exception {
-    ContainerProtos.GetCommittedBlockLengthResponseProto response;
-    String traceID = UUID.randomUUID().toString();
-    ContainerWithPipeline container = storageContainerLocationClient
-        .allocateContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    long containerID = container.getContainerInfo().getContainerID();
-    Pipeline pipeline = container.getPipeline();
-    XceiverClientSpi client =
-        xceiverClientManager.acquireClient(pipeline, containerID);
-    //create the container
-    ContainerProtocolCalls.createContainer(client, containerID, traceID);
-
-    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
-    byte[] data =
-        RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes();
-    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
-        ContainerTestHelper
-            .getWriteChunkRequest(container.getPipeline(), blockID,
-                data.length);
-    client.sendCommand(writeChunkRequest);
-    // Now, explicitly make a putKey request for the block.
-    ContainerProtos.ContainerCommandRequestProto putKeyRequest =
-        ContainerTestHelper
-            .getPutKeyRequest(pipeline, writeChunkRequest.getWriteChunk());
-    client.sendCommand(putKeyRequest);
-    response = ContainerProtocolCalls
-        .getCommittedBlockLength(client, blockID, traceID);
-    // make sure the block ids in the request and response are same.
-    Assert.assertTrue(
-        BlockID.getFromProtobuf(response.getBlockID()).equals(blockID));
-    Assert.assertTrue(response.getBlockLength() == data.length);
-    xceiverClientManager.releaseClient(client);
-  }
-
-  @Test
-  public void tesGetCommittedBlockLengthWithClosedContainer()
-      throws Exception {
-    String traceID = UUID.randomUUID().toString();
-    ContainerWithPipeline container = storageContainerLocationClient
-        .allocateContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    long containerID = container.getContainerInfo().getContainerID();
-    Pipeline pipeline = container.getPipeline();
-    XceiverClientSpi client =
-        xceiverClientManager.acquireClient(pipeline, containerID);
-    // create the container
-    ContainerProtocolCalls.createContainer(client, containerID, traceID);
-
-    byte[] data =
-        RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes();
-    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
-    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
-        ContainerTestHelper
-            .getWriteChunkRequest(container.getPipeline(), blockID,
-                data.length);
-    client.sendCommand(writeChunkRequest);
-    // close the container
-    ContainerProtocolCalls.closeContainer(client, containerID, traceID);
-    ContainerProtos.GetCommittedBlockLengthResponseProto response =
-        ContainerProtocolCalls
-            .getCommittedBlockLength(client, blockID, traceID);
-    // make sure the block ids in the request and response are same.
-    // This will also ensure that closing the container committed the block
-    // on the Datanodes.
-    Assert.assertTrue(
-        BlockID.getFromProtobuf(response.getBlockID()).equals(blockID));
-    Assert.assertTrue(response.getBlockLength() == data.length);
-    xceiverClientManager.releaseClient(client);
-  }
-
-  @Test
-  public void tesGetCommittedBlockLengthForInvalidBlock() throws Exception {
-    String traceID = UUID.randomUUID().toString();
-    ContainerWithPipeline container = storageContainerLocationClient
-        .allocateContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    long containerID = container.getContainerInfo().getContainerID();
-    XceiverClientSpi client = xceiverClientManager
-        .acquireClient(container.getPipeline(), containerID);
-    ContainerProtocolCalls.createContainer(client, containerID, traceID);
-
-    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
-    // move the container to closed state
-    ContainerProtocolCalls.closeContainer(client, containerID, traceID);
-    try {
-      // There is no block written inside the container. The request should
-      // fail.
-      ContainerProtocolCalls.getCommittedBlockLength(client, blockID, traceID);
-      Assert.fail("Expected exception not thrown");
-    } catch (StorageContainerException sce) {
-      Assert.assertTrue(sce.getMessage().contains("Unable to find the key"));
-    }
-    xceiverClientManager.releaseClient(client);
-  }
-
-  @Test
-  public void testGetCommittedBlockLengthForOpenBlock() throws Exception {
-    String traceID = UUID.randomUUID().toString();
-    ContainerWithPipeline container = storageContainerLocationClient
-        .allocateContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    long containerID = container.getContainerInfo().getContainerID();
-    XceiverClientSpi client = xceiverClientManager
-        .acquireClient(container.getPipeline(), containerID);
-    ContainerProtocolCalls
-        .createContainer(client, containerID, traceID);
-
-    BlockID blockID =
-        ContainerTestHelper.getTestBlockID(containerID);
-    ContainerProtos.ContainerCommandRequestProto requestProto =
-        ContainerTestHelper
-            .getWriteChunkRequest(container.getPipeline(), blockID, 1024);
-    client.sendCommand(requestProto);
-    try {
-      ContainerProtocolCalls.getCommittedBlockLength(client, blockID, traceID);
-      Assert.fail("Expected Exception not thrown");
-    } catch (StorageContainerException sce) {
-      Assert.assertEquals(ContainerProtos.Result.BLOCK_NOT_COMMITTED,
-          sce.getResult());
-    }
-    // now close the container, it should auto commit pending open blocks
-    ContainerProtocolCalls
-        .closeContainer(client, containerID, traceID);
-    ContainerProtos.GetCommittedBlockLengthResponseProto response =
-        ContainerProtocolCalls
-            .getCommittedBlockLength(client, blockID, traceID);
-    Assert.assertTrue(response.getBlockLength() == 1024);
-    xceiverClientManager.releaseClient(client);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/398d8955/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java
new file mode 100644
index 0000000..f82b0d3
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java
@@ -0,0 +1,254 @@
+/**
+ * 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.ozone.scm;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.hdds.scm.container.common.helpers.
+    ContainerWithPipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.
+    StorageContainerException;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.
+    ContainerPlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.
+    SCMContainerPlacementCapacity;
+import org.apache.hadoop.hdds.scm.protocolPB.
+    StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+import java.util.UUID;
+
+/**
+ * Test Container calls.
+ */
+public class TestGetCommittedBlockLengthAndPutKey {
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration ozoneConfig;
+  private static StorageContainerLocationProtocolClientSideTranslatorPB
+      storageContainerLocationClient;
+  private static XceiverClientManager xceiverClientManager;
+  private static String containerOwner = "OZONE";
+
+  @BeforeClass
+  public static void init() throws Exception {
+    ozoneConfig = new OzoneConfiguration();
+    ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
+        SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
+    cluster =
+        MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(1).build();
+    cluster.waitForClusterToBeReady();
+    storageContainerLocationClient =
+        cluster.getStorageContainerLocationClient();
+    xceiverClientManager = new XceiverClientManager(ozoneConfig);
+  }
+
+  @AfterClass
+  public static void shutdown() throws InterruptedException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+    IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
+  }
+
+  @Test
+  public void tesGetCommittedBlockLength() throws Exception {
+    ContainerProtos.GetCommittedBlockLengthResponseProto response;
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    Pipeline pipeline = container.getPipeline();
+    XceiverClientSpi client =
+        xceiverClientManager.acquireClient(pipeline, containerID);
+    //create the container
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    byte[] data =
+        RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes();
+    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
+        ContainerTestHelper
+            .getWriteChunkRequest(container.getPipeline(), blockID,
+                data.length);
+    client.sendCommand(writeChunkRequest);
+    // Now, explicitly make a putKey request for the block.
+    ContainerProtos.ContainerCommandRequestProto putKeyRequest =
+        ContainerTestHelper
+            .getPutKeyRequest(pipeline, writeChunkRequest.getWriteChunk());
+    client.sendCommand(putKeyRequest);
+    response = ContainerProtocolCalls
+        .getCommittedBlockLength(client, blockID, traceID);
+    // make sure the block ids in the request and response are same.
+    Assert.assertTrue(
+        BlockID.getFromProtobuf(response.getBlockID()).equals(blockID));
+    Assert.assertTrue(response.getBlockLength() == data.length);
+    xceiverClientManager.releaseClient(client);
+  }
+
+  @Test
+  public void tesGetCommittedBlockLengthWithClosedContainer()
+      throws Exception {
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    Pipeline pipeline = container.getPipeline();
+    XceiverClientSpi client =
+        xceiverClientManager.acquireClient(pipeline, containerID);
+    // create the container
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    byte[] data =
+        RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes();
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
+        ContainerTestHelper
+            .getWriteChunkRequest(container.getPipeline(), blockID,
+                data.length);
+    client.sendCommand(writeChunkRequest);
+    // close the container
+    ContainerProtocolCalls.closeContainer(client, containerID, traceID);
+    ContainerProtos.GetCommittedBlockLengthResponseProto response =
+        ContainerProtocolCalls
+            .getCommittedBlockLength(client, blockID, traceID);
+    // make sure the block ids in the request and response are same.
+    // This will also ensure that closing the container committed the block
+    // on the Datanodes.
+    Assert.assertTrue(
+        BlockID.getFromProtobuf(response.getBlockID()).equals(blockID));
+    Assert.assertTrue(response.getBlockLength() == data.length);
+    xceiverClientManager.releaseClient(client);
+  }
+
+  @Test
+  public void tesGetCommittedBlockLengthForInvalidBlock() throws Exception {
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    XceiverClientSpi client = xceiverClientManager
+        .acquireClient(container.getPipeline(), containerID);
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    // move the container to closed state
+    ContainerProtocolCalls.closeContainer(client, containerID, traceID);
+    try {
+      // There is no block written inside the container. The request should
+      // fail.
+      ContainerProtocolCalls.getCommittedBlockLength(client, blockID, traceID);
+      Assert.fail("Expected exception not thrown");
+    } catch (StorageContainerException sce) {
+      Assert.assertTrue(sce.getMessage().contains("Unable to find the key"));
+    }
+    xceiverClientManager.releaseClient(client);
+  }
+
+  @Test
+  public void testGetCommittedBlockLengthForOpenBlock() throws Exception {
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    XceiverClientSpi client = xceiverClientManager
+        .acquireClient(container.getPipeline(), containerID);
+    ContainerProtocolCalls
+        .createContainer(client, containerID, traceID);
+
+    BlockID blockID =
+        ContainerTestHelper.getTestBlockID(containerID);
+    ContainerProtos.ContainerCommandRequestProto requestProto =
+        ContainerTestHelper
+            .getWriteChunkRequest(container.getPipeline(), blockID, 1024);
+    client.sendCommand(requestProto);
+    try {
+      ContainerProtocolCalls.getCommittedBlockLength(client, blockID, traceID);
+      Assert.fail("Expected Exception not thrown");
+    } catch (StorageContainerException sce) {
+      Assert.assertEquals(ContainerProtos.Result.BLOCK_NOT_COMMITTED,
+          sce.getResult());
+    }
+    // now close the container, it should auto commit pending open blocks
+    ContainerProtocolCalls
+        .closeContainer(client, containerID, traceID);
+    ContainerProtos.GetCommittedBlockLengthResponseProto response =
+        ContainerProtocolCalls
+            .getCommittedBlockLength(client, blockID, traceID);
+    Assert.assertTrue(response.getBlockLength() == 1024);
+    xceiverClientManager.releaseClient(client);
+  }
+
+  @Test
+  public void tesPutKeyResposne() throws Exception {
+    ContainerProtos.PutKeyResponseProto response;
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    Pipeline pipeline = container.getPipeline();
+    XceiverClientSpi client =
+        xceiverClientManager.acquireClient(pipeline, containerID);
+    //create the container
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    byte[] data =
+        RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes();
+    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
+        ContainerTestHelper
+            .getWriteChunkRequest(container.getPipeline(), blockID,
+                data.length);
+    client.sendCommand(writeChunkRequest);
+    // Now, explicitly make a putKey request for the block.
+    ContainerProtos.ContainerCommandRequestProto putKeyRequest =
+        ContainerTestHelper
+            .getPutKeyRequest(pipeline, writeChunkRequest.getWriteChunk());
+    response = client.sendCommand(putKeyRequest).getPutKey();
+    // make sure the block ids in the request and response are same.
+    // This will also ensure that closing the container committed the block
+    // on the Datanodes.
+    Assert.assertEquals(BlockID
+        .getFromProtobuf(response.getCommittedBlockLength().getBlockID()),
+        blockID);
+    Assert.assertEquals(
+        response.getCommittedBlockLength().getBlockLength(), data.length);
+    xceiverClientManager.releaseClient(client);
+  }
+}
\ 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


[11/50] [abbrv] hadoop git commit: HDDS-267. Handle consistency issues during container update/close.

Posted by tm...@apache.org.
HDDS-267. Handle consistency issues during container update/close.


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

Branch: refs/heads/HADOOP-15407
Commit: d81cd3611a449bcd7970ff2f1392a5e868e28f7e
Parents: 8478732
Author: Hanisha Koneru <ha...@apache.org>
Authored: Wed Aug 8 16:47:25 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Wed Aug 8 16:47:25 2018 -0700

----------------------------------------------------------------------
 .../container/common/impl/ContainerData.java    |  1 -
 .../container/keyvalue/KeyValueContainer.java   | 54 ++++++-------------
 .../container/keyvalue/KeyValueHandler.java     | 21 ++++++--
 .../keyvalue/TestKeyValueContainer.java         | 16 ------
 .../container/keyvalue/TestKeyValueHandler.java | 55 ++++++++++++++++----
 .../common/impl/TestContainerPersistence.java   |  8 ---
 6 files changed, 80 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81cd361/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index 5803628..26954a7 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -257,7 +257,6 @@ public abstract class ContainerData {
    * Marks this container as closed.
    */
   public synchronized void closeContainer() {
-    // TODO: closed or closing here
     setState(ContainerLifeCycleState.CLOSED);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81cd361/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 353fe4f..c96f997 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -138,7 +138,7 @@ public class KeyValueContainer implements Container {
 
       // Create .container file
       File containerFile = getContainerFile();
-      writeToContainerFile(containerFile, true);
+      createContainerFile(containerFile);
 
     } catch (StorageContainerException ex) {
       if (containerMetaDataPath != null && containerMetaDataPath.getParentFile()
@@ -165,11 +165,11 @@ public class KeyValueContainer implements Container {
   }
 
   /**
-   * Creates .container file and checksum file.
+   * Writes to .container file.
    *
-   * @param containerFile
-   * @param isCreate true if we are creating a new container file and false if
-   *                we are updating an existing container file.
+   * @param containerFile container file name
+   * @param isCreate True if creating a new file. False is updating an
+   *                 existing container file.
    * @throws StorageContainerException
    */
   private void writeToContainerFile(File containerFile, boolean isCreate)
@@ -181,19 +181,18 @@ public class KeyValueContainer implements Container {
       ContainerDataYaml.createContainerFile(
           ContainerType.KeyValueContainer, containerData, tempContainerFile);
 
+      // NativeIO.renameTo is an atomic function. But it might fail if the
+      // container file already exists. Hence, we handle the two cases
+      // separately.
       if (isCreate) {
-        // When creating a new container, .container file should not exist
-        // already.
         NativeIO.renameTo(tempContainerFile, containerFile);
       } else {
-        // When updating a container, the .container file should exist. If
-        // not, the container is in an inconsistent state.
         Files.move(tempContainerFile.toPath(), containerFile.toPath(),
             StandardCopyOption.REPLACE_EXISTING);
       }
 
     } catch (IOException ex) {
-      throw new StorageContainerException("Error during creation of " +
+      throw new StorageContainerException("Error while creating/ updating " +
           ".container file. ContainerID: " + containerId, ex,
           CONTAINER_FILES_CREATE_ERROR);
     } finally {
@@ -206,27 +205,14 @@ public class KeyValueContainer implements Container {
     }
   }
 
+  private void createContainerFile(File containerFile)
+      throws StorageContainerException {
+    writeToContainerFile(containerFile, true);
+  }
 
   private void updateContainerFile(File containerFile)
       throws StorageContainerException {
-
-    long containerId = containerData.getContainerID();
-
-    if (!containerFile.exists()) {
-      throw new StorageContainerException("Container is an Inconsistent " +
-          "state, missing .container file. ContainerID: " + containerId,
-          INVALID_CONTAINER_STATE);
-    }
-
-    try {
-      writeToContainerFile(containerFile, false);
-    } catch (IOException e) {
-      //TODO : Container update failure is not handled currently. Might
-      // lead to loss of .container file. When Update container feature
-      // support is added, this failure should also be handled.
-      throw new StorageContainerException("Container update failed. " +
-          "ContainerID: " + containerId, CONTAINER_FILES_CREATE_ERROR);
-    }
+    writeToContainerFile(containerFile, false);
   }
 
 
@@ -256,19 +242,15 @@ public class KeyValueContainer implements Container {
     // complete this action
     try {
       writeLock();
-      long containerId = containerData.getContainerID();
-      if(!containerData.isValid()) {
-        LOG.debug("Invalid container data. Container Id: {}", containerId);
-        throw new StorageContainerException("Invalid container data. " +
-            "ContainerID: " + containerId, INVALID_CONTAINER_STATE);
-      }
+
       containerData.closeContainer();
       File containerFile = getContainerFile();
-
       // update the new container data to .container File
       updateContainerFile(containerFile);
 
     } catch (StorageContainerException ex) {
+      // Failed to update .container file. Reset the state to CLOSING
+      containerData.setState(ContainerLifeCycleState.CLOSING);
       throw ex;
     } finally {
       writeUnlock();
@@ -332,8 +314,6 @@ public class KeyValueContainer implements Container {
       // update the new container data to .container File
       updateContainerFile(containerFile);
     } catch (StorageContainerException  ex) {
-      // TODO:
-      // On error, reset the metadata.
       containerData.setMetadata(oldMetadata);
       throw ex;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81cd361/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index a281a53..f4699dd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerLifeCycleState;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .GetSmallFileRequestProto;
@@ -77,6 +79,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CLOSED_CONTAINER_RETRY;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_INTERNAL_ERROR;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CLOSED_CONTAINER_IO;
@@ -378,8 +382,18 @@ public class KeyValueHandler extends Handler {
       return ContainerUtils.malformedRequest(request);
     }
 
+    long containerID = kvContainer.getContainerData().getContainerID();
+    ContainerLifeCycleState containerState = kvContainer.getContainerState();
+
     try {
-      checkContainerOpen(kvContainer);
+      if (containerState == ContainerLifeCycleState.CLOSED) {
+        throw new StorageContainerException("Container already closed. " +
+            "ContainerID: " + containerID, CLOSED_CONTAINER_RETRY);
+      } else if (containerState == ContainerLifeCycleState.INVALID) {
+        LOG.debug("Invalid container data. ContainerID: {}", containerID);
+        throw new StorageContainerException("Invalid container data. " +
+            "ContainerID: " + containerID, INVALID_CONTAINER_STATE);
+      }
 
       KeyValueContainerData kvData = kvContainer.getContainerData();
 
@@ -773,10 +787,9 @@ public class KeyValueHandler extends Handler {
   private void checkContainerOpen(KeyValueContainer kvContainer)
       throws StorageContainerException {
 
-    ContainerProtos.ContainerLifeCycleState containerState =
-        kvContainer.getContainerState();
+    ContainerLifeCycleState containerState = kvContainer.getContainerState();
 
-    if (containerState == ContainerProtos.ContainerLifeCycleState.OPEN) {
+    if (containerState == ContainerLifeCycleState.OPEN) {
       return;
     } else {
       String msg = "Requested operation not allowed as ContainerState is " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81cd361/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index 37c7f8a..6ff2eca 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume
     .RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
-
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DiskChecker;
@@ -243,21 +242,6 @@ public class TestKeyValueContainer {
   }
 
   @Test
-  public void testCloseInvalidContainer() throws Exception {
-    try {
-      keyValueContainerData.setState(ContainerProtos.ContainerLifeCycleState
-          .INVALID);
-      keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
-      keyValueContainer.close();
-      fail("testCloseInvalidContainer failed");
-    } catch (StorageContainerException ex) {
-      assertEquals(ContainerProtos.Result.INVALID_CONTAINER_STATE,
-          ex.getResult());
-      GenericTestUtils.assertExceptionContains("Invalid container data", ex);
-    }
-  }
-
-  @Test
   public void testUpdateContainer() throws IOException {
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
     Map<String, String> metadata = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81cd361/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
index 747687b..ce12e1f 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
@@ -25,12 +25,16 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestRule;
@@ -59,8 +63,8 @@ public class TestKeyValueHandler {
   @Rule
   public TestRule timeout = new Timeout(300000);
 
-  private HddsDispatcher dispatcher;
-  private KeyValueHandler handler;
+  private static HddsDispatcher dispatcher;
+  private static KeyValueHandler handler;
 
   private final static String DATANODE_UUID = UUID.randomUUID().toString();
 
@@ -69,14 +73,11 @@ public class TestKeyValueHandler {
 
   private static final long DUMMY_CONTAINER_ID = 9999;
 
-  @Test
-  /**
-   * Test that Handler handles different command types correctly.
-   */
-  public void testHandlerCommandHandling() throws Exception{
+  @BeforeClass
+  public static void setup() throws StorageContainerException {
     // Create mock HddsDispatcher and KeyValueHandler.
-    this.handler = Mockito.mock(KeyValueHandler.class);
-    this.dispatcher = Mockito.mock(HddsDispatcher.class);
+    handler = Mockito.mock(KeyValueHandler.class);
+    dispatcher = Mockito.mock(HddsDispatcher.class);
     Mockito.when(dispatcher.getHandler(any())).thenReturn(handler);
     Mockito.when(dispatcher.dispatch(any())).thenCallRealMethod();
     Mockito.when(dispatcher.getContainer(anyLong())).thenReturn(
@@ -84,6 +85,13 @@ public class TestKeyValueHandler {
     Mockito.when(handler.handle(any(), any())).thenCallRealMethod();
     doCallRealMethod().when(dispatcher).setMetricsForTesting(any());
     dispatcher.setMetricsForTesting(Mockito.mock(ContainerMetrics.class));
+  }
+
+  @Test
+  /**
+   * Test that Handler handles different command types correctly.
+   */
+  public void testHandlerCommandHandling() throws Exception {
 
     // Test Create Container Request handling
     ContainerCommandRequestProto createContainerRequest =
@@ -250,4 +258,33 @@ public class TestKeyValueHandler {
   }
 
 
+  @Test
+  public void testCloseInvalidContainer() {
+    long containerID = 1234L;
+    Configuration conf = new Configuration();
+    KeyValueContainerData kvData = new KeyValueContainerData(containerID, 1);
+    KeyValueContainer container = new KeyValueContainer(kvData, conf);
+    kvData.setState(ContainerProtos.ContainerLifeCycleState.INVALID);
+
+    // Create Close container request
+    ContainerCommandRequestProto closeContainerRequest =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.CloseContainer)
+            .setContainerID(DUMMY_CONTAINER_ID)
+            .setDatanodeUuid(DATANODE_UUID)
+            .setCloseContainer(ContainerProtos.CloseContainerRequestProto
+                .getDefaultInstance())
+            .build();
+    dispatcher.dispatch(closeContainerRequest);
+
+    Mockito.when(handler.handleCloseContainer(any(), any()))
+        .thenCallRealMethod();
+    // Closing invalid container should return error response.
+    ContainerProtos.ContainerCommandResponseProto response =
+        handler.handleCloseContainer(closeContainerRequest, container);
+
+    Assert.assertTrue("Close container should return Invalid container error",
+        response.getResult().equals(
+            ContainerProtos.Result.INVALID_CONTAINER_STATE));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81cd361/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
index 5322c8e..016b94c 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
@@ -775,14 +775,6 @@ public class TestContainerPersistence {
     Assert.assertEquals("bilbo_new_1",
         actualNewData.getMetadata().get("owner"));
 
-    // Update a non-existing container
-    exception.expect(StorageContainerException.class);
-    exception.expectMessage("Container is an Inconsistent " +
-        "state, missing .container file.");
-    Container nonExistentContainer = new KeyValueContainer(
-        new KeyValueContainerData(RandomUtils.nextLong(),
-            ContainerTestHelper.CONTAINER_MAX_SIZE_GB), conf);
-    nonExistentContainer.update(newMetadata, false);
   }
 
   private KeyData writeKeyHelper(BlockID blockID)


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


[35/50] [abbrv] hadoop git commit: HADOOP-15446. ABFS: tune imports & javadocs; stabilise tests. Contributed by Steve Loughran and Da Zhou.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
index d107c9d..d696481 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
@@ -27,13 +27,11 @@ import org.junit.Test;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 /**
  * Test AzureBlobFileSystem back compatibility with WASB.
  */
-public class ITestAzureBlobFileSystemBackCompat extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemBackCompat extends
+    AbstractAbfsIntegrationTest {
   public ITestAzureBlobFileSystemBackCompat() {
     super();
   }
@@ -54,13 +52,13 @@ public class ITestAzureBlobFileSystemBackCompat extends DependencyInjectedTest {
     blockBlob.uploadText("");
 
     FileStatus[] fileStatuses = fs.listStatus(new Path("/test/10/"));
-    assertEquals(fileStatuses.length, 2);
-    assertEquals(fileStatuses[0].getPath().getName(), "10");
+    assertEquals(2, fileStatuses.length);
+    assertEquals("10", fileStatuses[0].getPath().getName());
     assertTrue(fileStatuses[0].isDirectory());
-    assertEquals(fileStatuses[0].getLen(), 0);
-    assertEquals(fileStatuses[1].getPath().getName(), "123");
+    assertEquals(0, fileStatuses[0].getLen());
+    assertEquals("123", fileStatuses[1].getPath().getName());
     assertTrue(fileStatuses[1].isDirectory());
-    assertEquals(fileStatuses[1].getLen(), 0);
+    assertEquals(0, fileStatuses[1].getLen());
   }
 
   private String getBlobConnectionString() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java
index c158e03..90eff97 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java
@@ -33,30 +33,29 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
 
 /**
  * Test copy operation.
  */
-public class ITestAzureBlobFileSystemCopy extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemCopy extends AbstractAbfsIntegrationTest {
   public ITestAzureBlobFileSystemCopy() {
     super();
   }
 
   @Test
   public void testCopyFromLocalFileSystem() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     Path localFilePath = new Path(System.getProperty("test.build.data",
         "azure_test"));
-    FileSystem localFs = FileSystem.get(new Configuration());
+    FileSystem localFs = FileSystem.getLocal(new Configuration());
     localFs.delete(localFilePath, true);
     try {
       writeString(localFs, localFilePath, "Testing");
       Path dstPath = new Path("copiedFromLocal");
       assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false,
           fs.getConf()));
-      assertTrue(fs.exists(dstPath));
+      assertIsFile(fs, dstPath);
       assertEquals("Testing", readString(fs, dstPath));
       fs.delete(dstPath, true);
     } finally {
@@ -65,36 +64,32 @@ public class ITestAzureBlobFileSystemCopy extends DependencyInjectedTest {
   }
 
   private String readString(FileSystem fs, Path testFile) throws IOException {
-    FSDataInputStream inputStream = fs.open(testFile);
-    String ret = readString(inputStream);
-    inputStream.close();
-    return ret;
+    return readString(fs.open(testFile));
   }
 
   private String readString(FSDataInputStream inputStream) throws IOException {
-    BufferedReader reader = new BufferedReader(new InputStreamReader(
-        inputStream));
-    final int bufferSize = 1024;
-    char[] buffer = new char[bufferSize];
-    int count = reader.read(buffer, 0, bufferSize);
-    if (count > bufferSize) {
-      throw new IOException("Exceeded buffer size");
+    try (BufferedReader reader = new BufferedReader(new InputStreamReader(
+        inputStream))) {
+      final int bufferSize = 1024;
+      char[] buffer = new char[bufferSize];
+      int count = reader.read(buffer, 0, bufferSize);
+      if (count > bufferSize) {
+        throw new IOException("Exceeded buffer size");
+      }
+      return new String(buffer, 0, count);
     }
-    inputStream.close();
-    return new String(buffer, 0, count);
   }
 
   private void writeString(FileSystem fs, Path path, String value)
       throws IOException {
-    FSDataOutputStream outputStream = fs.create(path, true);
-    writeString(outputStream, value);
+    writeString(fs.create(path, true), value);
   }
 
   private void writeString(FSDataOutputStream outputStream, String value)
       throws IOException {
-    BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(
-        outputStream));
-    writer.write(value);
-    writer.close();
+    try(BufferedWriter writer = new BufferedWriter(
+        new OutputStreamWriter(outputStream))) {
+      writer.write(value);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
index c9b99e6..1e43f9a 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
@@ -24,18 +24,17 @@ import java.util.EnumSet;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
 
 /**
  * Test create operation.
  */
-public class ITestAzureBlobFileSystemCreate extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemCreate extends
+    AbstractAbfsIntegrationTest {
   private static final Path TEST_FILE_PATH = new Path("testfile");
   private static final Path TEST_FOLDER_PATH = new Path("testFolder");
   private static final String TEST_CHILD_FILE = "childFile";
@@ -43,68 +42,65 @@ public class ITestAzureBlobFileSystemCreate extends DependencyInjectedTest {
     super();
   }
 
-  @Test(expected = FileAlreadyExistsException.class)
-  public void testCreateFileWithExistingDir() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.mkdirs(TEST_FOLDER_PATH);
-    fs.create(TEST_FOLDER_PATH);
-  }
-
   @Test
-  public void testEnsureFileCreated() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.create(TEST_FILE_PATH);
-
-    FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
-    assertNotNull(fileStatus);
+  public void testEnsureFileCreatedImmediately() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    FSDataOutputStream out = fs.create(TEST_FILE_PATH);
+    try {
+      assertIsFile(fs, TEST_FILE_PATH);
+    } finally {
+      out.close();
+    }
+    assertIsFile(fs, TEST_FILE_PATH);
   }
 
   @Test
   @SuppressWarnings("deprecation")
   public void testCreateNonRecursive() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
     try {
       fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null);
-      assertTrue("Should've thrown", false);
-    } catch (FileNotFoundException e) {
+      fail("Should've thrown");
+    } catch (FileNotFoundException expected) {
     }
     fs.mkdirs(TEST_FOLDER_PATH);
     fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
         .close();
-    assertTrue(fs.exists(testFile));
+    assertIsFile(fs, testFile);
   }
 
   @Test
   @SuppressWarnings("deprecation")
   public void testCreateNonRecursive1() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
     try {
       fs.createNonRecursive(testFile, FsPermission.getDefault(), EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1, 1024, null);
-      assertTrue("Should've thrown", false);
-    } catch (FileNotFoundException e) {
+      fail("Should've thrown");
+    } catch (FileNotFoundException expected) {
     }
     fs.mkdirs(TEST_FOLDER_PATH);
     fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
         .close();
-    assertTrue(fs.exists(testFile));
+    assertIsFile(fs, testFile);
+
   }
 
   @Test
   @SuppressWarnings("deprecation")
   public void testCreateNonRecursive2() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
 
     Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
     try {
       fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null);
-      assertTrue("Should've thrown", false);
+      fail("Should've thrown");
     } catch (FileNotFoundException e) {
     }
     fs.mkdirs(TEST_FOLDER_PATH);
     fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
         .close();
-    assertTrue(fs.exists(testFile));
+    assertIsFile(fs, testFile);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
index 372a087..91d1723 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
@@ -28,71 +28,79 @@ import java.util.concurrent.Future;
 
 import org.junit.Test;
 
+import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
-import static org.junit.Assert.assertEquals;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test delete operation.
  */
-public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemDelete extends
+    AbstractAbfsIntegrationTest {
   public ITestAzureBlobFileSystemDelete() {
     super();
   }
 
   @Test
   public void testDeleteRoot() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
 
     fs.mkdirs(new Path("/testFolder0"));
     fs.mkdirs(new Path("/testFolder1"));
     fs.mkdirs(new Path("/testFolder2"));
-    fs.create(new Path("/testFolder1/testfile"));
-    fs.create(new Path("/testFolder1/testfile2"));
-    fs.create(new Path("/testFolder1/testfile3"));
+    touch(new Path("/testFolder1/testfile"));
+    touch(new Path("/testFolder1/testfile2"));
+    touch(new Path("/testFolder1/testfile3"));
 
-    FileStatus[] ls = fs.listStatus(new Path("/"));
-    assertEquals(4, ls.length); // and user dir
+    Path root = new Path("/");
+    FileStatus[] ls = fs.listStatus(root);
+    assertEquals(3, ls.length);
 
-    fs.delete(new Path("/"), true);
-    ls = fs.listStatus(new Path("/"));
-    assertEquals(0, ls.length);
+    fs.delete(root, true);
+    ls = fs.listStatus(root);
+    assertEquals("listing size", 0, ls.length);
   }
 
-  @Test(expected = FileNotFoundException.class)
+  @Test()
   public void testOpenFileAfterDelete() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.create(new Path("/testFile"));
-    fs.delete(new Path("/testFile"), false);
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path testfile = new Path("/testFile");
+    touch(testfile);
+    assertDeleted(fs, testfile, false);
 
-    fs.open(new Path("/testFile"));
+    intercept(FileNotFoundException.class,
+        () -> fs.open(testfile));
   }
 
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testEnsureFileIsDeleted() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.create(new Path("testfile"));
-    fs.delete(new Path("testfile"), false);
-
-    fs.getFileStatus(new Path("testfile"));
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path testfile = new Path("testfile");
+    touch(testfile);
+    assertDeleted(fs, testfile, false);
+    assertPathDoesNotExist(fs, "deleted", testfile);
   }
 
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testDeleteDirectory() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.mkdirs(new Path("testfile"));
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path dir = new Path("testfile");
+    fs.mkdirs(dir);
     fs.mkdirs(new Path("testfile/test1"));
     fs.mkdirs(new Path("testfile/test1/test2"));
 
-    fs.delete(new Path("testfile"), true);
-    fs.getFileStatus(new Path("testfile"));
+    assertDeleted(fs, dir, true);
+    assertPathDoesNotExist(fs, "deleted", dir);
   }
 
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testDeleteFirstLevelDirectory() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final List<Future> tasks = new ArrayList<>();
+    final AzureBlobFileSystem fs = getFileSystem();
+    final List<Future<Void>> tasks = new ArrayList<>();
 
     ExecutorService es = Executors.newFixedThreadPool(10);
     for (int i = 0; i < 1000; i++) {
@@ -100,7 +108,7 @@ public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest {
       Callable<Void> callable = new Callable<Void>() {
         @Override
         public Void call() throws Exception {
-          fs.create(fileName);
+          touch(fileName);
           return null;
         }
       };
@@ -113,7 +121,12 @@ public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest {
     }
 
     es.shutdownNow();
-    fs.delete(new Path("/test"), true);
-    fs.getFileStatus(new Path("/test"));
+    Path dir = new Path("/test");
+    // first try a non-recursive delete, expect failure
+    intercept(FileAlreadyExistsException.class,
+        () -> fs.delete(dir, false));
+    assertDeleted(fs, dir, true);
+    assertPathDoesNotExist(fs, "deleted", dir);
+
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
index ad22f99..057dfa0 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
@@ -38,7 +38,7 @@ import static org.junit.Assert.assertArrayEquals;
 /**
  * Test end to end between ABFS client and ABFS server.
  */
-public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
   private static final Path TEST_FILE = new Path("testfile");
   private static final int TEST_BYTE = 100;
   private static final int TEST_OFFSET = 100;
@@ -53,11 +53,11 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
 
   @Test
   public void testWriteOneByteToFile() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    FSDataOutputStream stream = fs.create(TEST_FILE);
+    final AzureBlobFileSystem fs = getFileSystem();
 
-    stream.write(TEST_BYTE);
-    stream.close();
+    try(FSDataOutputStream stream = fs.create(TEST_FILE)) {
+      stream.write(TEST_BYTE);
+    }
 
     FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
     assertEquals(1, fileStatus.getLen());
@@ -65,52 +65,52 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
 
   @Test
   public void testReadWriteBytesToFile() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     testWriteOneByteToFile();
-    FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);
-    int i = inputStream.read();
-    inputStream.close();
-
-    assertEquals(TEST_BYTE, i);
+    try(FSDataInputStream inputStream = fs.open(TEST_FILE,
+        TEST_DEFAULT_BUFFER_SIZE)) {
+      assertEquals(TEST_BYTE, inputStream.read());
+    }
   }
 
   @Test (expected = IOException.class)
   public void testOOBWrites() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     int readBufferSize = fs.getAbfsStore().getAbfsConfiguration().getReadBufferSize();
 
-    fs.create(TEST_FILE);
-    FSDataOutputStream writeStream = fs.create(TEST_FILE);
-
     byte[] bytesToRead = new byte[readBufferSize];
     final byte[] b = new byte[2 * readBufferSize];
     new Random().nextBytes(b);
 
-    writeStream.write(b);
-    writeStream.flush();
-    writeStream.close();
 
-    FSDataInputStream readStream = fs.open(TEST_FILE);
-    readStream.read(bytesToRead, 0, readBufferSize);
+    try(FSDataOutputStream writeStream = fs.create(TEST_FILE)) {
+      writeStream.write(b);
+      writeStream.flush();
+    }
+
+    try (FSDataInputStream readStream = fs.open(TEST_FILE)) {
+      assertEquals(readBufferSize,
+          readStream.read(bytesToRead, 0, readBufferSize));
 
-    writeStream = fs.create(TEST_FILE);
-    writeStream.write(b);
-    writeStream.flush();
-    writeStream.close();
+      try (FSDataOutputStream writeStream = fs.create(TEST_FILE)) {
+        writeStream.write(b);
+        writeStream.flush();
+      }
 
-    readStream.read(bytesToRead, 0, readBufferSize);
-    readStream.close();
+      assertEquals(readBufferSize,
+          readStream.read(bytesToRead, 0, readBufferSize));
+    }
   }
 
   @Test
   public void testWriteWithBufferOffset() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final FSDataOutputStream stream = fs.create(TEST_FILE);
+    final AzureBlobFileSystem fs = getFileSystem();
 
     final byte[] b = new byte[1024 * 1000];
     new Random().nextBytes(b);
-    stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET);
-    stream.close();
+    try(final FSDataOutputStream stream = fs.create(TEST_FILE)) {
+      stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET);
+    }
 
     final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE];
     FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);
@@ -124,13 +124,11 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
 
   @Test
   public void testReadWriteHeavyBytesToFileWithSmallerChunks() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final FSDataOutputStream stream = fs.create(TEST_FILE);
+    final AzureBlobFileSystem fs = getFileSystem();
 
     final byte[] writeBuffer = new byte[5 * 1000 * 1024];
     new Random().nextBytes(writeBuffer);
-    stream.write(writeBuffer);
-    stream.close();
+    write(TEST_FILE, writeBuffer);
 
     final byte[] readBuffer = new byte[5 * 1000 * 1024];
     FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
index 616253b..04690de 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
@@ -26,7 +26,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -35,28 +34,24 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
 /**
  * Test end to end between ABFS client and ABFS server with heavy traffic.
  */
-public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemE2EScale extends
+    AbstractAbfsScaleTest {
   private static final int TEN = 10;
   private static final int ONE_THOUSAND = 1000;
   private static final int BASE_SIZE = 1024;
   private static final int ONE_MB = 1024 * 1024;
   private static final int DEFAULT_WRITE_TIMES = 100;
-  private static final Path TEST_FILE = new Path("testfile");
+  private static final Path TEST_FILE = new Path("ITestAzureBlobFileSystemE2EScale");
 
   public ITestAzureBlobFileSystemE2EScale() {
-    super();
   }
 
   @Test
-  public void testWriteHeavyBytesToFile() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+  public void testWriteHeavyBytesToFileAcrossThreads() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
     final FSDataOutputStream stream = fs.create(TEST_FILE);
     ExecutorService es = Executors.newFixedThreadPool(TEN);
 
@@ -65,7 +60,8 @@ public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest {
     new Random().nextBytes(b);
     List<Future<Void>> tasks = new ArrayList<>();
 
-    for (int i = 0; i < DEFAULT_WRITE_TIMES; i++) {
+    int operationCount = DEFAULT_WRITE_TIMES;
+    for (int i = 0; i < operationCount; i++) {
       Callable<Void> callable = new Callable<Void>() {
         @Override
         public Void call() throws Exception {
@@ -86,48 +82,38 @@ public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest {
 
     es.shutdownNow();
     FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
-    assertEquals(testWriteBufferSize * DEFAULT_WRITE_TIMES, fileStatus.getLen());
-  }
-
-  @Test
-  public void testReadWriteHeavyBytesToFile() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final FSDataOutputStream stream = fs.create(TEST_FILE);
-
-    int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE;
-    final byte[] b = new byte[testBufferSize];
-    new Random().nextBytes(b);
-    stream.write(b);
-    stream.close();
-
-    final byte[] r = new byte[testBufferSize];
-    FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB);
-    int result = inputStream.read(r);
-    inputStream.close();
-
-    assertNotEquals(-1, result);
-    assertArrayEquals(r, b);
+    assertEquals(testWriteBufferSize * operationCount, fileStatus.getLen());
   }
 
   @Test
   public void testReadWriteHeavyBytesToFileWithStatistics() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final FSDataOutputStream stream = fs.create(TEST_FILE);
-    final FileSystem.Statistics abfsStatistics = fs.getFsStatistics();
-    abfsStatistics.reset();
+    final AzureBlobFileSystem fs = getFileSystem();
+    final FileSystem.Statistics abfsStatistics;
+    int testBufferSize;
+    final byte[] sourceData;
+    try(final FSDataOutputStream stream = fs.create(TEST_FILE)) {
+      abfsStatistics = fs.getFsStatistics();
+      abfsStatistics.reset();
+
+      testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE;
+      sourceData = new byte[testBufferSize];
+      new Random().nextBytes(sourceData);
+      stream.write(sourceData);
+    }
 
-    int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE;
-    final byte[] b = new byte[testBufferSize];
-    new Random().nextBytes(b);
-    stream.write(b);
-    stream.close();
+    final byte[] remoteData = new byte[testBufferSize];
+    int bytesRead;
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB)) {
+      bytesRead = inputStream.read(remoteData);
+    }
 
-    final byte[] r = new byte[testBufferSize];
-    FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB);
-    inputStream.read(r);
-    inputStream.close();
+    String stats = abfsStatistics.toString();
+    assertEquals("Bytes read in " + stats,
+        remoteData.length, abfsStatistics.getBytesRead());
+    assertEquals("bytes written in " + stats,
+        sourceData.length, abfsStatistics.getBytesWritten());
+    assertEquals("bytesRead from read() call", testBufferSize, bytesRead );
+    assertArrayEquals("round tripped data", sourceData, remoteData);
 
-    Assert.assertEquals(r.length, abfsStatistics.getBytesRead());
-    Assert.assertEquals(b.length, abfsStatistics.getBytesWritten());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
index bfa662d..791694b 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.fs.azurebfs;
 
+import java.io.IOException;
+
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FileStatus;
@@ -25,12 +27,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 
-import static org.junit.Assert.assertEquals;
-
 /**
  * Test FileStatus.
  */
-public class ITestAzureBlobFileSystemFileStatus extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemFileStatus extends
+    AbstractAbfsIntegrationTest {
   private static final Path TEST_FILE = new Path("testFile");
   private static final Path TEST_FOLDER = new Path("testDir");
   public ITestAzureBlobFileSystemFileStatus() {
@@ -41,24 +42,38 @@ public class ITestAzureBlobFileSystemFileStatus extends DependencyInjectedTest {
   public void testEnsureStatusWorksForRoot() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
 
-    fs.getFileStatus(new Path("/"));
-    fs.listStatus(new Path("/"));
+    Path root = new Path("/");
+    FileStatus[] rootls = fs.listStatus(root);
+    assertEquals("root listing", 0, rootls.length);
   }
 
   @Test
   public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.create(TEST_FILE);
-    fs.mkdirs(TEST_FOLDER);
+    touch(TEST_FILE);
+    validateStatus(fs, TEST_FILE);
+  }
+
+  private FileStatus validateStatus(final AzureBlobFileSystem fs, final Path name)
+      throws IOException {
+    FileStatus fileStatus = fs.getFileStatus(name);
+    String errorInStatus = "error in " + fileStatus + " from " + fs;
+    assertEquals(errorInStatus + ": permission",
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL),
+        fileStatus.getPermission());
+    assertEquals(errorInStatus + ": owner",
+        fs.getOwnerUser(), fileStatus.getOwner());
+    assertEquals(errorInStatus + ": group",
+        fs.getOwnerUserPrimaryGroup(), fileStatus.getGroup());
+    return fileStatus;
+  }
 
-    FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
-    assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission());
-    assertEquals(fs.getOwnerUser(), fileStatus.getGroup());
-    assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner());
+  @Test
+  public void testFolderStatusPermissionsAndOwnerAndGroup() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(TEST_FOLDER);
 
-    fileStatus = fs.getFileStatus(TEST_FOLDER);
-    assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission());
-    assertEquals(fs.getOwnerUser(), fileStatus.getGroup());
-    assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner());
+    validateStatus(fs, TEST_FOLDER);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
index 8c2e8ce..d90f018 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
@@ -34,14 +34,10 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
 /**
  * Test flush operation.
  */
-public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
   private static final int BASE_SIZE = 1024;
   private static final int ONE_THOUSAND = 1000;
   private static final int TEST_BUFFER_SIZE = 5 * ONE_THOUSAND * BASE_SIZE;
@@ -56,146 +52,145 @@ public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest {
   }
 
   @Test
-  public void testAbfsOutputStreamAsyncFlushWithRetainUncommitedData() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
-
-    final byte[] b = new byte[TEST_BUFFER_SIZE];
-    new Random().nextBytes(b);
+  public void testAbfsOutputStreamAsyncFlushWithRetainUncommittedData() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final byte[] b;
+    try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
+      b = new byte[TEST_BUFFER_SIZE];
+      new Random().nextBytes(b);
 
-    for (int i = 0; i < 2; i++) {
-      stream.write(b);
+      for (int i = 0; i < 2; i++) {
+        stream.write(b);
 
-      for (int j = 0; j < FLUSH_TIMES; j++) {
-        stream.flush();
-        Thread.sleep(10);
+        for (int j = 0; j < FLUSH_TIMES; j++) {
+          stream.flush();
+          Thread.sleep(10);
+        }
       }
     }
 
-    stream.close();
-
     final byte[] r = new byte[TEST_BUFFER_SIZE];
-    FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB);
+    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) {
+      while (inputStream.available() != 0) {
+        int result = inputStream.read(r);
 
-    while (inputStream.available() != 0) {
-      int result = inputStream.read(r);
-
-      assertNotEquals(-1, result);
-      assertArrayEquals(r, b);
+        assertNotEquals("read returned -1", -1, result);
+        assertArrayEquals("buffer read from stream", r, b);
+      }
     }
-
-    inputStream.close();
   }
 
   @Test
   public void testAbfsOutputStreamSyncFlush() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
-
-    final byte[] b = new byte[TEST_BUFFER_SIZE];
-    new Random().nextBytes(b);
-    stream.write(b);
+    final AzureBlobFileSystem fs = getFileSystem();
+    final byte[] b;
+    try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
+      b = new byte[TEST_BUFFER_SIZE];
+      new Random().nextBytes(b);
+      stream.write(b);
 
-    for (int i = 0; i < FLUSH_TIMES; i++) {
-      stream.hsync();
-      stream.hflush();
-      Thread.sleep(10);
+      for (int i = 0; i < FLUSH_TIMES; i++) {
+        stream.hsync();
+        stream.hflush();
+        Thread.sleep(10);
+      }
     }
-    stream.close();
 
     final byte[] r = new byte[TEST_BUFFER_SIZE];
-    FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB);
-    int result = inputStream.read(r);
-
-    assertNotEquals(-1, result);
-    assertArrayEquals(r, b);
+    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) {
+      int result = inputStream.read(r);
 
-    inputStream.close();
+      assertNotEquals(-1, result);
+      assertArrayEquals(r, b);
+    }
   }
 
 
   @Test
   public void testWriteHeavyBytesToFileSyncFlush() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
-    final FileSystem.Statistics abfsStatistics = fs.getFsStatistics();
-    abfsStatistics.reset();
-
-    ExecutorService es = Executors.newFixedThreadPool(10);
-
-    final byte[] b = new byte[TEST_BUFFER_SIZE];
-    new Random().nextBytes(b);
-
-    List<Future<Void>> tasks = new ArrayList<>();
-    for (int i = 0; i < FLUSH_TIMES; i++) {
-      Callable<Void> callable = new Callable<Void>() {
-        @Override
-        public Void call() throws Exception {
-          stream.write(b);
-          return null;
-        }
-      };
-
-      tasks.add(es.submit(callable));
-    }
+    final AzureBlobFileSystem fs = getFileSystem();
+    final FileSystem.Statistics abfsStatistics;
+    ExecutorService es;
+    try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
+      abfsStatistics = fs.getFsStatistics();
+      abfsStatistics.reset();
+
+      es = Executors.newFixedThreadPool(10);
+
+      final byte[] b = new byte[TEST_BUFFER_SIZE];
+      new Random().nextBytes(b);
+
+      List<Future<Void>> tasks = new ArrayList<>();
+      for (int i = 0; i < FLUSH_TIMES; i++) {
+        Callable<Void> callable = new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            stream.write(b);
+            return null;
+          }
+        };
+
+        tasks.add(es.submit(callable));
+      }
 
-    boolean shouldStop = false;
-    while (!shouldStop) {
-      shouldStop = true;
-      for (Future<Void> task : tasks) {
-        if (!task.isDone()) {
-          stream.hsync();
-          shouldStop = false;
-          Thread.sleep(THREAD_SLEEP_TIME);
+      boolean shouldStop = false;
+      while (!shouldStop) {
+        shouldStop = true;
+        for (Future<Void> task : tasks) {
+          if (!task.isDone()) {
+            stream.hsync();
+            shouldStop = false;
+            Thread.sleep(THREAD_SLEEP_TIME);
+          }
         }
       }
-    }
 
-    tasks.clear();
-    stream.close();
+      tasks.clear();
+    }
 
     es.shutdownNow();
     FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
-    assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen());
-    assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, abfsStatistics.getBytesWritten());
+    long expectedWrites = (long) TEST_BUFFER_SIZE * FLUSH_TIMES;
+    assertEquals("Wrong file length in " + fileStatus, expectedWrites, fileStatus.getLen());
+    assertEquals("wrong bytes Written count in " + abfsStatistics,
+        expectedWrites, abfsStatistics.getBytesWritten());
   }
 
   @Test
   public void testWriteHeavyBytesToFileAsyncFlush() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.create(TEST_FILE_PATH);
-    final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
+    final AzureBlobFileSystem fs = getFileSystem();
     ExecutorService es = Executors.newFixedThreadPool(10);
+    try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
+
+      final byte[] b = new byte[TEST_BUFFER_SIZE];
+      new Random().nextBytes(b);
+
+      List<Future<Void>> tasks = new ArrayList<>();
+      for (int i = 0; i < FLUSH_TIMES; i++) {
+        Callable<Void> callable = new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            stream.write(b);
+            return null;
+          }
+        };
+
+        tasks.add(es.submit(callable));
+      }
 
-    final byte[] b = new byte[TEST_BUFFER_SIZE];
-    new Random().nextBytes(b);
-
-    List<Future<Void>> tasks = new ArrayList<>();
-    for (int i = 0; i < FLUSH_TIMES; i++) {
-      Callable<Void> callable = new Callable<Void>() {
-        @Override
-        public Void call() throws Exception {
-          stream.write(b);
-          return null;
-        }
-      };
-
-      tasks.add(es.submit(callable));
-    }
-
-    boolean shouldStop = false;
-    while (!shouldStop) {
-      shouldStop = true;
-      for (Future<Void> task : tasks) {
-        if (!task.isDone()) {
-          stream.flush();
-          shouldStop = false;
+      boolean shouldStop = false;
+      while (!shouldStop) {
+        shouldStop = true;
+        for (Future<Void> task : tasks) {
+          if (!task.isDone()) {
+            stream.flush();
+            shouldStop = false;
+          }
         }
       }
+      Thread.sleep(THREAD_SLEEP_TIME);
+      tasks.clear();
     }
-    Thread.sleep(THREAD_SLEEP_TIME);
-    tasks.clear();
-    stream.close();
 
     es.shutdownNow();
     FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
index d2ed400..5a6e46d 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
@@ -22,29 +22,32 @@ import java.io.FileNotFoundException;
 
 import org.junit.Test;
 
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 
 /**
  * Test filesystem initialization and creation.
  */
-public class ITestAzureBlobFileSystemInitAndCreate extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemInitAndCreate extends
+    AbstractAbfsIntegrationTest {
   public ITestAzureBlobFileSystemInitAndCreate() {
-    super();
 
     this.getConfiguration().unset(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION);
   }
 
   @Override
-  public void initialize() {
+  public void setup() {
   }
 
   @Override
-  public void testCleanup() {
+  public void teardown() {
   }
 
   @Test (expected = FileNotFoundException.class)
   public void ensureFilesystemWillNotBeCreatedIfCreationConfigIsNotSet() throws Exception {
-    super.initialize();
-    this.getFileSystem();
+    super.setup();
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    FileStatus[] fileStatuses = fs.listStatus(new Path("/"));
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java
index 6059766..b87abe6 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java
@@ -26,20 +26,21 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test listStatus operation.
  */
-public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemListStatus extends
+    AbstractAbfsIntegrationTest {
   private static final int TEST_FILES_NUMBER = 6000;
   public ITestAzureBlobFileSystemListStatus() {
     super();
@@ -47,8 +48,8 @@ public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest {
 
   @Test
   public void testListPath() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final List<Future> tasks = new ArrayList<>();
+    final AzureBlobFileSystem fs = getFileSystem();
+    final List<Future<Void>> tasks = new ArrayList<>();
 
     ExecutorService es = Executors.newFixedThreadPool(10);
     for (int i = 0; i < TEST_FILES_NUMBER; i++) {
@@ -56,7 +57,7 @@ public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest {
       Callable<Void> callable = new Callable<Void>() {
         @Override
         public Void call() throws Exception {
-          fs.create(fileName);
+          touch(fileName);
           return null;
         }
       };
@@ -70,63 +71,101 @@ public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest {
 
     es.shutdownNow();
     FileStatus[] files = fs.listStatus(new Path("/"));
-    Assert.assertEquals(files.length, TEST_FILES_NUMBER + 1 /* user directory */);
+    assertEquals(TEST_FILES_NUMBER, files.length /* user directory */);
   }
 
+  /**
+   * Creates a file, verifies that listStatus returns it,
+   * even while the file is still open for writing.
+   */
   @Test
   public void testListFileVsListDir() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.create(new Path("/testFile"));
-
-    FileStatus[] testFiles = fs.listStatus(new Path("/testFile"));
-    Assert.assertEquals(testFiles.length, 1);
-    Assert.assertFalse(testFiles[0].isDirectory());
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path path = new Path("/testFile");
+    try(FSDataOutputStream ignored = fs.create(path)) {
+      FileStatus[] testFiles = fs.listStatus(path);
+      assertEquals("length of test files", 1, testFiles.length);
+      FileStatus status = testFiles[0];
+      assertIsFileReference(status);
+    }
   }
 
   @Test
   public void testListFileVsListDir2() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     fs.mkdirs(new Path("/testFolder"));
     fs.mkdirs(new Path("/testFolder/testFolder2"));
     fs.mkdirs(new Path("/testFolder/testFolder2/testFolder3"));
-    fs.create(new Path("/testFolder/testFolder2/testFolder3/testFile"));
-
-    FileStatus[] testFiles = fs.listStatus(new Path("/testFolder/testFolder2/testFolder3/testFile"));
-    Assert.assertEquals(testFiles.length, 1);
-    Assert.assertEquals(testFiles[0].getPath(), new Path(this.getTestUrl(),
-        "/testFolder/testFolder2/testFolder3/testFile"));
-    Assert.assertFalse(testFiles[0].isDirectory());
+    Path testFile0Path = new Path("/testFolder/testFolder2/testFolder3/testFile");
+    ContractTestUtils.touch(fs, testFile0Path);
+
+    FileStatus[] testFiles = fs.listStatus(testFile0Path);
+    assertEquals("Wrong listing size of file " + testFile0Path,
+        1, testFiles.length);
+    FileStatus file0 = testFiles[0];
+    assertEquals("Wrong path for " + file0,
+        new Path(getTestUrl(), "/testFolder/testFolder2/testFolder3/testFile"),
+        file0.getPath());
+    assertIsFileReference(file0);
   }
 
   @Test(expected = FileNotFoundException.class)
   public void testListNonExistentDir() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     fs.listStatus(new Path("/testFile/"));
   }
 
   @Test
   public void testListFiles() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.mkdirs(new Path("/test"));
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path testDir = new Path("/test");
+    fs.mkdirs(testDir);
 
     FileStatus[] fileStatuses = fs.listStatus(new Path("/"));
-    assertEquals(fileStatuses.length, 2);
+    assertEquals(1, fileStatuses.length);
 
     fs.mkdirs(new Path("/test/sub"));
-    fileStatuses = fs.listStatus(new Path("/test"));
-    assertEquals(fileStatuses.length, 1);
-    assertEquals(fileStatuses[0].getPath().getName(), "sub");
-    assertTrue(fileStatuses[0].isDirectory());
-    assertEquals(fileStatuses[0].getLen(), 0);
-
-    fs.create(new Path("/test/f"));
-    fileStatuses = fs.listStatus(new Path("/test"));
-    assertEquals(fileStatuses.length, 2);
-    assertEquals(fileStatuses[0].getPath().getName(), "f");
-    assertFalse(fileStatuses[0].isDirectory());
-    assertEquals(fileStatuses[0].getLen(), 0);
-    assertEquals(fileStatuses[1].getPath().getName(), "sub");
-    assertTrue(fileStatuses[1].isDirectory());
-    assertEquals(fileStatuses[1].getLen(), 0);
+    fileStatuses = fs.listStatus(testDir);
+    assertEquals(1, fileStatuses.length);
+    assertEquals("sub", fileStatuses[0].getPath().getName());
+    assertIsDirectoryReference(fileStatuses[0]);
+    Path childF = fs.makeQualified(new Path("/test/f"));
+    touch(childF);
+    fileStatuses = fs.listStatus(testDir);
+    assertEquals(2, fileStatuses.length);
+    final FileStatus childStatus = fileStatuses[0];
+    assertEquals(childF, childStatus.getPath());
+    assertEquals("f", childStatus.getPath().getName());
+    assertIsFileReference(childStatus);
+    assertEquals(0, childStatus.getLen());
+    final FileStatus status1 = fileStatuses[1];
+    assertEquals("sub", status1.getPath().getName());
+    assertIsDirectoryReference(status1);
+    // look at the child through getFileStatus
+    LocatedFileStatus locatedChildStatus = fs.listFiles(childF, false).next();
+    assertIsFileReference(locatedChildStatus);
+
+    fs.delete(testDir, true);
+    intercept(FileNotFoundException.class,
+        () -> fs.listFiles(childF, false).next());
+
+    // do some final checks on the status (failing due to version checks)
+    assertEquals("Path mismatch of " + locatedChildStatus,
+        childF, locatedChildStatus.getPath());
+    assertEquals("locatedstatus.equals(status)",
+        locatedChildStatus, childStatus);
+    assertEquals("status.equals(locatedstatus)",
+        childStatus, locatedChildStatus);
+  }
+
+  private void assertIsDirectoryReference(FileStatus status) {
+    assertTrue("Not a directory: " + status, status.isDirectory());
+    assertFalse("Not a directory: " + status, status.isFile());
+    assertEquals(0, status.getLen());
+  }
+
+  private void assertIsFileReference(FileStatus status) {
+    assertFalse("Not a file: " + status, status.isDirectory());
+    assertTrue("Not a file: " + status, status.isFile());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java
index b61908c..1bb2c54 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java
@@ -18,71 +18,30 @@
 
 package org.apache.hadoop.fs.azurebfs;
 
-import java.util.concurrent.Callable;
-
 import org.junit.Test;
 
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.Path;
 
-import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertTrue;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
 
 /**
  * Test mkdir operation.
  */
-public class ITestAzureBlobFileSystemMkDir extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest {
   public ITestAzureBlobFileSystemMkDir() {
     super();
   }
 
   @Test
   public void testCreateDirWithExistingDir() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    assertTrue(fs.mkdirs(new Path("testFolder")));
-    assertTrue(fs.mkdirs(new Path("testFolder")));
-  }
-
-  @Test(expected = FileAlreadyExistsException.class)
-  public void createDirectoryUnderFile() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.create(new Path("testFile"));
-    fs.mkdirs(new Path("testFile/TestDirectory"));
-  }
-
-  @Test
-  public void testCreateDirectoryOverExistingFiles() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.create(new Path("/testPath"));
-    FileAlreadyExistsException ex = intercept(
-        FileAlreadyExistsException.class,
-        new Callable<Void>() {
-          @Override
-          public Void call() throws Exception {
-            fs.mkdirs(new Path("/testPath"));
-            return null;
-          }
-        });
-
-    assertTrue(ex instanceof FileAlreadyExistsException);
-
-    fs.create(new Path("/testPath1/file1"));
-    ex = intercept(
-        FileAlreadyExistsException.class,
-        new Callable<Void>() {
-          @Override
-          public Void call() throws Exception {
-            fs.mkdirs(new Path("/testPath1/file1"));
-            return null;
-          }
-        });
-
-    assertTrue(ex instanceof FileAlreadyExistsException);
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path path = new Path("testFolder");
+    assertMkdirs(fs, path);
+    assertMkdirs(fs, path);
   }
 
   @Test
   public void testCreateRoot() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    assertTrue(fs.mkdirs(new Path("/")));
+    assertMkdirs(getFileSystem(), new Path("/"));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java
deleted file mode 100644
index fef7f47..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java
+++ /dev/null
@@ -1,41 +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.fs.azurebfs;
-
-import java.io.FileNotFoundException;
-
-import org.junit.Test;
-
-import org.apache.hadoop.fs.Path;
-
-/**
- * Test open operation.
- */
-public class ITestAzureBlobFileSystemOpen extends DependencyInjectedTest {
-  public ITestAzureBlobFileSystemOpen() throws Exception {
-    super();
-  }
-
-  @Test(expected = FileNotFoundException.class)
-  public void testOpenDirectory() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.mkdirs(new Path("testFolder"));
-    fs.open(new Path("testFolder"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
index 8b96c69..c61de67 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
@@ -18,32 +18,31 @@
 package org.apache.hadoop.fs.azurebfs;
 
 
-import org.apache.hadoop.fs.Path;
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.Callable;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Test;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.Random;
-import java.util.concurrent.Callable;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertArrayEquals;
-
 
 /**
  * Test random read operation.
  */
-public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemRandomRead extends
+    AbstractAbfsScaleTest {
   private static final int KILOBYTE = 1024;
   private static final int MEGABYTE = KILOBYTE * KILOBYTE;
   private static final long TEST_FILE_SIZE = 8 * MEGABYTE;
@@ -62,6 +61,9 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
   private static final String ABFS = "ABFS";
   private static long testFileLength = 0;
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestAzureBlobFileSystemRandomRead.class);
+
   public ITestAzureBlobFileSystemRandomRead() throws Exception {
     super();
   }
@@ -76,7 +78,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
       // forward seek and read a kilobyte into first kilobyte of bufferV2
       inputStream.seek(5 * MEGABYTE);
       int numBytesRead = inputStream.read(buffer, 0, KILOBYTE);
-      assertEquals(KILOBYTE, numBytesRead);
+      assertEquals("Wrong number of bytes read", KILOBYTE, numBytesRead);
 
       int len = MEGABYTE;
       int offset = buffer.length - len;
@@ -84,7 +86,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
       // reverse seek and read a megabyte into last megabyte of bufferV1
       inputStream.seek(3 * MEGABYTE);
       numBytesRead = inputStream.read(buffer, offset, len);
-      assertEquals(len, numBytesRead);
+      assertEquals("Wrong number of bytes read after seek", len, numBytesRead);
     }
   }
 
@@ -391,7 +393,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
       afterSeekElapsedMs = sequentialRead(ABFS,
               this.getFileSystem(), true);
       ratio = afterSeekElapsedMs / beforeSeekElapsedMs;
-      System.out.println((String.format(
+      LOG.info((String.format(
               "beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d, ratio=%3$.2f",
               (long) beforeSeekElapsedMs,
               (long) afterSeekElapsedMs,
@@ -425,7 +427,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
 
       ratio = v2ElapsedMs / v1ElapsedMs;
 
-      System.out.println(String.format(
+      LOG.info(String.format(
               "v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f",
               (long) v1ElapsedMs,
               (long) v2ElapsedMs,
@@ -464,7 +466,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
       }
       long elapsedTimeMs = timer.elapsedTimeMs();
 
-      System.out.println(String.format(
+      LOG.info(String.format(
               "v%1$s: bytesRead=%2$d, elapsedMs=%3$d, Mbps=%4$.2f,"
                       + " afterReverseSeek=%5$s",
               version,
@@ -496,7 +498,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
         } while (bytesRead > 0 && totalBytesRead < minBytesToRead);
       long elapsedTimeMs = timer.elapsedTimeMs();
       inputStream.close();
-      System.out.println(String.format(
+      LOG.info(String.format(
               "v%1$d: totalBytesRead=%2$d, elapsedTimeMs=%3$d, Mbps=%4$.2f",
               version,
               totalBytesRead,
@@ -535,7 +537,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
       character = (character == 'z') ? 'a' : (char) ((int) character + 1);
     }
 
-    System.out.println(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE));
+    LOG.info(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE));
     ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
 
     try (FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
@@ -544,7 +546,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
         outputStream.write(buffer);
         bytesWritten += buffer.length;
       }
-      System.out.println(String.format("Closing stream %s", outputStream));
+      LOG.info("Closing stream {}", outputStream);
       ContractTestUtils.NanoTimer closeTimer
               = new ContractTestUtils.NanoTimer();
       outputStream.close();
@@ -578,4 +580,4 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
     assertArrayEquals("Mismatch in read data", bufferV1, bufferV2);
   }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
index a0e648c..1a0edaf 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.fs.azurebfs;
 
-import java.io.FileNotFoundException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Callable;
@@ -26,93 +25,74 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
 
 /**
  * Test rename operation.
  */
-public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest {
+public class ITestAzureBlobFileSystemRename extends
+    AbstractAbfsIntegrationTest {
   public ITestAzureBlobFileSystemRename() {
-    super();
   }
 
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testEnsureFileIsRenamed() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.create(new Path("testfile"));
-    fs.rename(new Path("testfile"), new Path("testfile2"));
-
-    FileStatus fileStatus = fs.getFileStatus(new Path("testfile2"));
-    assertNotNull(fileStatus);
-
-    fs.getFileStatus(new Path("testfile"));
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path src = path("testEnsureFileIsRenamed-src");
+    touch(src);
+    Path dest = path("testEnsureFileIsRenamed-dest");
+    fs.delete(dest, true);
+    assertRenameOutcome(fs, src, dest, true);
+
+    assertIsFile(fs, dest);
+    assertPathDoesNotExist(fs, "expected renamed", src);
   }
 
   @Test
-  public void testRenameFile() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.mkdirs(new Path("/testSrc"));
-    fs.create(new Path("/testSrc/file1"));
-
-    fs.rename(new Path("/testSrc"), new Path("/testDst"));
-    FileStatus[] fileStatus = fs.listStatus(new Path("/testDst"));
-    assertNotNull(fileStatus);
+  public void testRenameFileUnderDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path sourceDir = new Path("/testSrc");
+    assertMkdirs(fs, sourceDir);
+    String filename = "file1";
+    Path file1 = new Path(sourceDir, filename);
+    touch(file1);
+
+    Path destDir = new Path("/testDst");
+    assertRenameOutcome(fs, sourceDir, destDir, true);
+    FileStatus[] fileStatus = fs.listStatus(destDir);
+    assertNotNull("Null file status", fileStatus);
+    FileStatus status = fileStatus[0];
+    assertEquals("Wrong filename in " + status,
+        filename, status.getPath().getName());
   }
 
   @Test
-  public void testRenameFileUsingUnicode() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    //known issue: ListStatus operation to folders/files whose name contains '?' will fail
-    //This is because Auto rest client didn't encode '?' in the uri query parameters
-    String[] folders1 = new String[]{"/%2c%26", "/ÖáΠ⇒", "/A +B", "/A~`!@#$%^&*()-_+={};:'>,<B"};
-    String[] folders2 = new String[]{"/abcÖ⇒123", "/abcÖáΠ⇒123", "/B+ C", "/B~`!@#$%^&*()-_+={};:'>,<C"};
-    String[] files = new String[]{"/%2c%27", "/中文", "/C +D", "/C~`!@#$%^&*()-_+={};:'>,<D"};
-
-    for (int i = 0; i < 4; i++) {
-      Path folderPath1 = new Path(folders1[i]);
-      assertTrue(fs.mkdirs(folderPath1));
-      assertTrue(fs.exists(folderPath1));
-
-      Path filePath = new Path(folders1[i] + files[i]);
-      fs.create(filePath);
-      assertTrue(fs.exists(filePath));
-
-      Path folderPath2 = new Path(folders2[i]);
-      fs.rename(folderPath1, folderPath2);
-      assertFalse(fs.exists(folderPath1));
-      assertTrue(fs.exists(folderPath2));
-
-      FileStatus[] fileStatus = fs.listStatus(folderPath2);
-      assertEquals("/" + fileStatus[0].getPath().getName(), files[i]);
-      assertNotNull(fileStatus);
-    }
-  }
-
-  @Test(expected = FileNotFoundException.class)
   public void testRenameDirectory() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     fs.mkdirs(new Path("testDir"));
-    fs.mkdirs(new Path("testDir/test1"));
+    Path test1 = new Path("testDir/test1");
+    fs.mkdirs(test1);
     fs.mkdirs(new Path("testDir/test1/test2"));
     fs.mkdirs(new Path("testDir/test1/test2/test3"));
 
-    Assert.assertTrue(fs.rename(new Path("testDir/test1"), new Path("testDir/test10")));
-    fs.getFileStatus(new Path("testDir/test1"));
+    assertRenameOutcome(fs, test1,
+        new Path("testDir/test10"), true);
+    assertPathDoesNotExist(fs, "rename source dir", test1 );
   }
 
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testRenameFirstLevelDirectory() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final List<Future> tasks = new ArrayList<>();
+    final AzureBlobFileSystem fs = getFileSystem();
+    final List<Future<Void>> tasks = new ArrayList<>();
 
     ExecutorService es = Executors.newFixedThreadPool(10);
     for (int i = 0; i < 1000; i++) {
@@ -120,7 +100,7 @@ public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest {
       Callable<Void> callable = new Callable<Void>() {
         @Override
         public Void call() throws Exception {
-          fs.create(fileName);
+          touch(fileName);
           return null;
         }
       };
@@ -133,20 +113,25 @@ public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest {
     }
 
     es.shutdownNow();
-    fs.rename(new Path("/test"), new Path("/renamedDir"));
+    Path source = new Path("/test");
+    Path dest = new Path("/renamedDir");
+    assertRenameOutcome(fs, source, dest, true);
 
-    FileStatus[] files = fs.listStatus(new Path("/renamedDir"));
-    Assert.assertEquals(files.length, 1000);
-    fs.getFileStatus(new Path("/test"));
+    FileStatus[] files = fs.listStatus(dest);
+    assertEquals("Wrong number of files in listing", 1000, files.length);
+    assertPathDoesNotExist(fs, "rename source dir", source);
   }
 
   @Test
   public void testRenameRoot() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    boolean renamed = fs.rename(new Path("/"), new Path("/ddd"));
-    assertFalse(renamed);
-
-    renamed = fs.rename(new Path(fs.getUri().toString() + "/"), new Path(fs.getUri().toString() + "/s"));
-    assertFalse(renamed);
+    final AzureBlobFileSystem fs = getFileSystem();
+    assertRenameOutcome(fs,
+        new Path("/"),
+        new Path("/testRenameRoot"),
+        false);
+    assertRenameOutcome(fs,
+        new Path(fs.getUri().toString() + "/"),
+        new Path(fs.getUri().toString() + "/s"),
+        false);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java
new file mode 100644
index 0000000..0ac7fcf
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java
@@ -0,0 +1,98 @@
+/*
+ * 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.azurebfs;
+
+import java.util.Arrays;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
+
+/**
+ * Parameterized test of rename operations of unicode paths.
+ */
+@RunWith(Parameterized.class)
+public class ITestAzureBlobFileSystemRenameUnicode extends
+    AbstractAbfsIntegrationTest {
+
+  @Parameterized.Parameter
+  public String srcDir;
+
+  @Parameterized.Parameter(1)
+  public String destDir;
+
+  @Parameterized.Parameter(2)
+  public String filename;
+
+  @Parameterized.Parameters
+  public static Iterable<Object[]> params() {
+    return Arrays.asList(
+        new Object[][]{
+            {"/src", "/dest", "filename"},
+            {"/%2c%26", "/abcÖ⇒123", "%2c%27"},
+            {"/ÖáΠ⇒", "/abcÖáΠ⇒123", "中文"},
+            {"/A +B", "/B+ C", "C +D"},
+            {
+                "/A~`!@#$%^&*()-_+={};:'>,<B",
+                "/B~`!@#$%^&*()-_+={};:'>,<C",
+                "C~`!@#$%^&*()-_+={};:'>,<D"
+            }
+        });
+  }
+
+  public ITestAzureBlobFileSystemRenameUnicode() {
+  }
+
+  /**
+   * Known issue: ListStatus operation to folders/files whose name contains '?' will fail.
+   * This is because Auto rest client didn't encode '?' in the uri query parameters
+   */
+  @Test
+  public void testRenameFileUsingUnicode() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path folderPath1 = new Path(srcDir);
+    assertMkdirs(fs, folderPath1);
+    assertIsDirectory(fs, folderPath1);
+    Path filePath = new Path(folderPath1 + "/" + filename);
+    touch(filePath);
+    assertIsFile(fs, filePath);
+
+    Path folderPath2 = new Path(destDir);
+    assertRenameOutcome(fs, folderPath1, folderPath2, true);
+    assertPathDoesNotExist(fs, "renamed", folderPath1);
+    assertIsDirectory(fs, folderPath2);
+    assertPathExists(fs, "renamed file", new Path(folderPath2 + "/" + filename));
+
+    FileStatus[] fileStatus = fs.listStatus(folderPath2);
+    assertNotNull(fileStatus);
+    assertTrue("Empty listing returned from listStatus(\"" + folderPath2 + "\")",
+        fileStatus.length > 0);
+    assertEquals(fileStatus[0].getPath().getName(), filename);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
index 29af1b8..3a44909 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
@@ -20,9 +20,9 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.net.URI;
 
-import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
@@ -30,30 +30,45 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 /**
  * Test AzureBlobFileSystem initialization.
  */
-public class ITestFileSystemInitialization extends DependencyInjectedTest {
+public class ITestFileSystemInitialization extends AbstractAbfsIntegrationTest {
   public ITestFileSystemInitialization() {
     super();
   }
 
   @Test
   public void ensureAzureBlobFileSystemIsInitialized() throws Exception {
-    final FileSystem fs = this.getFileSystem();
-    final String accountName = this.getAccountName();
-    final String filesystem = this.getFileSystemName();
+    final AzureBlobFileSystem fs = getFileSystem();
+    final String accountName = getAccountName();
+    final String filesystem = getFileSystemName();
 
-    Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SCHEME, filesystem + "@" + accountName, null, null, null));
-    Assert.assertNotNull(fs.getWorkingDirectory());
+    assertEquals(fs.getUri(),
+        new URI(FileSystemUriSchemes.ABFS_SCHEME,
+            filesystem + "@" + accountName,
+            null,
+            null,
+            null));
+    assertNotNull("working directory", fs.getWorkingDirectory());
   }
 
   @Test
   public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception {
-    final String accountName = this.getAccountName();
-    final String filesystem = this.getFileSystemName();
-    final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null);
-    this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
-
-    final FileSystem fs = this.getFileSystem();
-    Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null));
-    Assert.assertNotNull(fs.getWorkingDirectory());
+    final String accountName = getAccountName();
+    final String filesystem = getFileSystemName();
+    final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME,
+        filesystem + "@" + accountName,
+        null,
+        null,
+        null);
+    Configuration conf = getConfiguration();
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
+
+    try(SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.newInstance(conf)) {
+      assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME,
+          filesystem + "@" + accountName,
+          null,
+          null,
+          null));
+      assertNotNull("working directory", fs.getWorkingDirectory());
+    }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
index 62d967e..1c71125 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.util.Hashtable;
 
-import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -34,31 +33,29 @@ import static org.junit.Assert.assertEquals;
 /**
  * Test FileSystemProperties.
  */
-public class ITestFileSystemProperties extends DependencyInjectedTest {
+public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest {
   private static final int TEST_DATA = 100;
   private static final Path TEST_PATH = new Path("/testfile");
   public ITestFileSystemProperties() {
-    super();
   }
 
   @Test
   public void testReadWriteBytesToFileAndEnsureThreadPoolCleanup() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     testWriteOneByteToFileAndEnsureThreadPoolCleanup();
 
-    FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024);
-    int i = inputStream.read();
-
-    assertEquals(TEST_DATA, i);
+    try(FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024)) {
+      int i = inputStream.read();
+      assertEquals(TEST_DATA, i);
+    }
   }
 
   @Test
   public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    FSDataOutputStream stream = fs.create(TEST_PATH);
-
-    stream.write(TEST_DATA);
-    stream.close();
+    final AzureBlobFileSystem fs = getFileSystem();
+    try(FSDataOutputStream stream = fs.create(TEST_PATH)) {
+      stream.write(TEST_DATA);
+    }
 
     FileStatus fileStatus = fs.getFileStatus(TEST_PATH);
     assertEquals(1, fileStatus.getLen());
@@ -67,60 +64,60 @@ public class ITestFileSystemProperties extends DependencyInjectedTest {
   @Test
   @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant")
   public void testBase64FileSystemProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
 
     final Hashtable<String, String> properties = new Hashtable<>();
     properties.put("key", "{ value: value }");
     fs.getAbfsStore().setFilesystemProperties(properties);
     Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
 
-    Assert.assertEquals(properties, fetchedProperties);
+    assertEquals(properties, fetchedProperties);
   }
 
   @Test
   public void testBase64PathProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     final Hashtable<String, String> properties = new Hashtable<>();
     properties.put("key", "{ value: valueTest }");
-    fs.create(TEST_PATH);
+    touch(TEST_PATH);
     fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
     Hashtable<String, String> fetchedProperties =
             fs.getAbfsStore().getPathProperties(TEST_PATH);
 
-    Assert.assertEquals(properties, fetchedProperties);
+    assertEquals(properties, fetchedProperties);
   }
 
   @Test (expected = Exception.class)
   public void testBase64InvalidFileSystemProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     final Hashtable<String, String> properties = new Hashtable<>();
     properties.put("key", "{ value: value歲 }");
     fs.getAbfsStore().setFilesystemProperties(properties);
     Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
 
-    Assert.assertEquals(properties, fetchedProperties);
+    assertEquals(properties, fetchedProperties);
   }
 
   @Test (expected = Exception.class)
   public void testBase64InvalidPathProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     final Hashtable<String, String> properties = new Hashtable<>();
     properties.put("key", "{ value: valueTest兩 }");
-    fs.create(TEST_PATH);
+    touch(TEST_PATH);
     fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
     Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getPathProperties(TEST_PATH);
 
-    Assert.assertEquals(properties, fetchedProperties);
+    assertEquals(properties, fetchedProperties);
   }
 
   @Test
   public void testSetFileSystemProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem();
     final Hashtable<String, String> properties = new Hashtable<>();
     properties.put("containerForDevTest", "true");
     fs.getAbfsStore().setFilesystemProperties(properties);
     Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
 
-    Assert.assertEquals(properties, fetchedProperties);
+    assertEquals(properties, fetchedProperties);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
index ef61e52..5d1cf91 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
@@ -20,10 +20,9 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.net.URI;
 
-import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.hadoop.fs.AbstractFileSystem;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
@@ -31,33 +30,76 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 
 /**
  * Test AzureBlobFileSystem registration.
+ * Use casts to have interesting stack traces on failures.
  */
-public class ITestFileSystemRegistration extends DependencyInjectedTest {
+public class ITestFileSystemRegistration extends AbstractAbfsIntegrationTest {
+
+  protected static final String ABFS = "org.apache.hadoop.fs.azurebfs.Abfs";
+  protected static final String ABFSS = "org.apache.hadoop.fs.azurebfs.Abfss";
+
   public ITestFileSystemRegistration() throws Exception {
-    super();
+  }
+
+  private void assertConfigMatches(Configuration conf, String key, String expected) {
+    String v = conf.get(key);
+    assertNotNull("No value for key " + key, v);
+    assertEquals("Wrong value for key " + key, expected, v);
+  }
+
+  @Test
+  public void testAbfsFileSystemRegistered() throws Throwable {
+    assertConfigMatches(new Configuration(true),
+        "fs.abfs.impl",
+        "org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem");
+  }
+
+  @Test
+  public void testSecureAbfsFileSystemRegistered() throws Throwable {
+    assertConfigMatches(new Configuration(true),
+        "fs.abfss.impl",
+        "org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem");
+  }
+
+  @Test
+  public void testAbfsFileContextRegistered() throws Throwable {
+    assertConfigMatches(new Configuration(true),
+        "fs.AbstractFileSystem.abfs.impl",
+        ABFS);
+  }
+
+  @Test
+  public void testSecureAbfsFileContextRegistered() throws Throwable {
+    assertConfigMatches(new Configuration(true),
+        "fs.AbstractFileSystem.abfss.impl",
+        ABFSS);
   }
 
   @Test
   public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
-    FileSystem fs = FileSystem.get(this.getConfiguration());
-    Assert.assertTrue(fs instanceof AzureBlobFileSystem);
+    AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(getConfiguration());
+    assertNotNull("filesystem", fs);
 
-    AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem();
-    Assert.assertTrue(afs instanceof Abfs);
+    Abfs afs = (Abfs) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem();
+    assertNotNull("filecontext", afs);
   }
 
   @Test
   public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
-    final String accountName = this.getAccountName();
-    final String fileSystemName = this.getFileSystemName();
-
-    final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, fileSystemName + "@" + accountName, null, null, null);
-    this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
+    final String accountName = getAccountName();
+    final String fileSystemName = getFileSystemName();
 
-    FileSystem fs = FileSystem.get(this.getConfiguration());
-    Assert.assertTrue(fs instanceof SecureAzureBlobFileSystem);
+    final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME,
+        fileSystemName + "@" + accountName,
+        null,
+        null,
+        null);
+    getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+        defaultUri.toString());
 
-    AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem();
-    Assert.assertTrue(afs instanceof Abfss);
+    SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.get(
+        getConfiguration());
+    assertNotNull("filesystem", fs);
+    Abfss afs = (Abfss) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem();
+    assertNotNull("filecontext", afs);
   }
-}
\ 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


[17/50] [abbrv] hadoop git commit: YARN-8633. Update DataTables version in yarn-common in line with JQuery 3 upgrade. Contributed by Akhil PB.

Posted by tm...@apache.org.
YARN-8633. Update DataTables version in yarn-common in line with JQuery 3 upgrade. Contributed by Akhil PB.


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

Branch: refs/heads/HADOOP-15407
Commit: 00013d6ef7fdf65fa8a0f6eb56c0aef2f6e19444
Parents: da9a39e
Author: Sunil G <su...@apache.org>
Authored: Thu Aug 9 12:18:32 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Aug 9 12:18:32 2018 +0530

----------------------------------------------------------------------
 LICENSE.txt                                     |   2 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |   8 +-
 .../hadoop/yarn/webapp/view/JQueryUI.java       |   4 +-
 .../webapps/static/dt-1.10.7/css/demo_page.css  | 110 ++++
 .../webapps/static/dt-1.10.7/css/demo_table.css | 538 +++++++++++++++++++
 .../webapps/static/dt-1.10.7/css/jui-dt.css     | 322 +++++++++++
 .../static/dt-1.10.7/images/Sorting icons.psd   | Bin 0 -> 27490 bytes
 .../static/dt-1.10.7/images/back_disabled.jpg   | Bin 0 -> 612 bytes
 .../static/dt-1.10.7/images/back_enabled.jpg    | Bin 0 -> 807 bytes
 .../webapps/static/dt-1.10.7/images/favicon.ico | Bin 0 -> 894 bytes
 .../dt-1.10.7/images/forward_disabled.jpg       | Bin 0 -> 635 bytes
 .../static/dt-1.10.7/images/forward_enabled.jpg | Bin 0 -> 852 bytes
 .../static/dt-1.10.7/images/sort_asc.png        | Bin 0 -> 263 bytes
 .../dt-1.10.7/images/sort_asc_disabled.png      | Bin 0 -> 252 bytes
 .../static/dt-1.10.7/images/sort_both.png       | Bin 0 -> 282 bytes
 .../static/dt-1.10.7/images/sort_desc.png       | Bin 0 -> 260 bytes
 .../dt-1.10.7/images/sort_desc_disabled.png     | Bin 0 -> 251 bytes
 .../dt-1.10.7/js/jquery.dataTables.min.js       | 160 ++++++
 .../webapps/static/dt-1.9.4/css/demo_page.css   | 110 ----
 .../webapps/static/dt-1.9.4/css/demo_table.css  | 538 -------------------
 .../webapps/static/dt-1.9.4/css/jui-dt.css      | 322 -----------
 .../static/dt-1.9.4/images/Sorting icons.psd    | Bin 27490 -> 0 bytes
 .../static/dt-1.9.4/images/back_disabled.jpg    | Bin 612 -> 0 bytes
 .../static/dt-1.9.4/images/back_enabled.jpg     | Bin 807 -> 0 bytes
 .../webapps/static/dt-1.9.4/images/favicon.ico  | Bin 894 -> 0 bytes
 .../static/dt-1.9.4/images/forward_disabled.jpg | Bin 635 -> 0 bytes
 .../static/dt-1.9.4/images/forward_enabled.jpg  | Bin 852 -> 0 bytes
 .../webapps/static/dt-1.9.4/images/sort_asc.png | Bin 263 -> 0 bytes
 .../dt-1.9.4/images/sort_asc_disabled.png       | Bin 252 -> 0 bytes
 .../static/dt-1.9.4/images/sort_both.png        | Bin 282 -> 0 bytes
 .../static/dt-1.9.4/images/sort_desc.png        | Bin 260 -> 0 bytes
 .../dt-1.9.4/images/sort_desc_disabled.png      | Bin 251 -> 0 bytes
 .../static/dt-1.9.4/js/jquery.dataTables.min.js | 157 ------
 33 files changed, 1137 insertions(+), 1134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index f8de86a..393ed0e 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -553,7 +553,7 @@ For:
 hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.js
 hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.css
 hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery.dataTables.min.js
-hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/
+hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/
 --------------------------------------------------------------------------------
 Copyright (C) 2008-2016, SpryMedia Ltd.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
index eddcbaa..685eac9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
@@ -237,10 +237,10 @@
             <exclude>src/main/resources/webapps/test/.keep</exclude>
             <exclude>src/main/resources/webapps/proxy/.keep</exclude>
             <exclude>src/main/resources/webapps/node/.keep</exclude>
-            <exclude>src/main/resources/webapps/static/dt-1.9.4/css/jui-dt.css</exclude>
-            <exclude>src/main/resources/webapps/static/dt-1.9.4/css/demo_table.css</exclude>
-            <exclude>src/main/resources/webapps/static/dt-1.9.4/images/Sorting icons.psd</exclude>
-            <exclude>src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js</exclude>
+            <exclude>src/main/resources/webapps/static/dt-1.10.7/css/jui-dt.css</exclude>
+            <exclude>src/main/resources/webapps/static/dt-1.10.7/css/demo_table.css</exclude>
+            <exclude>src/main/resources/webapps/static/dt-1.10.7/images/Sorting icons.psd</exclude>
+            <exclude>src/main/resources/webapps/static/dt-1.10.7/js/jquery.dataTables.min.js</exclude>
             <exclude>src/main/resources/webapps/static/jt/jquery.jstree.js</exclude>
             <exclude>src/main/resources/webapps/static/jquery/jquery-ui-1.12.1.custom.min.js</exclude>
             <exclude>src/main/resources/webapps/static/jquery/jquery-3.3.1.min.js</exclude>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
index b8e954d..eef33eb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
@@ -66,10 +66,10 @@ public class JQueryUI extends HtmlBlock {
   @Override
   protected void render(Block html) {
     html.link(root_url("static/jquery/themes-1.9.1/base/jquery-ui.css"))
-        .link(root_url("static/dt-1.9.4/css/jui-dt.css"))
+        .link(root_url("static/dt-1.10.7/css/jui-dt.css"))
         .script(root_url("static/jquery/jquery-3.3.1.min.js"))
         .script(root_url("static/jquery/jquery-ui-1.12.1.custom.min.js"))
-        .script(root_url("static/dt-1.9.4/js/jquery.dataTables.min.js"))
+        .script(root_url("static/dt-1.10.7/js/jquery.dataTables.min.js"))
         .script(root_url("static/yarn.dt.plugins.js"))
         .script(root_url("static/dt-sorting/natural.js"))
         .style("#jsnotice { padding: 0.2em; text-align: center; }",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/demo_page.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/demo_page.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/demo_page.css
new file mode 100644
index 0000000..b60ee7d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/demo_page.css
@@ -0,0 +1,110 @@
+/**
+* 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.
+*/
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * General page setup
+ */
+#dt_example {
+	font: 80%/1.45em "Lucida Grande", Verdana, Arial, Helvetica, sans-serif;
+	margin: 0;
+	padding: 0;
+	color: #333;
+	background-color: #fff;
+}
+
+
+#dt_example #container {
+	width: 800px;
+	margin: 30px auto;
+	padding: 0;
+}
+
+
+#dt_example #footer {
+	margin: 50px auto 0 auto;
+	padding: 0;
+}
+
+#dt_example #demo {
+	margin: 30px auto 0 auto;
+}
+
+#dt_example .demo_jui {
+	margin: 30px auto 0 auto;
+}
+
+#dt_example .big {
+	font-size: 1.3em;
+	font-weight: bold;
+	line-height: 1.6em;
+	color: #4E6CA3;
+}
+
+#dt_example .spacer {
+	height: 20px;
+	clear: both;
+}
+
+#dt_example .clear {
+	clear: both;
+}
+
+#dt_example pre {
+	padding: 15px;
+	background-color: #F5F5F5;
+	border: 1px solid #CCCCCC;
+}
+
+#dt_example h1 {
+	margin-top: 2em;
+	font-size: 1.3em;
+	font-weight: normal;
+	line-height: 1.6em;
+	color: #4E6CA3;
+	border-bottom: 1px solid #B0BED9;
+	clear: both;
+}
+
+#dt_example h2 {
+	font-size: 1.2em;
+	font-weight: normal;
+	line-height: 1.6em;
+	color: #4E6CA3;
+	clear: both;
+}
+
+#dt_example a {
+	color: #0063DC;
+	text-decoration: none;
+}
+
+#dt_example a:hover {
+	text-decoration: underline;
+}
+
+#dt_example ul {
+	color: #4E6CA3;
+}
+
+.css_right {
+	float: right;
+}
+
+.css_left {
+	float: left;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/demo_table.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/demo_table.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/demo_table.css
new file mode 100644
index 0000000..37b9203
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/demo_table.css
@@ -0,0 +1,538 @@
+/*
+ *  File:         demo_table.css
+ *  CVS:          $Id$
+ *  Description:  CSS descriptions for DataTables demo pages
+ *  Author:       Allan Jardine
+ *  Created:      Tue May 12 06:47:22 BST 2009
+ *  Modified:     $Date$ by $Author$
+ *  Language:     CSS
+ *  Project:      DataTables
+ *
+ *  Copyright 2009 Allan Jardine. All Rights Reserved.
+ *
+ * ***************************************************************************
+ * DESCRIPTION
+ *
+ * The styles given here are suitable for the demos that are used with the standard DataTables
+ * distribution (see www.datatables.net). You will most likely wish to modify these styles to
+ * meet the layout requirements of your site.
+ *
+ * Common issues:
+ *   'full_numbers' pagination - I use an extra selector on the body tag to ensure that there is
+ *     no conflict between the two pagination types. If you want to use full_numbers pagination
+ *     ensure that you either have "example_alt_pagination" as a body class name, or better yet,
+ *     modify that selector.
+ *   Note that the path used for Images is relative. All images are by default located in
+ *     ../images/ - relative to this CSS file.
+ */
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * DataTables features
+ */
+
+.dataTables_wrapper {
+	position: relative;
+	min-height: 302px;
+	clear: both;
+	_height: 302px;
+	zoom: 1; /* Feeling sorry for IE */
+}
+
+.dataTables_processing {
+	position: absolute;
+	top: 50%;
+	left: 50%;
+	width: 250px;
+	height: 30px;
+	margin-left: -125px;
+	margin-top: -15px;
+	padding: 14px 0 2px 0;
+	border: 1px solid #ddd;
+	text-align: center;
+	color: #999;
+	font-size: 14px;
+	background-color: white;
+}
+
+.dataTables_length {
+	width: 40%;
+	float: left;
+}
+
+.dataTables_filter {
+	width: 50%;
+	float: right;
+	text-align: right;
+}
+
+.dataTables_info {
+	width: 60%;
+	float: left;
+}
+
+.dataTables_paginate {
+	width: 44px;
+	* width: 50px;
+	float: right;
+	text-align: right;
+}
+
+/* Pagination nested */
+.paginate_disabled_previous, .paginate_enabled_previous, .paginate_disabled_next, .paginate_enabled_next {
+	height: 19px;
+	width: 19px;
+	margin-left: 3px;
+	float: left;
+}
+
+.paginate_disabled_previous {
+	background-image: url('../images/back_disabled.jpg');
+}
+
+.paginate_enabled_previous {
+	background-image: url('../images/back_enabled.jpg');
+}
+
+.paginate_disabled_next {
+	background-image: url('../images/forward_disabled.jpg');
+}
+
+.paginate_enabled_next {
+	background-image: url('../images/forward_enabled.jpg');
+}
+
+
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * DataTables display
+ */
+table.display {
+	margin: 0 auto;
+	clear: both;
+	width: 100%;
+
+	/* Note Firefox 3.5 and before have a bug with border-collapse
+	 * ( https://bugzilla.mozilla.org/show%5Fbug.cgi?id=155955 )
+	 * border-spacing: 0; is one possible option. Conditional-css.com is
+	 * useful for this kind of thing
+	 *
+	 * Further note IE 6/7 has problems when calculating widths with border width.
+	 * It subtracts one px relative to the other browsers from the first column, and
+	 * adds one to the end...
+	 *
+	 * If you want that effect I'd suggest setting a border-top/left on th/td's and
+	 * then filling in the gaps with other borders.
+	 */
+}
+
+table.display thead th {
+	padding: 3px 18px 3px 10px;
+	border-bottom: 1px solid black;
+	font-weight: bold;
+	cursor: pointer;
+	* cursor: hand;
+}
+
+table.display tfoot th {
+	padding: 3px 18px 3px 10px;
+	border-top: 1px solid black;
+	font-weight: bold;
+}
+
+table.display tr.heading2 td {
+	border-bottom: 1px solid #aaa;
+}
+
+table.display td {
+	padding: 3px 10px;
+}
+
+table.display td.center {
+	text-align: center;
+}
+
+
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * DataTables sorting
+ */
+
+.sorting_asc {
+	background: url('../images/sort_asc.png') no-repeat center right;
+}
+
+.sorting_desc {
+	background: url('../images/sort_desc.png') no-repeat center right;
+}
+
+.sorting {
+	background: url('../images/sort_both.png') no-repeat center right;
+}
+
+.sorting_asc_disabled {
+	background: url('../images/sort_asc_disabled.png') no-repeat center right;
+}
+
+.sorting_desc_disabled {
+	background: url('../images/sort_desc_disabled.png') no-repeat center right;
+}
+
+
+
+
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * DataTables row classes
+ */
+table.display tr.odd.gradeA {
+	background-color: #ddffdd;
+}
+
+table.display tr.even.gradeA {
+	background-color: #eeffee;
+}
+
+table.display tr.odd.gradeC {
+	background-color: #ddddff;
+}
+
+table.display tr.even.gradeC {
+	background-color: #eeeeff;
+}
+
+table.display tr.odd.gradeX {
+	background-color: #ffdddd;
+}
+
+table.display tr.even.gradeX {
+	background-color: #ffeeee;
+}
+
+table.display tr.odd.gradeU {
+	background-color: #ddd;
+}
+
+table.display tr.even.gradeU {
+	background-color: #eee;
+}
+
+
+tr.odd {
+	background-color: #E2E4FF;
+}
+
+tr.even {
+	background-color: white;
+}
+
+
+
+
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * Misc
+ */
+.dataTables_scroll {
+	clear: both;
+}
+
+.dataTables_scrollBody {
+	*margin-top: -1px;
+}
+
+.top, .bottom {
+	padding: 15px;
+	background-color: #F5F5F5;
+	border: 1px solid #CCCCCC;
+}
+
+.top .dataTables_info {
+	float: none;
+}
+
+.clear {
+	clear: both;
+}
+
+.dataTables_empty {
+	text-align: center;
+}
+
+tfoot input {
+	margin: 0.5em 0;
+	width: 100%;
+	color: #444;
+}
+
+tfoot input.search_init {
+	color: #999;
+}
+
+td.group {
+	background-color: #d1cfd0;
+	border-bottom: 2px solid #A19B9E;
+	border-top: 2px solid #A19B9E;
+}
+
+td.details {
+	background-color: #d1cfd0;
+	border: 2px solid #A19B9E;
+}
+
+
+.example_alt_pagination div.dataTables_info {
+	width: 40%;
+}
+
+.paging_full_numbers {
+	width: 400px;
+	height: 22px;
+	line-height: 22px;
+}
+
+.paging_full_numbers span.paginate_button,
+ 	.paging_full_numbers span.paginate_active {
+	border: 1px solid #aaa;
+	-webkit-border-radius: 5px;
+	-moz-border-radius: 5px;
+	padding: 2px 5px;
+	margin: 0 3px;
+	cursor: pointer;
+	*cursor: hand;
+}
+
+.paging_full_numbers span.paginate_button {
+	background-color: #ddd;
+}
+
+.paging_full_numbers span.paginate_button:hover {
+	background-color: #ccc;
+}
+
+.paging_full_numbers span.paginate_active {
+	background-color: #99B3FF;
+}
+
+table.display tr.even.row_selected td {
+	background-color: #B0BED9;
+}
+
+table.display tr.odd.row_selected td {
+	background-color: #9FAFD1;
+}
+
+
+/*
+ * Sorting classes for columns
+ */
+/* For the standard odd/even */
+tr.odd td.sorting_1 {
+	background-color: #D3D6FF;
+}
+
+tr.odd td.sorting_2 {
+	background-color: #DADCFF;
+}
+
+tr.odd td.sorting_3 {
+	background-color: #E0E2FF;
+}
+
+tr.even td.sorting_1 {
+	background-color: #EAEBFF;
+}
+
+tr.even td.sorting_2 {
+	background-color: #F2F3FF;
+}
+
+tr.even td.sorting_3 {
+	background-color: #F9F9FF;
+}
+
+
+/* For the Conditional-CSS grading rows */
+/*
+ 	Colour calculations (based off the main row colours)
+  Level 1:
+		dd > c4
+		ee > d5
+	Level 2:
+	  dd > d1
+	  ee > e2
+ */
+tr.odd.gradeA td.sorting_1 {
+	background-color: #c4ffc4;
+}
+
+tr.odd.gradeA td.sorting_2 {
+	background-color: #d1ffd1;
+}
+
+tr.odd.gradeA td.sorting_3 {
+	background-color: #d1ffd1;
+}
+
+tr.even.gradeA td.sorting_1 {
+	background-color: #d5ffd5;
+}
+
+tr.even.gradeA td.sorting_2 {
+	background-color: #e2ffe2;
+}
+
+tr.even.gradeA td.sorting_3 {
+	background-color: #e2ffe2;
+}
+
+tr.odd.gradeC td.sorting_1 {
+	background-color: #c4c4ff;
+}
+
+tr.odd.gradeC td.sorting_2 {
+	background-color: #d1d1ff;
+}
+
+tr.odd.gradeC td.sorting_3 {
+	background-color: #d1d1ff;
+}
+
+tr.even.gradeC td.sorting_1 {
+	background-color: #d5d5ff;
+}
+
+tr.even.gradeC td.sorting_2 {
+	background-color: #e2e2ff;
+}
+
+tr.even.gradeC td.sorting_3 {
+	background-color: #e2e2ff;
+}
+
+tr.odd.gradeX td.sorting_1 {
+	background-color: #ffc4c4;
+}
+
+tr.odd.gradeX td.sorting_2 {
+	background-color: #ffd1d1;
+}
+
+tr.odd.gradeX td.sorting_3 {
+	background-color: #ffd1d1;
+}
+
+tr.even.gradeX td.sorting_1 {
+	background-color: #ffd5d5;
+}
+
+tr.even.gradeX td.sorting_2 {
+	background-color: #ffe2e2;
+}
+
+tr.even.gradeX td.sorting_3 {
+	background-color: #ffe2e2;
+}
+
+tr.odd.gradeU td.sorting_1 {
+	background-color: #c4c4c4;
+}
+
+tr.odd.gradeU td.sorting_2 {
+	background-color: #d1d1d1;
+}
+
+tr.odd.gradeU td.sorting_3 {
+	background-color: #d1d1d1;
+}
+
+tr.even.gradeU td.sorting_1 {
+	background-color: #d5d5d5;
+}
+
+tr.even.gradeU td.sorting_2 {
+	background-color: #e2e2e2;
+}
+
+tr.even.gradeU td.sorting_3 {
+	background-color: #e2e2e2;
+}
+
+
+/*
+ * Row highlighting example
+ */
+.ex_highlight #example tbody tr.even:hover, #example tbody tr.even td.highlighted {
+	background-color: #ECFFB3;
+}
+
+.ex_highlight #example tbody tr.odd:hover, #example tbody tr.odd td.highlighted {
+	background-color: #E6FF99;
+}
+
+.ex_highlight_row #example tr.even:hover {
+	background-color: #ECFFB3;
+}
+
+.ex_highlight_row #example tr.even:hover td.sorting_1 {
+	background-color: #DDFF75;
+}
+
+.ex_highlight_row #example tr.even:hover td.sorting_2 {
+	background-color: #E7FF9E;
+}
+
+.ex_highlight_row #example tr.even:hover td.sorting_3 {
+	background-color: #E2FF89;
+}
+
+.ex_highlight_row #example tr.odd:hover {
+	background-color: #E6FF99;
+}
+
+.ex_highlight_row #example tr.odd:hover td.sorting_1 {
+	background-color: #D6FF5C;
+}
+
+.ex_highlight_row #example tr.odd:hover td.sorting_2 {
+	background-color: #E0FF84;
+}
+
+.ex_highlight_row #example tr.odd:hover td.sorting_3 {
+	background-color: #DBFF70;
+}
+
+
+/*
+ * KeyTable
+ */
+table.KeyTable td {
+	border: 3px solid transparent;
+}
+
+table.KeyTable td.focus {
+	border: 3px solid #3366FF;
+}
+
+table.display tr.gradeA {
+	background-color: #eeffee;
+}
+
+table.display tr.gradeC {
+	background-color: #ddddff;
+}
+
+table.display tr.gradeX {
+	background-color: #ffdddd;
+}
+
+table.display tr.gradeU {
+	background-color: #ddd;
+}
+
+div.box {
+	height: 100px;
+	padding: 10px;
+	overflow: auto;
+	border: 1px solid #8080FF;
+	background-color: #E5E5FF;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/jui-dt.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/jui-dt.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/jui-dt.css
new file mode 100644
index 0000000..de8faea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/css/jui-dt.css
@@ -0,0 +1,322 @@
+/*
+ *  File:         demo_table_jui.css
+ *  CVS:          $Id$
+ *  Description:  CSS descriptions for DataTables demo pages
+ *  Author:       Allan Jardine
+ *  Created:      Tue May 12 06:47:22 BST 2009
+ *  Modified:     $Date$ by $Author$
+ *  Language:     CSS
+ *  Project:      DataTables
+ *
+ *  Copyright 2009 Allan Jardine. All Rights Reserved.
+ *
+ * ***************************************************************************
+ * DESCRIPTION
+ *
+ * The styles given here are suitable for the demos that are used with the standard DataTables
+ * distribution (see www.datatables.net). You will most likely wish to modify these styles to
+ * meet the layout requirements of your site.
+ *
+ * Common issues:
+ *   'full_numbers' pagination - I use an extra selector on the body tag to ensure that there is
+ *     no conflict between the two pagination types. If you want to use full_numbers pagination
+ *     ensure that you either have "example_alt_pagination" as a body class name, or better yet,
+ *     modify that selector.
+ *   Note that the path used for Images is relative. All images are by default located in
+ *     ../images/ - relative to this CSS file.
+ */
+
+
+/*
+ * jQuery UI specific styling
+ */
+
+.paging_two_button .ui-button {
+	float: left;
+	cursor: pointer;
+	* cursor: hand;
+}
+
+.paging_full_numbers .ui-button {
+	padding: 2px 6px;
+	margin: 0;
+	cursor: pointer;
+	* cursor: hand;
+}
+
+.ui-buttonset .ui-button {
+	margin-right: -0.1em !important;
+}
+
+.paging_full_numbers {
+	width: 350px !important;
+}
+
+.ui-toolbar {
+	padding: 5px;
+}
+
+.dataTables_paginate {
+	width: auto;
+}
+
+.dataTables_info {
+	padding-top: 3px;
+}
+
+table.display thead th {
+	padding: 3px 0px 3px 10px;
+	cursor: pointer;
+	* cursor: hand;
+}
+
+div.dataTables_wrapper .ui-widget-header {
+	font-weight: normal;
+}
+
+
+/*
+ * Sort arrow icon positioning
+ */
+table.display thead th div.DataTables_sort_wrapper {
+	position: relative;
+	padding-right: 20px;
+	padding-right: 20px;
+}
+
+table.display thead th div.DataTables_sort_wrapper span {
+	position: absolute;
+	top: 50%;
+	margin-top: -8px;
+	right: 0;
+}
+
+
+
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ *
+ * Everything below this line is the same as demo_table.css. This file is
+ * required for 'cleanliness' of the markup
+ *
+ * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
+
+
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * DataTables features
+ */
+
+.dataTables_wrapper {
+	position: relative;
+	min-height: 35px;
+	_height: 35px;
+	clear: both;
+}
+
+.dataTables_processing {
+	position: absolute;
+	top: 0px;
+	left: 50%;
+	width: 250px;
+	margin-left: -125px;
+	border: 1px solid #ddd;
+	text-align: center;
+	color: #999;
+	font-size: 11px;
+	padding: 2px 0;
+}
+
+.dataTables_length {
+	width: 40%;
+	float: left;
+}
+
+.dataTables_filter {
+	width: 50%;
+	float: right;
+	text-align: right;
+}
+
+.dataTables_info {
+	width: 50%;
+	float: left;
+}
+
+.dataTables_paginate {
+	float: right;
+	text-align: right;
+}
+
+/* Pagination nested */
+.paginate_disabled_previous, .paginate_enabled_previous, .paginate_disabled_next, .paginate_enabled_next {
+	height: 19px;
+	width: 19px;
+	margin-left: 3px;
+	float: left;
+}
+
+.paginate_disabled_previous {
+	background-image: url('../images/back_disabled.jpg');
+}
+
+.paginate_enabled_previous {
+	background-image: url('../images/back_enabled.jpg');
+}
+
+.paginate_disabled_next {
+	background-image: url('../images/forward_disabled.jpg');
+}
+
+.paginate_enabled_next {
+	background-image: url('../images/forward_enabled.jpg');
+}
+
+
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * DataTables display
+ */
+table.display {
+	margin: 0 auto;
+	width: 100%;
+	clear: both;
+	border-collapse: collapse;
+}
+
+table.display tfoot th {
+	padding: 3px 0px 3px 10px;
+	font-weight: bold;
+	font-weight: normal;
+}
+
+table.display tr.heading2 td {
+	border-bottom: 1px solid #aaa;
+}
+
+table.display td {
+	padding: 3px 10px;
+}
+
+table.display td.center {
+	text-align: center;
+}
+
+
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * DataTables sorting
+ */
+
+.sorting_asc {
+	background: url('../images/sort_asc.jpg') no-repeat center right;
+}
+
+.sorting_desc {
+	background: url('../images/sort_desc.jpg') no-repeat center right;
+}
+
+.sorting {
+	background: url('../images/sort_both.jpg') no-repeat center right;
+}
+
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
+ * Misc
+ */
+.dataTables_scroll {
+	clear: both;
+}
+
+.top, .bottom {
+	padding: 15px;
+	background-color: #F5F5F5;
+	border: 1px solid #CCCCCC;
+}
+
+.top .dataTables_info {
+	float: none;
+}
+
+.clear {
+	clear: both;
+}
+
+.dataTables_empty {
+	text-align: center;
+}
+
+tfoot input {
+	margin: 0.5em 0;
+	width: 100%;
+	color: #444;
+}
+
+tfoot input.search_init {
+	color: #999;
+}
+
+td.group {
+	background-color: #d1cfd0;
+	border-bottom: 2px solid #A19B9E;
+	border-top: 2px solid #A19B9E;
+}
+
+td.details {
+	background-color: #d1cfd0;
+	border: 2px solid #A19B9E;
+}
+
+
+.example_alt_pagination div.dataTables_info {
+	width: 40%;
+}
+
+.paging_full_numbers span.paginate_button,
+ 	.paging_full_numbers span.paginate_active {
+	border: 1px solid #aaa;
+	-webkit-border-radius: 5px;
+	-moz-border-radius: 5px;
+	padding: 2px 5px;
+	margin: 0 3px;
+	cursor: pointer;
+	*cursor: hand;
+}
+
+.paging_full_numbers span.paginate_button {
+	background-color: #ddd;
+}
+
+.paging_full_numbers span.paginate_button:hover {
+	background-color: #ccc;
+}
+
+.paging_full_numbers span.paginate_active {
+	background-color: #99B3FF;
+}
+
+table.display tr.even.row_selected td {
+	background-color: #B0BED9;
+}
+
+table.display tr.odd.row_selected td {
+	background-color: #9FAFD1;
+}
+
+/* Striping */
+tr.odd { background: rgba(255, 255, 255, 0.1); }
+tr.even { background: rgba(0, 0, 255, 0.05); }
+
+
+/*
+ * Sorting classes for columns
+ */
+tr.odd td.sorting_1 { background: rgba(0, 0, 0, 0.03); }
+tr.odd td.sorting_2 { background: rgba(0, 0, 0, 0.02); }
+tr.odd td.sorting_3 { background: rgba(0, 0, 0, 0.02); }
+tr.even td.sorting_1 { background: rgba(0, 0, 0, 0.08); }
+tr.even td.sorting_2 { background: rgba(0, 0, 0, 0.06); }
+tr.even td.sorting_3 { background: rgba(0, 0, 0, 0.06); }
+
+.css_left { position: relative; float: left; }
+.css_right { position: relative; float: right; }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/Sorting icons.psd
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/Sorting icons.psd b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/Sorting icons.psd
new file mode 100644
index 0000000..53b2e06
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/Sorting icons.psd differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/back_disabled.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/back_disabled.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/back_disabled.jpg
new file mode 100644
index 0000000..1e73a54
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/back_disabled.jpg differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/back_enabled.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/back_enabled.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/back_enabled.jpg
new file mode 100644
index 0000000..a6d764c
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/back_enabled.jpg differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/favicon.ico
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/favicon.ico b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/favicon.ico
new file mode 100644
index 0000000..6eeaa2a
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/favicon.ico differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/forward_disabled.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/forward_disabled.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/forward_disabled.jpg
new file mode 100644
index 0000000..28a9dc5
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/forward_disabled.jpg differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/forward_enabled.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/forward_enabled.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/forward_enabled.jpg
new file mode 100644
index 0000000..598c075
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/forward_enabled.jpg differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_asc.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_asc.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_asc.png
new file mode 100644
index 0000000..a56d0e2
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_asc.png differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_asc_disabled.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_asc_disabled.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_asc_disabled.png
new file mode 100644
index 0000000..b7e621e
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_asc_disabled.png differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_both.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_both.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_both.png
new file mode 100644
index 0000000..839ac4b
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_both.png differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_desc.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_desc.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_desc.png
new file mode 100644
index 0000000..90b2951
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_desc.png differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_desc_disabled.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_desc_disabled.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_desc_disabled.png
new file mode 100644
index 0000000..2409653
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.7/images/sort_desc_disabled.png differ


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


[31/50] [abbrv] hadoop git commit: HDDS-327. CloseContainer command handler should not throw exception if the container is already closed. Contributed by Shashikant Banerjee.

Posted by tm...@apache.org.
HDDS-327. CloseContainer command handler should not throw exception if the container is already closed. Contributed by Shashikant Banerjee.


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

Branch: refs/heads/HADOOP-15407
Commit: a28624d2a42bf6ad35f859b6ba92fd541d72a67d
Parents: 398d895
Author: Hanisha Koneru <ha...@apache.org>
Authored: Fri Aug 10 11:43:22 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Fri Aug 10 11:43:22 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a28624d2/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 8364a77..9ddb474 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -387,8 +387,8 @@ public class KeyValueHandler extends Handler {
 
     try {
       if (containerState == ContainerLifeCycleState.CLOSED) {
-        throw new StorageContainerException("Container already closed. " +
-            "ContainerID: " + containerID, CLOSED_CONTAINER_RETRY);
+        LOG.debug("Container {} is already closed.", containerID);
+        return ContainerUtils.getSuccessResponse(request);
       } else if (containerState == ContainerLifeCycleState.INVALID) {
         LOG.debug("Invalid container data. ContainerID: {}", containerID);
         throw new StorageContainerException("Invalid container data. " +


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


[06/50] [abbrv] hadoop git commit: Make 3.1.1 awared by other branches

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49c68760/hadoop-common-project/hadoop-common/src/site/markdown/release/3.1.1/RELEASENOTES.3.1.1.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/3.1.1/RELEASENOTES.3.1.1.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.1.1/RELEASENOTES.3.1.1.md
new file mode 100644
index 0000000..8e2c804
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.1.1/RELEASENOTES.3.1.1.md
@@ -0,0 +1,498 @@
+
+<!---
+# 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.
+-->
+# Apache Hadoop Changelog
+
+## Release 3.1.1 - 2018-08-02
+
+
+
+### IMPORTANT ISSUES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HADOOP-14667](https://issues.apache.org/jira/browse/HADOOP-14667) | Flexible Visual Studio support |  Major | build | Allen Wittenauer | Allen Wittenauer |
+
+
+### NEW FEATURES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HDFS-13056](https://issues.apache.org/jira/browse/HDFS-13056) | Expose file-level composite CRCs in HDFS which are comparable across different instances/layouts |  Major | datanode, distcp, erasure-coding, federation, hdfs | Dennis Huo | Dennis Huo |
+| [HDFS-13283](https://issues.apache.org/jira/browse/HDFS-13283) | Percentage based Reserved Space Calculation for DataNode |  Major | datanode, hdfs | Lukas Majercak | Lukas Majercak |
+
+
+### IMPROVEMENTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-8028](https://issues.apache.org/jira/browse/YARN-8028) | Support authorizeUserAccessToQueue in RMWebServices |  Major | . | Wangda Tan | Wangda Tan |
+| [HADOOP-15332](https://issues.apache.org/jira/browse/HADOOP-15332) | Fix typos in hadoop-aws markdown docs |  Minor | . | Gabor Bota | Gabor Bota |
+| [HADOOP-15330](https://issues.apache.org/jira/browse/HADOOP-15330) | Remove jdk1.7 profile from hadoop-annotations module |  Minor | . | Akira Ajisaka | fang zhenyi |
+| [HADOOP-15342](https://issues.apache.org/jira/browse/HADOOP-15342) | Update ADLS connector to use the current SDK version (2.2.7) |  Major | fs/adl | Atul Sikaria | Atul Sikaria |
+| [YARN-1151](https://issues.apache.org/jira/browse/YARN-1151) | Ability to configure auxiliary services from HDFS-based JAR files |  Major | nodemanager | john lilley | Xuan Gong |
+| [HDFS-13418](https://issues.apache.org/jira/browse/HDFS-13418) |  NetworkTopology should be configurable when enable DFSNetworkTopology |  Major | . | Tao Jie | Tao Jie |
+| [HDFS-13439](https://issues.apache.org/jira/browse/HDFS-13439) | Add test case for read block operation when it is moved |  Major | . | Ajay Kumar | Ajay Kumar |
+| [HDFS-13462](https://issues.apache.org/jira/browse/HDFS-13462) | Add BIND\_HOST configuration for JournalNode's HTTP and RPC Servers |  Major | hdfs, journal-node | Lukas Majercak | Lukas Majercak |
+| [YARN-8140](https://issues.apache.org/jira/browse/YARN-8140) | Improve log message when launch cmd is ran for stopped yarn service |  Major | yarn-native-services | Yesha Vora | Eric Yang |
+| [MAPREDUCE-7086](https://issues.apache.org/jira/browse/MAPREDUCE-7086) | Add config to allow FileInputFormat to ignore directories when recursive=false |  Major | . | Sergey Shelukhin | Sergey Shelukhin |
+| [HDFS-12981](https://issues.apache.org/jira/browse/HDFS-12981) | renameSnapshot a Non-Existent snapshot to itself should throw error |  Minor | hdfs | Sailesh Patel | Kitti Nanasi |
+| [YARN-8239](https://issues.apache.org/jira/browse/YARN-8239) | [UI2] Clicking on Node Manager UI under AM container info / App Attempt page goes to old RM UI |  Major | yarn-ui-v2 | Sumana Sathish | Sunil Govindan |
+| [YARN-8260](https://issues.apache.org/jira/browse/YARN-8260) | [UI2] Per-application tracking URL is no longer available in YARN UI2 |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8201](https://issues.apache.org/jira/browse/YARN-8201) | Skip stacktrace of few exception from ClientRMService |  Minor | . | Bibin A Chundatt | Bilwa S T |
+| [HADOOP-15441](https://issues.apache.org/jira/browse/HADOOP-15441) | Log kms url and token service at debug level. |  Minor | . | Wei-Chiu Chuang | Gabor Bota |
+| [HDFS-13544](https://issues.apache.org/jira/browse/HDFS-13544) | Improve logging for JournalNode in federated cluster |  Major | federation, hdfs | Hanisha Koneru | Hanisha Koneru |
+| [YARN-8249](https://issues.apache.org/jira/browse/YARN-8249) | Few REST api's in RMWebServices are missing static user check |  Critical | webapp, yarn | Sunil Govindan | Sunil Govindan |
+| [HDFS-13512](https://issues.apache.org/jira/browse/HDFS-13512) | WebHdfs getFileStatus doesn't return ecPolicy |  Major | . | Ajay Kumar | Ajay Kumar |
+| [HADOOP-15250](https://issues.apache.org/jira/browse/HADOOP-15250) | Split-DNS MultiHomed Server Network Cluster Network IPC Client Bind Addr Wrong |  Critical | ipc, net | Greg Senia | Ajay Kumar |
+| [HDFS-13589](https://issues.apache.org/jira/browse/HDFS-13589) | Add dfsAdmin command to query if "upgrade" is finalized |  Major | hdfs | Hanisha Koneru | Hanisha Koneru |
+| [HADOOP-15486](https://issues.apache.org/jira/browse/HADOOP-15486) | Make NetworkTopology#netLock fair |  Major | net | Nanda kumar | Nanda kumar |
+| [YARN-8213](https://issues.apache.org/jira/browse/YARN-8213) | Add Capacity Scheduler performance metrics |  Critical | capacityscheduler, metrics | Weiwei Yang | Weiwei Yang |
+| [HDFS-13628](https://issues.apache.org/jira/browse/HDFS-13628) | Update Archival Storage doc for Provided Storage |  Major | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [HADOOP-15449](https://issues.apache.org/jira/browse/HADOOP-15449) | Increase default timeout of ZK session to avoid frequent NameNode failover |  Critical | common | Karthik Palanisamy | Karthik Palanisamy |
+| [YARN-8333](https://issues.apache.org/jira/browse/YARN-8333) | Load balance YARN services using RegistryDNS multiple A records |  Major | yarn-native-services | Eric Yang | Eric Yang |
+| [HDFS-13602](https://issues.apache.org/jira/browse/HDFS-13602) | Add checkOperation(WRITE) checks in FSNamesystem |  Major | ha, namenode | Erik Krogen | Chao Sun |
+| [HDFS-13155](https://issues.apache.org/jira/browse/HDFS-13155) | BlockPlacementPolicyDefault.chooseTargetInOrder Not Checking Return Value for NULL |  Minor | namenode | BELUGA BEHR | Zsolt Venczel |
+| [YARN-8389](https://issues.apache.org/jira/browse/YARN-8389) | Improve the description of machine-list property in Federation docs |  Major | documentation, federation | Takanobu Asanuma | Takanobu Asanuma |
+| [HDFS-13511](https://issues.apache.org/jira/browse/HDFS-13511) | Provide specialized exception when block length cannot be obtained |  Major | . | Ted Yu | Gabor Bota |
+| [HDFS-13659](https://issues.apache.org/jira/browse/HDFS-13659) | Add more test coverage for contentSummary for snapshottable path |  Major | namenode, test | Wei-Chiu Chuang | Wei-Chiu Chuang |
+| [YARN-8400](https://issues.apache.org/jira/browse/YARN-8400) | Fix typos in YARN Federation documentation page |  Trivial | . | Bibin A Chundatt | Giovanni Matteo Fumarola |
+| [HADOOP-15499](https://issues.apache.org/jira/browse/HADOOP-15499) | Performance severe drop when running RawErasureCoderBenchmark with NativeRSRawErasureCoder |  Major | . | Sammi Chen | Sammi Chen |
+| [HDFS-13653](https://issues.apache.org/jira/browse/HDFS-13653) | Make dfs.client.failover.random.order a per nameservice configuration |  Major | federation | Ekanth Sethuramalingam | Ekanth Sethuramalingam |
+| [YARN-8394](https://issues.apache.org/jira/browse/YARN-8394) | Improve data locality documentation for Capacity Scheduler |  Major | . | Weiwei Yang | Weiwei Yang |
+| [HDFS-13641](https://issues.apache.org/jira/browse/HDFS-13641) | Add metrics for edit log tailing |  Major | metrics | Chao Sun | Chao Sun |
+| [HDFS-13686](https://issues.apache.org/jira/browse/HDFS-13686) | Add overall metrics for FSNamesystemLock |  Major | hdfs, namenode | Lukas Majercak | Lukas Majercak |
+| [HDFS-13692](https://issues.apache.org/jira/browse/HDFS-13692) | StorageInfoDefragmenter floods log when compacting StorageInfo TreeSet |  Minor | . | Yiqun Lin | Bharat Viswanadham |
+| [YARN-8214](https://issues.apache.org/jira/browse/YARN-8214) | Change default RegistryDNS port |  Major | . | Billie Rinaldi | Billie Rinaldi |
+| [HDFS-13703](https://issues.apache.org/jira/browse/HDFS-13703) | Avoid allocation of CorruptedBlocks hashmap when no corrupted blocks are hit |  Major | performance | Todd Lipcon | Todd Lipcon |
+| [HADOOP-15554](https://issues.apache.org/jira/browse/HADOOP-15554) | Improve JIT performance for Configuration parsing |  Minor | conf, performance | Todd Lipcon | Todd Lipcon |
+| [HDFS-13714](https://issues.apache.org/jira/browse/HDFS-13714) | Fix TestNameNodePrunesMissingStorages test failures on Windows |  Major | hdfs, namenode, test | Lukas Majercak | Lukas Majercak |
+| [HDFS-13712](https://issues.apache.org/jira/browse/HDFS-13712) | BlockReaderRemote.read() logging improvement |  Minor | hdfs-client | Gergo Repas | Gergo Repas |
+| [YARN-8302](https://issues.apache.org/jira/browse/YARN-8302) | ATS v2 should handle HBase connection issue properly |  Major | ATSv2 | Yesha Vora | Billie Rinaldi |
+| [HDFS-13674](https://issues.apache.org/jira/browse/HDFS-13674) | Improve documentation on Metrics |  Minor | documentation, metrics | Chao Sun | Chao Sun |
+| [HDFS-13719](https://issues.apache.org/jira/browse/HDFS-13719) | Docs around dfs.image.transfer.timeout are misleading |  Major | . | Kitti Nanasi | Kitti Nanasi |
+| [HADOOP-15598](https://issues.apache.org/jira/browse/HADOOP-15598) | DataChecksum calculate checksum is contented on hashtable synchronization |  Major | common | Prasanth Jayachandran | Prasanth Jayachandran |
+| [YARN-8501](https://issues.apache.org/jira/browse/YARN-8501) | Reduce complexity of RMWebServices' getApps method |  Major | restapi | Szilard Nemeth | Szilard Nemeth |
+| [HADOOP-15547](https://issues.apache.org/jira/browse/HADOOP-15547) | WASB: improve listStatus performance |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [YARN-8155](https://issues.apache.org/jira/browse/YARN-8155) | Improve ATSv2 client logging in RM and NM publisher |  Major | . | Rohith Sharma K S | Abhishek Modi |
+| [HADOOP-15476](https://issues.apache.org/jira/browse/HADOOP-15476) | fix logging for split-dns multihome |  Major | . | Ajay Kumar | Ajay Kumar |
+
+
+### BUG FIXES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-8040](https://issues.apache.org/jira/browse/YARN-8040) | [UI2] New YARN UI webapp does not respect current pathname for REST api |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [HADOOP-15062](https://issues.apache.org/jira/browse/HADOOP-15062) | TestCryptoStreamsWithOpensslAesCtrCryptoCodec fails on Debian 9 |  Major | . | Miklos Szegedi | Miklos Szegedi |
+| [HDFS-11043](https://issues.apache.org/jira/browse/HDFS-11043) | TestWebHdfsTimeouts fails |  Major | webhdfs | Andrew Wang | Chao Sun |
+| [HADOOP-15331](https://issues.apache.org/jira/browse/HADOOP-15331) | Fix a race condition causing parsing error of java.io.BufferedInputStream in class org.apache.hadoop.conf.Configuration |  Major | common | Miklos Szegedi | Miklos Szegedi |
+| [HDFS-11900](https://issues.apache.org/jira/browse/HDFS-11900) | Hedged reads thread pool creation not synchronized |  Major | hdfs-client | John Zhuge | John Zhuge |
+| [YARN-8032](https://issues.apache.org/jira/browse/YARN-8032) | Yarn service should expose failuresValidityInterval to users and use it for launching containers |  Major | . | Chandni Singh | Chandni Singh |
+| [YARN-8043](https://issues.apache.org/jira/browse/YARN-8043) | Add the exception message for failed launches running under LCE |  Major | . | Shane Kumpf | Shane Kumpf |
+| [YARN-7734](https://issues.apache.org/jira/browse/YARN-7734) | YARN-5418 breaks TestContainerLogsPage.testContainerLogPageAccess |  Major | . | Miklos Szegedi | Tao Yang |
+| [HDFS-13087](https://issues.apache.org/jira/browse/HDFS-13087) | Snapshotted encryption zone information should be immutable |  Major | encryption | LiXin Ge | LiXin Ge |
+| [HADOOP-12862](https://issues.apache.org/jira/browse/HADOOP-12862) | LDAP Group Mapping over SSL can not specify trust store |  Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang |
+| [HADOOP-15317](https://issues.apache.org/jira/browse/HADOOP-15317) | Improve NetworkTopology chooseRandom's loop |  Major | . | Xiao Chen | Xiao Chen |
+| [HADOOP-15355](https://issues.apache.org/jira/browse/HADOOP-15355) | TestCommonConfigurationFields is broken by HADOOP-15312 |  Major | test | Konstantin Shvachko | LiXin Ge |
+| [YARN-7764](https://issues.apache.org/jira/browse/YARN-7764) | Findbugs warning: Resource#getResources may expose internal representation |  Major | api | Weiwei Yang | Weiwei Yang |
+| [YARN-8106](https://issues.apache.org/jira/browse/YARN-8106) | Update LogAggregationIndexedFileController to use readFull instead read to avoid IOException while loading log meta |  Critical | log-aggregation | Prabhu Joseph | Prabhu Joseph |
+| [YARN-8115](https://issues.apache.org/jira/browse/YARN-8115) | [UI2] URL data like nodeHTTPAddress must be encoded in UI before using to access NM |  Major | yarn-ui-v2 | Sunil Govindan | Sreenath Somarajapuram |
+| [HDFS-13350](https://issues.apache.org/jira/browse/HDFS-13350) | Negative legacy block ID will confuse Erasure Coding to be considered as striped block |  Major | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [YARN-8119](https://issues.apache.org/jira/browse/YARN-8119) | [UI2] Timeline Server address' url scheme should be removed while accessing via KNOX |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8083](https://issues.apache.org/jira/browse/YARN-8083) | [UI2] All YARN related configurations are paged together in conf page |  Major | yarn-ui-v2 | Zoltan Haindrich | Gergely Novák |
+| [HADOOP-15366](https://issues.apache.org/jira/browse/HADOOP-15366) | Add a helper shutdown routine in HadoopExecutor to ensure clean shutdown |  Minor | . | Shashikant Banerjee | Shashikant Banerjee |
+| [YARN-7905](https://issues.apache.org/jira/browse/YARN-7905) | Parent directory permission incorrect during public localization |  Critical | . | Bibin A Chundatt | Bilwa S T |
+| [HADOOP-15374](https://issues.apache.org/jira/browse/HADOOP-15374) | Add links of the new features of 3.1.0 to the top page |  Major | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-7804](https://issues.apache.org/jira/browse/YARN-7804) | Refresh action on Grid view page should not be redirected to graph view |  Major | yarn-ui-v2 | Yesha Vora | Gergely Novák |
+| [HDFS-13420](https://issues.apache.org/jira/browse/HDFS-13420) | License header is displayed in ArchivalStorage/MemoryStorage html pages |  Minor | documentation | Akira Ajisaka | Akira Ajisaka |
+| [HDFS-13328](https://issues.apache.org/jira/browse/HDFS-13328) | Abstract ReencryptionHandler recursive logic in separate class. |  Major | namenode | Surendra Singh Lilhore | Surendra Singh Lilhore |
+| [HADOOP-15357](https://issues.apache.org/jira/browse/HADOOP-15357) | Configuration.getPropsWithPrefix no longer does variable substitution |  Major | . | Jim Brennan | Jim Brennan |
+| [YARN-7984](https://issues.apache.org/jira/browse/YARN-7984) | Delete registry entries from ZK on ServiceClient stop and clean up stop/destroy behavior |  Critical | yarn-native-services | Billie Rinaldi | Billie Rinaldi |
+| [YARN-8133](https://issues.apache.org/jira/browse/YARN-8133) | Doc link broken for yarn-service from overview page. |  Blocker | yarn-native-services | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8116](https://issues.apache.org/jira/browse/YARN-8116) | Nodemanager fails with NumberFormatException: For input string: "" |  Critical | . | Yesha Vora | Chandni Singh |
+| [MAPREDUCE-7062](https://issues.apache.org/jira/browse/MAPREDUCE-7062) | Update mapreduce.job.tags description for making use for ATSv2 purpose. |  Major | . | Charan Hebri | Charan Hebri |
+| [YARN-8073](https://issues.apache.org/jira/browse/YARN-8073) | TimelineClientImpl doesn't honor yarn.timeline-service.versions configuration |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8127](https://issues.apache.org/jira/browse/YARN-8127) | Resource leak when async scheduling is enabled |  Critical | . | Weiwei Yang | Tao Yang |
+| [HDFS-13427](https://issues.apache.org/jira/browse/HDFS-13427) | Fix the section titles of transparent encryption document |  Minor | documentation | Akira Ajisaka | Akira Ajisaka |
+| [HDFS-7101](https://issues.apache.org/jira/browse/HDFS-7101) | Potential null dereference in DFSck#doWork() |  Minor | . | Ted Yu | skrho |
+| [HDFS-13426](https://issues.apache.org/jira/browse/HDFS-13426) | Fix javadoc in FsDatasetAsyncDiskService#removeVolume |  Minor | hdfs | Shashikant Banerjee | Shashikant Banerjee |
+| [YARN-8120](https://issues.apache.org/jira/browse/YARN-8120) | JVM can crash with SIGSEGV when exiting due to custom leveldb logger |  Major | nodemanager, resourcemanager | Jason Lowe | Jason Lowe |
+| [YARN-8147](https://issues.apache.org/jira/browse/YARN-8147) | TestClientRMService#testGetApplications sporadically fails |  Major | test | Jason Lowe | Jason Lowe |
+| [HDFS-13436](https://issues.apache.org/jira/browse/HDFS-13436) | Fix javadoc of package-info.java |  Major | documentation | Akira Ajisaka | Akira Ajisaka |
+| [HADOOP-15379](https://issues.apache.org/jira/browse/HADOOP-15379) | Make IrqHandler.bind() public |  Minor | util | Steve Loughran | Ajay Kumar |
+| [YARN-8154](https://issues.apache.org/jira/browse/YARN-8154) | Fix missing titles in PlacementConstraints document |  Minor | documentation | Akira Ajisaka | Weiwei Yang |
+| [YARN-8153](https://issues.apache.org/jira/browse/YARN-8153) | Guaranteed containers always stay in SCHEDULED on NM after restart |  Major | . | Yang Wang | Yang Wang |
+| [HADOOP-14970](https://issues.apache.org/jira/browse/HADOOP-14970) | MiniHadoopClusterManager doesn't respect lack of format option |  Minor | . | Erik Krogen | Erik Krogen |
+| [HDFS-13438](https://issues.apache.org/jira/browse/HDFS-13438) | Fix javadoc in FsVolumeList#removeVolume |  Minor | . | Shashikant Banerjee | Shashikant Banerjee |
+| [YARN-8142](https://issues.apache.org/jira/browse/YARN-8142) | yarn service application stops when AM is killed with SIGTERM |  Major | yarn-native-services | Yesha Vora | Billie Rinaldi |
+| [MAPREDUCE-7077](https://issues.apache.org/jira/browse/MAPREDUCE-7077) | Pipe mapreduce job fails with Permission denied for jobTokenPassword |  Critical | . | Yesha Vora | Akira Ajisaka |
+| [HDFS-13330](https://issues.apache.org/jira/browse/HDFS-13330) | ShortCircuitCache#fetchOrCreate never retries |  Major | . | Wei-Chiu Chuang | Gabor Bota |
+| [YARN-8156](https://issues.apache.org/jira/browse/YARN-8156) | Increase the default value of yarn.timeline-service.app-collector.linger-period.ms |  Major | . | Rohith Sharma K S | Charan Hebri |
+| [YARN-8165](https://issues.apache.org/jira/browse/YARN-8165) | Incorrect queue name logging in AbstractContainerAllocator |  Trivial | capacityscheduler | Weiwei Yang | Weiwei Yang |
+| [HDFS-12828](https://issues.apache.org/jira/browse/HDFS-12828) | OIV ReverseXML Processor fails with escaped characters |  Critical | hdfs | Erik Krogen | Erik Krogen |
+| [HADOOP-15391](https://issues.apache.org/jira/browse/HADOOP-15391) | Add missing css file in hadoop-aws, hadoop-aliyun, hadoop-azure and hadoop-azure-datalake modules |  Major | documentation | Yiqun Lin | Yiqun Lin |
+| [YARN-8171](https://issues.apache.org/jira/browse/YARN-8171) | [UI2] AM Node link from attempt page should not redirect to new tab |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8145](https://issues.apache.org/jira/browse/YARN-8145) | yarn rmadmin -getGroups doesn't return updated groups for user |  Major | . | Sumana Sathish | Sunil Govindan |
+| [HDFS-13463](https://issues.apache.org/jira/browse/HDFS-13463) | Fix javadoc in FsDatasetImpl#checkAndUpdate |  Minor | datanode | Shashikant Banerjee | Shashikant Banerjee |
+| [HDFS-13464](https://issues.apache.org/jira/browse/HDFS-13464) | Fix javadoc in FsVolumeList#handleVolumeFailures |  Minor | documentation | Shashikant Banerjee | Shashikant Banerjee |
+| [HADOOP-15396](https://issues.apache.org/jira/browse/HADOOP-15396) | Some java source files are executable |  Minor | . | Akira Ajisaka | Shashikant Banerjee |
+| [YARN-6827](https://issues.apache.org/jira/browse/YARN-6827) | [ATS1/1.5] NPE exception while publishing recovering applications into ATS during RM restart. |  Major | resourcemanager | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8182](https://issues.apache.org/jira/browse/YARN-8182) | [UI2] Proxy- Clicking on nodes under Nodes HeatMap gives 401 error |  Critical | . | Sumana Sathish | Sunil Govindan |
+| [YARN-8189](https://issues.apache.org/jira/browse/YARN-8189) | [UI2] Nodes page column headers are half truncated |  Major | . | Sunil Govindan | Sunil Govindan |
+| [YARN-7830](https://issues.apache.org/jira/browse/YARN-7830) | [UI2] Post selecting grid view in Attempt page, attempt info page should also be opened with grid view |  Major | yarn-ui-v2 | Yesha Vora | Gergely Novák |
+| [YARN-7786](https://issues.apache.org/jira/browse/YARN-7786) | NullPointerException while launching ApplicationMaster |  Major | . | lujie | lujie |
+| [HDFS-10183](https://issues.apache.org/jira/browse/HDFS-10183) | Prevent race condition during class initialization |  Minor | fs | Pavel Avgustinov | Pavel Avgustinov |
+| [HDFS-13388](https://issues.apache.org/jira/browse/HDFS-13388) | RequestHedgingProxyProvider calls multiple configured NNs all the time |  Major | hdfs-client | Jinglun | Jinglun |
+| [YARN-7956](https://issues.apache.org/jira/browse/YARN-7956) | [UI2] Avoid duplicating Components link under Services/\<ServiceName\>/Components |  Major | yarn-ui-v2 | Yesha Vora | Yesha Vora |
+| [HDFS-13433](https://issues.apache.org/jira/browse/HDFS-13433) | webhdfs requests can be routed incorrectly in federated cluster |  Critical | webhdfs | Arpit Agarwal | Arpit Agarwal |
+| [HDFS-13408](https://issues.apache.org/jira/browse/HDFS-13408) | MiniDFSCluster to support being built on randomized base directory |  Major | test | Xiao Liang | Xiao Liang |
+| [HDFS-13356](https://issues.apache.org/jira/browse/HDFS-13356) | Balancer:Set default value of minBlockSize to 10mb |  Major | balancer & mover | Bharat Viswanadham | Bharat Viswanadham |
+| [HADOOP-15390](https://issues.apache.org/jira/browse/HADOOP-15390) | Yarn RM logs flooded by DelegationTokenRenewer trying to renew KMS tokens |  Critical | . | Xiao Chen | Xiao Chen |
+| [HDFS-13336](https://issues.apache.org/jira/browse/HDFS-13336) | Test cases of TestWriteToReplica failed in windows |  Major | . | Xiao Liang | Xiao Liang |
+| [YARN-8196](https://issues.apache.org/jira/browse/YARN-8196) | yarn.webapp.api-service.enable should be highlighted in the quickstart |  Trivial | documentation | Davide  Vergari | Billie Rinaldi |
+| [YARN-8183](https://issues.apache.org/jira/browse/YARN-8183) | Fix ConcurrentModificationException inside RMAppAttemptMetrics#convertAtomicLongMaptoLongMap |  Critical | yarn | Sumana Sathish | Suma Shivaprasad |
+| [YARN-8188](https://issues.apache.org/jira/browse/YARN-8188) | RM Nodes UI data table index for sorting column need to be corrected post Application tags display |  Major | resourcemanager, webapp | Weiwei Yang | Weiwei Yang |
+| [HADOOP-15411](https://issues.apache.org/jira/browse/HADOOP-15411) | AuthenticationFilter should use Configuration.getPropsWithPrefix instead of iterator |  Critical | . | Suma Shivaprasad | Suma Shivaprasad |
+| [MAPREDUCE-7042](https://issues.apache.org/jira/browse/MAPREDUCE-7042) | Killed MR job data does not move to mapreduce.jobhistory.done-dir when ATS v2 is enabled |  Major | . | Yesha Vora | Xuan Gong |
+| [YARN-8205](https://issues.apache.org/jira/browse/YARN-8205) | Application State is not updated to ATS if AM launching is delayed. |  Critical | . | Sumana Sathish | Rohith Sharma K S |
+| [YARN-8004](https://issues.apache.org/jira/browse/YARN-8004) | Add unit tests for inter queue preemption for dominant resource calculator |  Critical | yarn | Sumana Sathish | Zian Chen |
+| [YARN-8208](https://issues.apache.org/jira/browse/YARN-8208) | Add log statement for Docker client configuration file at INFO level |  Minor | yarn-native-services | Yesha Vora | Yesha Vora |
+| [YARN-8211](https://issues.apache.org/jira/browse/YARN-8211) | Yarn registry dns log finds BufferUnderflowException on port ping |  Major | yarn-native-services | Yesha Vora | Eric Yang |
+| [YARN-8221](https://issues.apache.org/jira/browse/YARN-8221) | RMWebServices also need to honor yarn.resourcemanager.display.per-user-apps |  Major | webapp | Sunil Govindan | Sunil Govindan |
+| [YARN-8210](https://issues.apache.org/jira/browse/YARN-8210) | AMRMClient logging on every heartbeat to track updation of AM RM token causes too many log lines to be generated in AM logs |  Major | yarn | Suma Shivaprasad | Suma Shivaprasad |
+| [YARN-8005](https://issues.apache.org/jira/browse/YARN-8005) | Add unit tests for queue priority with dominant resource calculator |  Critical | . | Sumana Sathish | Zian Chen |
+| [YARN-8225](https://issues.apache.org/jira/browse/YARN-8225) | YARN precommit build failing in TestPlacementConstraintTransformations |  Critical | . | Billie Rinaldi | Shane Kumpf |
+| [HDFS-13509](https://issues.apache.org/jira/browse/HDFS-13509) | Bug fix for breakHardlinks() of ReplicaInfo/LocalReplica, and fix TestFileAppend failures on Windows |  Major | . | Xiao Liang | Xiao Liang |
+| [YARN-8187](https://issues.apache.org/jira/browse/YARN-8187) | [UI2] Individual Node page does not contain breadcrumb trail |  Critical | yarn-ui-v2 | Sumana Sathish | Zian Chen |
+| [YARN-7799](https://issues.apache.org/jira/browse/YARN-7799) | YARN Service dependency follow up work |  Critical | client, resourcemanager | Gour Saha | Billie Rinaldi |
+| [MAPREDUCE-7073](https://issues.apache.org/jira/browse/MAPREDUCE-7073) | Optimize TokenCache#obtainTokensForNamenodesInternal |  Major | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HADOOP-15406](https://issues.apache.org/jira/browse/HADOOP-15406) | hadoop-nfs dependencies for mockito and junit are not test scope |  Major | nfs | Jason Lowe | Jason Lowe |
+| [YARN-6385](https://issues.apache.org/jira/browse/YARN-6385) | Fix checkstyle warnings in TestFileSystemApplicationHistoryStore |  Minor | . | Yiqun Lin | Yiqun Lin |
+| [YARN-8222](https://issues.apache.org/jira/browse/YARN-8222) | Fix potential NPE when gets RMApp from RM context |  Critical | . | Tao Yang | Tao Yang |
+| [YARN-8209](https://issues.apache.org/jira/browse/YARN-8209) | NPE in DeletionService |  Critical | . | Chandni Singh | Eric Badger |
+| [HDFS-13481](https://issues.apache.org/jira/browse/HDFS-13481) | TestRollingFileSystemSinkWithHdfs#testFlushThread: test failed intermittently |  Major | hdfs | Gabor Bota | Gabor Bota |
+| [YARN-8217](https://issues.apache.org/jira/browse/YARN-8217) | RmAuthenticationFilterInitializer /TimelineAuthenticationFilterInitializer should use Configuration.getPropsWithPrefix instead of iterator |  Major | . | Suma Shivaprasad | Suma Shivaprasad |
+| [YARN-7818](https://issues.apache.org/jira/browse/YARN-7818) | Remove privileged operation warnings during container launch for the ContainerRuntimes |  Major | . | Yesha Vora | Shane Kumpf |
+| [YARN-8223](https://issues.apache.org/jira/browse/YARN-8223) | ClassNotFoundException when auxiliary service is loaded from HDFS |  Blocker | . | Charan Hebri | Zian Chen |
+| [YARN-8079](https://issues.apache.org/jira/browse/YARN-8079) | Support static and archive unmodified local resources in service AM |  Critical | . | Wangda Tan | Suma Shivaprasad |
+| [YARN-8025](https://issues.apache.org/jira/browse/YARN-8025) | UsersManangers#getComputedResourceLimitForActiveUsers throws NPE due to preComputedActiveUserLimit is empty |  Major | yarn | Jiandan Yang | Tao Yang |
+| [YARN-8251](https://issues.apache.org/jira/browse/YARN-8251) | [UI2] Clicking on Application link at the header goes to Diagnostics Tab instead of AppAttempt Tab |  Major | yarn-ui-v2 | Sumana Sathish | Yesha Vora |
+| [YARN-8232](https://issues.apache.org/jira/browse/YARN-8232) | RMContainer lost queue name when RM HA happens |  Major | resourcemanager | Hu Ziqian | Hu Ziqian |
+| [YARN-7894](https://issues.apache.org/jira/browse/YARN-7894) | Improve ATS response for DS\_CONTAINER when container launch fails |  Major | timelineserver | Charan Hebri | Chandni Singh |
+| [YARN-8264](https://issues.apache.org/jira/browse/YARN-8264) | [UI2 GPU] GPU Info tab disappears if we click any sub link under List of Applications or List of Containers |  Major | . | Sumana Sathish | Sunil Govindan |
+| [HDFS-13537](https://issues.apache.org/jira/browse/HDFS-13537) | TestHdfsHelper does not generate jceks path properly for relative path in Windows |  Major | . | Xiao Liang | Xiao Liang |
+| [YARN-8202](https://issues.apache.org/jira/browse/YARN-8202) | DefaultAMSProcessor should properly check units of requested custom resource types against minimum/maximum allocation |  Blocker | . | Szilard Nemeth | Szilard Nemeth |
+| [HADOOP-15446](https://issues.apache.org/jira/browse/HADOOP-15446) | WASB: PageBlobInputStream.skip breaks HBASE replication |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [YARN-7003](https://issues.apache.org/jira/browse/YARN-7003) | DRAINING state of queues is not recovered after RM restart |  Major | capacityscheduler | Tao Yang | Tao Yang |
+| [YARN-8244](https://issues.apache.org/jira/browse/YARN-8244) |  TestContainerSchedulerQueuing.testStartMultipleContainers failed |  Major | . | Miklos Szegedi | Jim Brennan |
+| [YARN-8265](https://issues.apache.org/jira/browse/YARN-8265) | Service AM should retrieve new IP for docker container relaunched by NM |  Critical | yarn-native-services | Eric Yang | Billie Rinaldi |
+| [YARN-8271](https://issues.apache.org/jira/browse/YARN-8271) | [UI2] Improve labeling of certain tables |  Major | yarn-ui-v2 | Yesha Vora | Yesha Vora |
+| [YARN-8288](https://issues.apache.org/jira/browse/YARN-8288) | Fix wrong number of table columns in Resource Model doc |  Major | . | Weiwei Yang | Weiwei Yang |
+| [HDFS-13539](https://issues.apache.org/jira/browse/HDFS-13539) | DFSStripedInputStream NPE when reportCheckSumFailure |  Major | . | Xiao Chen | Xiao Chen |
+| [YARN-8266](https://issues.apache.org/jira/browse/YARN-8266) | [UI2] Clicking on application from cluster view should redirect to application attempt page |  Major | yarn-ui-v2 | Yesha Vora | Yesha Vora |
+| [YARN-8166](https://issues.apache.org/jira/browse/YARN-8166) | [UI2] Service page header links are broken |  Major | yarn-ui-v2 | Yesha Vora | Yesha Vora |
+| [YARN-8236](https://issues.apache.org/jira/browse/YARN-8236) | Invalid kerberos principal file name cause NPE in native service |  Critical | yarn-native-services | Sunil Govindan | Gour Saha |
+| [YARN-8278](https://issues.apache.org/jira/browse/YARN-8278) | DistributedScheduling is not working in HA |  Blocker | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HADOOP-15442](https://issues.apache.org/jira/browse/HADOOP-15442) | ITestS3AMetrics.testMetricsRegister can't know metrics source's name |  Major | fs/s3, metrics | Sean Mackrory | Sean Mackrory |
+| [YARN-8300](https://issues.apache.org/jira/browse/YARN-8300) | Fix NPE in DefaultUpgradeComponentsFinder |  Major | yarn | Suma Shivaprasad | Suma Shivaprasad |
+| [HDFS-13581](https://issues.apache.org/jira/browse/HDFS-13581) | DN UI logs link is broken when https is enabled |  Minor | datanode | Namit Maheshwari | Shashikant Banerjee |
+| [YARN-8128](https://issues.apache.org/jira/browse/YARN-8128) | Document better the per-node per-app file limit in YARN log aggregation |  Major | . | Xuan Gong | Xuan Gong |
+| [YARN-8293](https://issues.apache.org/jira/browse/YARN-8293) | In YARN Services UI, "User Name for service" should be completely removed in secure clusters |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8141](https://issues.apache.org/jira/browse/YARN-8141) | YARN Native Service: Respect YARN\_CONTAINER\_RUNTIME\_DOCKER\_LOCAL\_RESOURCE\_MOUNTS specified in service spec |  Critical | yarn-native-services | Wangda Tan | Chandni Singh |
+| [YARN-8296](https://issues.apache.org/jira/browse/YARN-8296) | Update YarnServiceApi documentation and yarn service UI code to remove references to unique\_component\_support |  Major | yarn-native-services, yarn-ui-v2 | Suma Shivaprasad | Suma Shivaprasad |
+| [HDFS-13586](https://issues.apache.org/jira/browse/HDFS-13586) | Fsync fails on directories on Windows |  Critical | datanode, hdfs | Lukas Majercak | Lukas Majercak |
+| [HADOOP-15478](https://issues.apache.org/jira/browse/HADOOP-15478) | WASB: hflush() and hsync() regression |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [YARN-8179](https://issues.apache.org/jira/browse/YARN-8179) | Preemption does not happen due to natural\_termination\_factor when DRF is used |  Major | . | kyungwan nam | kyungwan nam |
+| [HADOOP-15450](https://issues.apache.org/jira/browse/HADOOP-15450) | Avoid fsync storm triggered by DiskChecker and handle disk full situation |  Blocker | . | Kihwal Lee | Arpit Agarwal |
+| [YARN-8290](https://issues.apache.org/jira/browse/YARN-8290) | SystemMetricsPublisher.appACLsUpdated should be invoked after application information is published to ATS to avoid "User is not set in the application report" Exception |  Critical | . | Yesha Vora | Eric Yang |
+| [YARN-8332](https://issues.apache.org/jira/browse/YARN-8332) | Incorrect min/max allocation property name in resource types doc |  Critical | documentation | Weiwei Yang | Weiwei Yang |
+| [HDFS-13601](https://issues.apache.org/jira/browse/HDFS-13601) | Optimize ByteString conversions in PBHelper |  Major | . | Andrew Wang | Andrew Wang |
+| [HDFS-13540](https://issues.apache.org/jira/browse/HDFS-13540) | DFSStripedInputStream should only allocate new buffers when reading |  Major | . | Xiao Chen | Xiao Chen |
+| [YARN-8297](https://issues.apache.org/jira/browse/YARN-8297) | Incorrect ATS Url used for Wire encrypted cluster |  Blocker | yarn-ui-v2 | Yesha Vora | Sunil Govindan |
+| [HDFS-13588](https://issues.apache.org/jira/browse/HDFS-13588) | Fix TestFsDatasetImpl test failures on Windows |  Major | . | Xiao Liang | Xiao Liang |
+| [YARN-8310](https://issues.apache.org/jira/browse/YARN-8310) | Handle old NMTokenIdentifier, AMRMTokenIdentifier, and ContainerTokenIdentifier formats |  Major | . | Robert Kanter | Robert Kanter |
+| [YARN-8344](https://issues.apache.org/jira/browse/YARN-8344) | Missing nm.stop() in TestNodeManagerResync to fix testKillContainersOnResync |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-8327](https://issues.apache.org/jira/browse/YARN-8327) | Fix TestAggregatedLogFormat#testReadAcontainerLogs1 on Windows |  Major | log-aggregation | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HDFS-13611](https://issues.apache.org/jira/browse/HDFS-13611) | Unsafe use of Text as a ConcurrentHashMap key in PBHelperClient |  Major | . | Andrew Wang | Andrew Wang |
+| [YARN-8316](https://issues.apache.org/jira/browse/YARN-8316) | Diagnostic message should improve when yarn service fails to launch due to ATS unavailability |  Major | yarn-native-services | Yesha Vora | Billie Rinaldi |
+| [YARN-8357](https://issues.apache.org/jira/browse/YARN-8357) | Yarn Service: NPE when service is saved first and then started. |  Critical | . | Chandni Singh | Chandni Singh |
+| [HDFS-13618](https://issues.apache.org/jira/browse/HDFS-13618) | Fix TestDataNodeFaultInjector test failures on Windows |  Major | test | Xiao Liang | Xiao Liang |
+| [HADOOP-15473](https://issues.apache.org/jira/browse/HADOOP-15473) | Configure serialFilter in KeyProvider to avoid UnrecoverableKeyException caused by JDK-8189997 |  Critical | kms | Gabor Bota | Gabor Bota |
+| [YARN-8292](https://issues.apache.org/jira/browse/YARN-8292) | Fix the dominant resource preemption cannot happen when some of the resource vector becomes negative |  Critical | yarn | Sumana Sathish | Wangda Tan |
+| [YARN-8338](https://issues.apache.org/jira/browse/YARN-8338) | TimelineService V1.5 doesn't come up after HADOOP-15406 |  Critical | . | Vinod Kumar Vavilapalli | Vinod Kumar Vavilapalli |
+| [YARN-8339](https://issues.apache.org/jira/browse/YARN-8339) | Service AM should localize static/archive resource types to container working directory instead of 'resources' |  Critical | yarn-native-services | Suma Shivaprasad | Suma Shivaprasad |
+| [YARN-8369](https://issues.apache.org/jira/browse/YARN-8369) | Javadoc build failed due to "bad use of '\>'" |  Critical | build, docs | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-8362](https://issues.apache.org/jira/browse/YARN-8362) | Number of remaining retries are updated twice after a container failure in NM |  Critical | . | Chandni Singh | Chandni Singh |
+| [YARN-8377](https://issues.apache.org/jira/browse/YARN-8377) | Javadoc build failed in hadoop-yarn-server-nodemanager |  Critical | build, docs | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-8368](https://issues.apache.org/jira/browse/YARN-8368) | yarn app start cli should print applicationId |  Critical | . | Yesha Vora | Rohith Sharma K S |
+| [YARN-8350](https://issues.apache.org/jira/browse/YARN-8350) | NPE in service AM related to placement policy |  Critical | yarn-native-services | Billie Rinaldi | Gour Saha |
+| [YARN-8367](https://issues.apache.org/jira/browse/YARN-8367) | Fix NPE in SingleConstraintAppPlacementAllocator when placement constraint in SchedulingRequest is null |  Major | scheduler | Gour Saha | Weiwei Yang |
+| [YARN-8197](https://issues.apache.org/jira/browse/YARN-8197) | Tracking URL in the app state does not get redirected to MR ApplicationMaster for Running applications |  Critical | yarn | Sumana Sathish | Sunil Govindan |
+| [YARN-8308](https://issues.apache.org/jira/browse/YARN-8308) | Yarn service app fails due to issues with Renew Token |  Major | yarn-native-services | Yesha Vora | Gour Saha |
+| [HDFS-13636](https://issues.apache.org/jira/browse/HDFS-13636) | Cross-Site Scripting vulnerability in HttpServer2 |  Major | . | Haibo Yan | Haibo Yan |
+| [YARN-7962](https://issues.apache.org/jira/browse/YARN-7962) | Race Condition When Stopping DelegationTokenRenewer causes RM crash during failover |  Critical | resourcemanager | BELUGA BEHR | BELUGA BEHR |
+| [YARN-8372](https://issues.apache.org/jira/browse/YARN-8372) | Distributed shell app master should not release containers when shutdown if keep-container is true |  Critical | distributed-shell | Charan Hebri | Suma Shivaprasad |
+| [YARN-8319](https://issues.apache.org/jira/browse/YARN-8319) | More YARN pages need to honor yarn.resourcemanager.display.per-user-apps |  Major | webapp | Vinod Kumar Vavilapalli | Sunil Govindan |
+| [MAPREDUCE-7097](https://issues.apache.org/jira/browse/MAPREDUCE-7097) | MapReduce JHS should honor yarn.webapp.filter-entity-list-by-user |  Major | . | Vinod Kumar Vavilapalli | Sunil Govindan |
+| [YARN-8276](https://issues.apache.org/jira/browse/YARN-8276) | [UI2] After version field became mandatory, form-based submission of new YARN service doesn't work |  Critical | yarn-ui-v2 | Gergely Novák | Gergely Novák |
+| [HDFS-13339](https://issues.apache.org/jira/browse/HDFS-13339) | Volume reference can't be released and may lead to deadlock when DataXceiver does a check volume |  Critical | datanode | liaoyuxiangqin | Zsolt Venczel |
+| [YARN-8382](https://issues.apache.org/jira/browse/YARN-8382) | cgroup file leak in NM |  Major | nodemanager | Hu Ziqian | Hu Ziqian |
+| [YARN-8365](https://issues.apache.org/jira/browse/YARN-8365) | Revisit the record type used by Registry DNS for upstream resolution |  Major | yarn-native-services | Shane Kumpf | Shane Kumpf |
+| [HDFS-13545](https://issues.apache.org/jira/browse/HDFS-13545) |  "guarded" is misspelled as "gaurded" in FSPermissionChecker.java |  Trivial | documentation | Jianchao Jia | Jianchao Jia |
+| [YARN-8396](https://issues.apache.org/jira/browse/YARN-8396) | Click on an individual container continuously spins and doesn't load the page |  Blocker | . | Charan Hebri | Sunil Govindan |
+| [MAPREDUCE-7103](https://issues.apache.org/jira/browse/MAPREDUCE-7103) | Fix TestHistoryViewerPrinter on windows due to a mismatch line separator |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HADOOP-15217](https://issues.apache.org/jira/browse/HADOOP-15217) | FsUrlConnection does not handle paths with spaces |  Major | fs | Joseph Fourny | Zsolt Venczel |
+| [HDFS-12950](https://issues.apache.org/jira/browse/HDFS-12950) | [oiv] ls will fail in  secure cluster |  Major | . | Brahma Reddy Battula | Wei-Chiu Chuang |
+| [YARN-8386](https://issues.apache.org/jira/browse/YARN-8386) |  App log can not be viewed from Logs tab in secure cluster |  Critical | yarn-ui-v2 | Yesha Vora | Sunil Govindan |
+| [YARN-8359](https://issues.apache.org/jira/browse/YARN-8359) | Exclude containermanager.linux test classes on Windows |  Major | . | Giovanni Matteo Fumarola | Jason Lowe |
+| [HDFS-13664](https://issues.apache.org/jira/browse/HDFS-13664) | Refactor ConfiguredFailoverProxyProvider to make inheritance easier |  Minor | hdfs-client | Chao Sun | Chao Sun |
+| [HDFS-12670](https://issues.apache.org/jira/browse/HDFS-12670) | can't renew HDFS tokens with only the hdfs client jar |  Critical | . | Thomas Graves | Arpit Agarwal |
+| [HDFS-13667](https://issues.apache.org/jira/browse/HDFS-13667) | Typo: Marking all "datandoes" as stale |  Trivial | namenode | Wei-Chiu Chuang | Nanda kumar |
+| [YARN-8413](https://issues.apache.org/jira/browse/YARN-8413) | Flow activity page is failing with "Timeline server failed with an error" |  Major | yarn-ui-v2 | Yesha Vora | Sunil Govindan |
+| [YARN-8405](https://issues.apache.org/jira/browse/YARN-8405) | RM zk-state-store.parent-path ACLs has been changed since HADOOP-14773 |  Major | . | Rohith Sharma K S | Íñigo Goiri |
+| [YARN-8419](https://issues.apache.org/jira/browse/YARN-8419) | [UI2] User cannot submit a new service as submit button is always disabled |  Major | . | Suma Shivaprasad | Suma Shivaprasad |
+| [MAPREDUCE-7108](https://issues.apache.org/jira/browse/MAPREDUCE-7108) | TestFileOutputCommitter fails on Windows |  Minor | test | Zuoming Zhang | Zuoming Zhang |
+| [MAPREDUCE-7101](https://issues.apache.org/jira/browse/MAPREDUCE-7101) | Add config parameter to allow JHS to alway scan user dir irrespective of modTime |  Critical | . | Wangda Tan | Thomas Marquardt |
+| [HADOOP-15527](https://issues.apache.org/jira/browse/HADOOP-15527) | loop until TIMEOUT before sending kill -9 |  Major | . | Vinod Kumar Vavilapalli | Vinod Kumar Vavilapalli |
+| [YARN-8404](https://issues.apache.org/jira/browse/YARN-8404) | Timeline event publish need to be async to avoid Dispatcher thread leak in case ATS is down |  Blocker | . | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8410](https://issues.apache.org/jira/browse/YARN-8410) | Registry DNS lookup fails to return for CNAMEs |  Major | yarn-native-services | Shane Kumpf | Shane Kumpf |
+| [HDFS-13675](https://issues.apache.org/jira/browse/HDFS-13675) | Speed up TestDFSAdminWithHA |  Major | hdfs, namenode | Lukas Majercak | Lukas Majercak |
+| [HDFS-13673](https://issues.apache.org/jira/browse/HDFS-13673) | TestNameNodeMetrics fails on Windows |  Minor | test | Zuoming Zhang | Zuoming Zhang |
+| [HDFS-13676](https://issues.apache.org/jira/browse/HDFS-13676) | TestEditLogRace fails on Windows |  Minor | test | Zuoming Zhang | Zuoming Zhang |
+| [HDFS-13174](https://issues.apache.org/jira/browse/HDFS-13174) | hdfs mover -p /path times out after 20 min |  Major | balancer & mover | Istvan Fajth | Istvan Fajth |
+| [HADOOP-15523](https://issues.apache.org/jira/browse/HADOOP-15523) | Shell command timeout given is in seconds whereas it is taken as millisec while scheduling |  Major | . | Bilwa S T | Bilwa S T |
+| [HDFS-13682](https://issues.apache.org/jira/browse/HDFS-13682) | Cannot create encryption zone after KMS auth token expires |  Critical | encryption, kms, namenode | Xiao Chen | Xiao Chen |
+| [YARN-8445](https://issues.apache.org/jira/browse/YARN-8445) | YARN native service doesn't allow service name equals to component name |  Major | . | Chandni Singh | Chandni Singh |
+| [YARN-8444](https://issues.apache.org/jira/browse/YARN-8444) | NodeResourceMonitor crashes on bad swapFree value |  Major | . | Jim Brennan | Jim Brennan |
+| [YARN-8326](https://issues.apache.org/jira/browse/YARN-8326) | Yarn 3.0 seems runs slower than Yarn 2.6 |  Major | yarn | Hsin-Liang Huang | Shane Kumpf |
+| [YARN-8443](https://issues.apache.org/jira/browse/YARN-8443) | Total #VCores in cluster metrics is wrong when CapacityScheduler reserved some containers |  Major | webapp | Tao Yang | Tao Yang |
+| [YARN-8457](https://issues.apache.org/jira/browse/YARN-8457) | Compilation is broken with -Pyarn-ui |  Major | webapp | Sunil Govindan | Sunil Govindan |
+| [YARN-8464](https://issues.apache.org/jira/browse/YARN-8464) | Async scheduling thread could be interrupted when there are no NodeManagers in cluster |  Blocker | capacity scheduler | Charan Hebri | Sunil Govindan |
+| [YARN-8423](https://issues.apache.org/jira/browse/YARN-8423) | GPU does not get released even though the application gets killed. |  Critical | yarn | Sumana Sathish | Sunil Govindan |
+| [YARN-8401](https://issues.apache.org/jira/browse/YARN-8401) | [UI2] new ui is not accessible with out internet connection |  Blocker | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HDFS-13705](https://issues.apache.org/jira/browse/HDFS-13705) | The native ISA-L library loading failure should be made warning rather than an error message |  Minor | erasure-coding | Nilotpal Nandi | Shashikant Banerjee |
+| [YARN-8409](https://issues.apache.org/jira/browse/YARN-8409) | ActiveStandbyElectorBasedElectorService is failing with NPE |  Major | . | Yesha Vora | Chandni Singh |
+| [YARN-8379](https://issues.apache.org/jira/browse/YARN-8379) | Improve balancing resources in already satisfied queues by using Capacity Scheduler preemption |  Major | . | Wangda Tan | Zian Chen |
+| [YARN-8455](https://issues.apache.org/jira/browse/YARN-8455) | Add basic ACL check for all ATS v2 REST APIs |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8469](https://issues.apache.org/jira/browse/YARN-8469) | [UI2] URL needs to be trimmed to handle index.html redirection while accessing via knox |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8451](https://issues.apache.org/jira/browse/YARN-8451) | Multiple NM heartbeat thread created when a slow NM resync with RM |  Major | nodemanager | Botong Huang | Botong Huang |
+| [HADOOP-15548](https://issues.apache.org/jira/browse/HADOOP-15548) | Randomize local dirs |  Minor | . | Jim Brennan | Jim Brennan |
+| [HADOOP-15574](https://issues.apache.org/jira/browse/HADOOP-15574) | Suppress build error if there are no docs after excluding private annotations |  Major | . | Takanobu Asanuma | Takanobu Asanuma |
+| [HDFS-13702](https://issues.apache.org/jira/browse/HDFS-13702) | Remove HTrace hooks from DFSClient to reduce CPU usage |  Major | performance | Todd Lipcon | Todd Lipcon |
+| [HDFS-13635](https://issues.apache.org/jira/browse/HDFS-13635) | Incorrect message when block is not found |  Major | datanode | Wei-Chiu Chuang | Gabor Bota |
+| [YARN-8415](https://issues.apache.org/jira/browse/YARN-8415) | TimelineWebServices.getEntity should throw ForbiddenException instead of 404 when ACL checks fail |  Major | . | Sumana Sathish | Suma Shivaprasad |
+| [HDFS-13715](https://issues.apache.org/jira/browse/HDFS-13715) | diskbalancer does not work if one of the blockpools are empty on a Federated cluster |  Major | diskbalancer | Namit Maheshwari | Bharat Viswanadham |
+| [YARN-8459](https://issues.apache.org/jira/browse/YARN-8459) | Improve Capacity Scheduler logs to debug invalid states |  Major | capacity scheduler | Wangda Tan | Wangda Tan |
+| [HADOOP-15571](https://issues.apache.org/jira/browse/HADOOP-15571) | Multiple FileContexts created with the same configuration object should be allowed to have different umask |  Critical | . | Vinod Kumar Vavilapalli | Vinod Kumar Vavilapalli |
+| [HDFS-13121](https://issues.apache.org/jira/browse/HDFS-13121) | NPE when request file descriptors when SC read |  Minor | hdfs-client | Gang Xie | Zsolt Venczel |
+| [YARN-6265](https://issues.apache.org/jira/browse/YARN-6265) | yarn.resourcemanager.fail-fast is used inconsistently |  Major | resourcemanager | Daniel Templeton | Yuanbo Liu |
+| [YARN-8473](https://issues.apache.org/jira/browse/YARN-8473) | Containers being launched as app tears down can leave containers in NEW state |  Major | nodemanager | Jason Lowe | Jason Lowe |
+| [YARN-8512](https://issues.apache.org/jira/browse/YARN-8512) | ATSv2 entities are not published to HBase from second attempt onwards |  Major | . | Yesha Vora | Rohith Sharma K S |
+| [YARN-8491](https://issues.apache.org/jira/browse/YARN-8491) | TestServiceCLI#testEnableFastLaunch fail when umask is 077 |  Major | . | K G Bakthavachalam | K G Bakthavachalam |
+| [HADOOP-15541](https://issues.apache.org/jira/browse/HADOOP-15541) | AWS SDK can mistake stream timeouts for EOF and throw SdkClientExceptions |  Major | fs/s3 | Sean Mackrory | Sean Mackrory |
+| [HDFS-13723](https://issues.apache.org/jira/browse/HDFS-13723) | Occasional "Should be different group" error in TestRefreshUserMappings#testGroupMappingRefresh |  Major | security, test | Siyao Meng | Siyao Meng |
+| [HDFS-12837](https://issues.apache.org/jira/browse/HDFS-12837) | Intermittent failure in TestReencryptionWithKMS |  Major | encryption, test | Surendra Singh Lilhore | Xiao Chen |
+| [HDFS-13729](https://issues.apache.org/jira/browse/HDFS-13729) | Fix broken links to RBF documentation |  Minor | documentation | jwhitter | Gabor Bota |
+| [YARN-8518](https://issues.apache.org/jira/browse/YARN-8518) | test-container-executor test\_is\_empty() is broken |  Major | . | Jim Brennan | Jim Brennan |
+| [YARN-8515](https://issues.apache.org/jira/browse/YARN-8515) | container-executor can crash with SIGPIPE after nodemanager restart |  Major | . | Jim Brennan | Jim Brennan |
+| [YARN-8421](https://issues.apache.org/jira/browse/YARN-8421) | when moving app, activeUsers is increased, even though app does not have outstanding request |  Major | . | kyungwan nam |  |
+| [YARN-8511](https://issues.apache.org/jira/browse/YARN-8511) | When AM releases a container, RM removes allocation tags before it is released by NM |  Major | capacity scheduler | Weiwei Yang | Weiwei Yang |
+| [HDFS-13524](https://issues.apache.org/jira/browse/HDFS-13524) | Occasional "All datanodes are bad" error in TestLargeBlock#testLargeBlockSize |  Major | . | Wei-Chiu Chuang | Siyao Meng |
+| [YARN-8538](https://issues.apache.org/jira/browse/YARN-8538) | Fix valgrind leak check on container executor |  Major | . | Billie Rinaldi | Billie Rinaldi |
+| [HADOOP-15610](https://issues.apache.org/jira/browse/HADOOP-15610) | Hadoop Docker Image Pip Install Fails |  Critical | . | Jack Bearden | Jack Bearden |
+| [HADOOP-15614](https://issues.apache.org/jira/browse/HADOOP-15614) | TestGroupsCaching.testExceptionOnBackgroundRefreshHandled reliably fails |  Major | . | Kihwal Lee | Weiwei Yang |
+| [MAPREDUCE-7118](https://issues.apache.org/jira/browse/MAPREDUCE-7118) | Distributed cache conflicts breaks backwards compatability |  Blocker | mrv2 | Jason Lowe | Jason Lowe |
+| [YARN-8528](https://issues.apache.org/jira/browse/YARN-8528) | Final states in ContainerAllocation might be modified externally causing unexpected allocation results |  Major | capacity scheduler | Xintong Song | Xintong Song |
+| [YARN-8541](https://issues.apache.org/jira/browse/YARN-8541) | RM startup failure on recovery after user deletion |  Blocker | resourcemanager | yimeng | Bibin A Chundatt |
+| [HADOOP-15593](https://issues.apache.org/jira/browse/HADOOP-15593) | UserGroupInformation TGT renewer throws NPE |  Blocker | security | Wei-Chiu Chuang | Gabor Bota |
+| [HDFS-13765](https://issues.apache.org/jira/browse/HDFS-13765) | Fix javadoc for FSDirMkdirOp#createParentDirectories |  Minor | documentation | Lokesh Jain | Lokesh Jain |
+| [YARN-8508](https://issues.apache.org/jira/browse/YARN-8508) | On NodeManager container gets cleaned up before its pid file is created |  Critical | . | Sumana Sathish | Chandni Singh |
+| [YARN-8434](https://issues.apache.org/jira/browse/YARN-8434) | Update federation documentation of Nodemanager configurations |  Minor | . | Bibin A Chundatt | Bibin A Chundatt |
+| [YARN-8591](https://issues.apache.org/jira/browse/YARN-8591) | [ATSv2] NPE while checking for entity acl in non-secure cluster |  Major | timelinereader, timelineserver | Akhil PB | Rohith Sharma K S |
+| [YARN-8558](https://issues.apache.org/jira/browse/YARN-8558) | NM recovery level db not cleaned up properly on container finish |  Critical | . | Bibin A Chundatt | Bibin A Chundatt |
+| [YARN-8418](https://issues.apache.org/jira/browse/YARN-8418) | App local logs could leaked if log aggregation fails to initialize for the app |  Critical | . | Bibin A Chundatt | Bibin A Chundatt |
+| [YARN-8522](https://issues.apache.org/jira/browse/YARN-8522) | Application fails with InvalidResourceRequestException |  Critical | . | Yesha Vora | Zian Chen |
+| [YARN-8606](https://issues.apache.org/jira/browse/YARN-8606) | Opportunistic scheduling does not work post RM failover |  Blocker | . | Bibin A Chundatt | Bibin A Chundatt |
+| [YARN-8600](https://issues.apache.org/jira/browse/YARN-8600) | RegistryDNS hang when remote lookup does not reply |  Critical | yarn | Eric Yang | Eric Yang |
+
+
+### TESTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [MAPREDUCE-7066](https://issues.apache.org/jira/browse/MAPREDUCE-7066) | TestQueue fails on Java9 |  Major | test | Takanobu Asanuma | Takanobu Asanuma |
+| [HADOOP-15313](https://issues.apache.org/jira/browse/HADOOP-15313) | TestKMS should close providers |  Major | kms, test | Xiao Chen | Xiao Chen |
+| [HDFS-13129](https://issues.apache.org/jira/browse/HDFS-13129) | Add a test for DfsAdmin refreshSuperUserGroupsConfiguration |  Minor | namenode | Mukul Kumar Singh | Mukul Kumar Singh |
+| [HDFS-13503](https://issues.apache.org/jira/browse/HDFS-13503) | Fix TestFsck test failures on Windows |  Major | hdfs | Xiao Liang | Xiao Liang |
+| [HDFS-13315](https://issues.apache.org/jira/browse/HDFS-13315) | Add a test for the issue reported in HDFS-11481 which is fixed by HDFS-10997. |  Major | . | Yongjun Zhang | Yongjun Zhang |
+| [HDFS-13542](https://issues.apache.org/jira/browse/HDFS-13542) | TestBlockManager#testNeededReplicationWhileAppending fails due to improper cluster shutdown in TestBlockManager#testBlockManagerMachinesArray on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13551](https://issues.apache.org/jira/browse/HDFS-13551) | TestMiniDFSCluster#testClusterSetStorageCapacity does not shut down cluster |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-11700](https://issues.apache.org/jira/browse/HDFS-11700) | TestHDFSServerPorts#testBackupNodePorts doesn't pass on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13548](https://issues.apache.org/jira/browse/HDFS-13548) | TestResolveHdfsSymlink#testFcResolveAfs fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13567](https://issues.apache.org/jira/browse/HDFS-13567) | TestNameNodeMetrics#testGenerateEDEKTime,TestNameNodeMetrics#testResourceCheck should use a different cluster basedir |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13557](https://issues.apache.org/jira/browse/HDFS-13557) | TestDFSAdmin#testListOpenFiles fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13559](https://issues.apache.org/jira/browse/HDFS-13559) | TestBlockScanner does not close TestContext properly |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13570](https://issues.apache.org/jira/browse/HDFS-13570) | TestQuotaByStorageType,TestQuota,TestDFSOutputStream fail on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13558](https://issues.apache.org/jira/browse/HDFS-13558) | TestDatanodeHttpXFrame does not shut down cluster |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13554](https://issues.apache.org/jira/browse/HDFS-13554) | TestDatanodeRegistration#testForcedRegistration does not shut down cluster |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13556](https://issues.apache.org/jira/browse/HDFS-13556) | TestNestedEncryptionZones does not shut down cluster |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13560](https://issues.apache.org/jira/browse/HDFS-13560) | Insufficient system resources exist to complete the requested service for some tests on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13592](https://issues.apache.org/jira/browse/HDFS-13592) | TestNameNodePrunesMissingStorages#testNameNodePrunesUnreportedStorages does not shut down cluster properly |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13593](https://issues.apache.org/jira/browse/HDFS-13593) | TestBlockReaderLocalLegacy#testBlockReaderLocalLegacyWithAppend fails on Windows |  Minor | test | Anbang Hu | Anbang Hu |
+| [HDFS-13587](https://issues.apache.org/jira/browse/HDFS-13587) | TestQuorumJournalManager fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13619](https://issues.apache.org/jira/browse/HDFS-13619) | TestAuditLoggerWithCommands fails on Windows |  Minor | test | Anbang Hu | Anbang Hu |
+| [HDFS-13620](https://issues.apache.org/jira/browse/HDFS-13620) | Randomize the test directory path for TestHDFSFileSystemContract |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13627](https://issues.apache.org/jira/browse/HDFS-13627) | TestErasureCodingExerciseAPIs fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13591](https://issues.apache.org/jira/browse/HDFS-13591) | TestDFSShell#testSetrepLow fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13632](https://issues.apache.org/jira/browse/HDFS-13632) | Randomize baseDir for MiniJournalCluster in MiniQJMHACluster for TestDFSAdminWithHA |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13629](https://issues.apache.org/jira/browse/HDFS-13629) | Some tests in TestDiskBalancerCommand fail on Windows due to MiniDFSCluster path conflict and improper path usage |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13631](https://issues.apache.org/jira/browse/HDFS-13631) | TestDFSAdmin#testCheckNumOfBlocksInReportCommand should use a separate MiniDFSCluster path |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13651](https://issues.apache.org/jira/browse/HDFS-13651) | TestReencryptionHandler fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13648](https://issues.apache.org/jira/browse/HDFS-13648) | Fix TestGetConf#testGetJournalNodes on Windows due to a mismatch line separator |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [MAPREDUCE-7102](https://issues.apache.org/jira/browse/MAPREDUCE-7102) | Fix TestJavaSerialization for Windows due a mismatch line separator |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [MAPREDUCE-7105](https://issues.apache.org/jira/browse/MAPREDUCE-7105) | Fix TestNativeCollectorOnlyHandler.testOnCall on Windows because of the path format |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HDFS-13652](https://issues.apache.org/jira/browse/HDFS-13652) | Randomize baseDir for MiniDFSCluster in TestBlockScanner |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13649](https://issues.apache.org/jira/browse/HDFS-13649) | Randomize baseDir for MiniDFSCluster in TestReconstructStripedFile and TestReconstructStripedFileWithRandomECPolicy |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13650](https://issues.apache.org/jira/browse/HDFS-13650) | Randomize baseDir for MiniDFSCluster in TestDFSStripedInputStream and TestDFSStripedInputStreamWithRandomECPolicy |  Minor | . | Anbang Hu | Anbang Hu |
+| [YARN-8370](https://issues.apache.org/jira/browse/YARN-8370) | Some Node Manager tests fail on Windows due to improper path/file separator |  Minor | . | Anbang Hu | Anbang Hu |
+| [YARN-8422](https://issues.apache.org/jira/browse/YARN-8422) | TestAMSimulator failing with NPE |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HADOOP-15532](https://issues.apache.org/jira/browse/HADOOP-15532) | TestBasicDiskValidator fails with NoSuchFileException |  Minor | . | Íñigo Goiri | Giovanni Matteo Fumarola |
+| [HDFS-13563](https://issues.apache.org/jira/browse/HDFS-13563) | TestDFSAdminWithHA times out on Windows |  Minor | . | Anbang Hu | Lukas Majercak |
+| [HDFS-13681](https://issues.apache.org/jira/browse/HDFS-13681) | Fix TestStartup.testNNFailToStartOnReadOnlyNNDir test failure on Windows |  Major | test | Xiao Liang | Xiao Liang |
+
+
+### SUB-TASKS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-8002](https://issues.apache.org/jira/browse/YARN-8002) | Support NOT\_SELF and ALL namespace types for allocation tag |  Major | resourcemanager | Weiwei Yang | Weiwei Yang |
+| [HDFS-13289](https://issues.apache.org/jira/browse/HDFS-13289) | RBF: TestConnectionManager#testCleanup() test case need correction |  Minor | . | Dibyendu Karmakar | Dibyendu Karmakar |
+| [YARN-8013](https://issues.apache.org/jira/browse/YARN-8013) | Support application tags when defining application namespaces for placement constraints |  Major | . | Weiwei Yang | Weiwei Yang |
+| [YARN-6936](https://issues.apache.org/jira/browse/YARN-6936) | [Atsv2] Retrospect storing entities into sub application table from client perspective |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [HDFS-13353](https://issues.apache.org/jira/browse/HDFS-13353) | RBF: TestRouterWebHDFSContractCreate failed |  Major | test | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-8107](https://issues.apache.org/jira/browse/YARN-8107) | Give an informative message when incorrect format is used in ATSv2 filter attributes |  Major | ATSv2 | Charan Hebri | Rohith Sharma K S |
+| [YARN-8110](https://issues.apache.org/jira/browse/YARN-8110) | AMRMProxy recover should catch for all throwable to avoid premature exit |  Major | . | Botong Huang | Botong Huang |
+| [YARN-8048](https://issues.apache.org/jira/browse/YARN-8048) | Support auto-spawning of admin configured services during bootstrap of rm/apiserver |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [HDFS-13402](https://issues.apache.org/jira/browse/HDFS-13402) | RBF: Fix  java doc for StateStoreFileSystemImpl |  Minor | hdfs | Yiran Wu | Yiran Wu |
+| [YARN-7574](https://issues.apache.org/jira/browse/YARN-7574) | Add support for Node Labels on Auto Created Leaf Queue Template |  Major | capacity scheduler | Suma Shivaprasad | Suma Shivaprasad |
+| [HDFS-13410](https://issues.apache.org/jira/browse/HDFS-13410) | RBF: Support federation with no subclusters |  Minor | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-13384](https://issues.apache.org/jira/browse/HDFS-13384) | RBF: Improve timeout RPC call mechanism |  Minor | . | Íñigo Goiri | Íñigo Goiri |
+| [HADOOP-15376](https://issues.apache.org/jira/browse/HADOOP-15376) | Remove double semi colons on imports that make Clover fall over. |  Minor | . | Ewan Higgs | Ewan Higgs |
+| [YARN-7973](https://issues.apache.org/jira/browse/YARN-7973) | Support ContainerRelaunch for Docker containers |  Major | . | Shane Kumpf | Shane Kumpf |
+| [YARN-7941](https://issues.apache.org/jira/browse/YARN-7941) | Transitive dependencies for component are not resolved |  Major | . | Rohith Sharma K S | Billie Rinaldi |
+| [HADOOP-15346](https://issues.apache.org/jira/browse/HADOOP-15346) | S3ARetryPolicy for 400/BadArgument to be "fail" |  Major | fs/s3 | Steve Loughran | Steve Loughran |
+| [HDFS-13045](https://issues.apache.org/jira/browse/HDFS-13045) | RBF: Improve error message returned from subcluster |  Minor | . | Wei Yan | Íñigo Goiri |
+| [HDFS-13428](https://issues.apache.org/jira/browse/HDFS-13428) | RBF: Remove LinkedList From StateStoreFileImpl.java |  Trivial | federation | BELUGA BEHR | BELUGA BEHR |
+| [HDFS-13386](https://issues.apache.org/jira/browse/HDFS-13386) | RBF: Wrong date information in list file(-ls) result |  Minor | . | Dibyendu Karmakar | Dibyendu Karmakar |
+| [YARN-7221](https://issues.apache.org/jira/browse/YARN-7221) | Add security check for privileged docker container |  Major | security | Eric Yang | Eric Yang |
+| [YARN-7936](https://issues.apache.org/jira/browse/YARN-7936) | Add default service AM Xmx |  Major | . | Jian He | Jian He |
+| [YARN-8018](https://issues.apache.org/jira/browse/YARN-8018) | Yarn Service Upgrade: Add support for initiating service upgrade |  Major | . | Chandni Singh | Chandni Singh |
+| [HADOOP-14999](https://issues.apache.org/jira/browse/HADOOP-14999) | AliyunOSS: provide one asynchronous multi-part based uploading mechanism |  Major | fs/oss | Genmao Yu | Genmao Yu |
+| [YARN-7142](https://issues.apache.org/jira/browse/YARN-7142) | Support placement policy in yarn native services |  Major | yarn-native-services | Billie Rinaldi | Gour Saha |
+| [YARN-8138](https://issues.apache.org/jira/browse/YARN-8138) | Add unit test to validate queue priority preemption works under node partition. |  Minor | . | Charan Hebri | Zian Chen |
+| [YARN-8060](https://issues.apache.org/jira/browse/YARN-8060) | Create default readiness check for service components |  Major | yarn-native-services | Billie Rinaldi | Billie Rinaldi |
+| [HDFS-13435](https://issues.apache.org/jira/browse/HDFS-13435) | RBF: Improve the error loggings for printing the stack trace |  Major | . | Yiqun Lin | Yiqun Lin |
+| [YARN-8126](https://issues.apache.org/jira/browse/YARN-8126) | Support auto-spawning of admin configured services during bootstrap of RM |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-7996](https://issues.apache.org/jira/browse/YARN-7996) | Allow user supplied Docker client configurations with YARN native services |  Major | . | Shane Kumpf | Shane Kumpf |
+| [HDFS-13466](https://issues.apache.org/jira/browse/HDFS-13466) | RBF: Add more router-related information to the UI |  Minor | . | Wei Yan | Wei Yan |
+| [YARN-5888](https://issues.apache.org/jira/browse/YARN-5888) | [UI2] Improve unit tests for new YARN UI |  Minor | yarn-ui-v2 | Akhil PB | Akhil PB |
+| [HDFS-13453](https://issues.apache.org/jira/browse/HDFS-13453) | RBF: getMountPointDates should fetch latest subdir time/date when parent dir is not present but /parent/child dirs are present in mount table |  Major | . | Dibyendu Karmakar | Dibyendu Karmakar |
+| [YARN-8111](https://issues.apache.org/jira/browse/YARN-8111) | Simplify PlacementConstraints API by removing allocationTagToIntraApp |  Minor | resourcemanager | Weiwei Yang | Weiwei Yang |
+| [YARN-8064](https://issues.apache.org/jira/browse/YARN-8064) | Docker ".cmd" files should not be put in hadoop.tmp.dir |  Critical | . | Eric Badger | Eric Badger |
+| [HDFS-13478](https://issues.apache.org/jira/browse/HDFS-13478) | RBF: Disabled Nameservice store API |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-8177](https://issues.apache.org/jira/browse/YARN-8177) | Documentation changes for auto creation of Leaf Queues with node label |  Major | . | Suma Shivaprasad | Suma Shivaprasad |
+| [HDFS-13490](https://issues.apache.org/jira/browse/HDFS-13490) | RBF: Fix setSafeMode in the Router |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-13484](https://issues.apache.org/jira/browse/HDFS-13484) | RBF: Disable Nameservices from the federation |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-7939](https://issues.apache.org/jira/browse/YARN-7939) | Yarn Service Upgrade: add support to upgrade a component instance |  Major | . | Chandni Singh | Chandni Singh |
+| [HDFS-13326](https://issues.apache.org/jira/browse/HDFS-13326) | RBF: Improve the interfaces to modify and view mount tables |  Minor | . | Wei Yan | Gang Li |
+| [YARN-8122](https://issues.apache.org/jira/browse/YARN-8122) | Component health threshold monitor |  Major | . | Gour Saha | Gour Saha |
+| [HDFS-13499](https://issues.apache.org/jira/browse/HDFS-13499) | RBF: Show disabled name services in the UI |  Minor | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-8215](https://issues.apache.org/jira/browse/YARN-8215) | ATS v2 returns invalid YARN\_CONTAINER\_ALLOCATED\_HOST\_HTTP\_ADDRESS from NM |  Critical | ATSv2 | Yesha Vora | Rohith Sharma K S |
+| [YARN-8152](https://issues.apache.org/jira/browse/YARN-8152) | Add chart in SLS to illustrate the throughput of the scheduler |  Major | scheduler-load-simulator | Weiwei Yang | Tao Yang |
+| [YARN-8204](https://issues.apache.org/jira/browse/YARN-8204) | Yarn Service Upgrade: Add a flag to disable upgrade |  Major | . | Chandni Singh | Chandni Singh |
+| [YARN-7781](https://issues.apache.org/jira/browse/YARN-7781) | Update YARN-Services-Examples.md to be in sync with the latest code |  Major | . | Gour Saha | Gour Saha |
+| [HDFS-13508](https://issues.apache.org/jira/browse/HDFS-13508) | RBF: Normalize paths (automatically) when adding, updating, removing or listing mount table entries |  Minor | . | Ekanth Sethuramalingam | Ekanth Sethuramalingam |
+| [HDFS-13434](https://issues.apache.org/jira/browse/HDFS-13434) | RBF: Fix dead links in RBF document |  Major | documentation | Akira Ajisaka | Chetna Chaudhari |
+| [YARN-8195](https://issues.apache.org/jira/browse/YARN-8195) | Fix constraint cardinality check in the presence of multiple target allocation tags |  Critical | . | Weiwei Yang | Weiwei Yang |
+| [YARN-8228](https://issues.apache.org/jira/browse/YARN-8228) | Docker does not support hostnames greater than 64 characters |  Critical | yarn-native-services | Yesha Vora | Shane Kumpf |
+| [YARN-8212](https://issues.apache.org/jira/browse/YARN-8212) | Pending backlog for async allocation threads should be configurable |  Major | . | Weiwei Yang | Tao Yang |
+| [YARN-2674](https://issues.apache.org/jira/browse/YARN-2674) | Distributed shell AM may re-launch containers if RM work preserving restart happens |  Major | applications, resourcemanager | Chun Chen | Shane Kumpf |
+| [HDFS-13488](https://issues.apache.org/jira/browse/HDFS-13488) | RBF: Reject requests when a Router is overloaded |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-8113](https://issues.apache.org/jira/browse/YARN-8113) | Update placement constraints doc with application namespaces and inter-app constraints |  Major | documentation | Weiwei Yang | Weiwei Yang |
+| [YARN-8194](https://issues.apache.org/jira/browse/YARN-8194) | Exception when reinitializing a container using LinuxContainerExecutor |  Blocker | . | Chandni Singh | Chandni Singh |
+| [YARN-7961](https://issues.apache.org/jira/browse/YARN-7961) | Improve status response when yarn application is destroyed |  Major | yarn-native-services | Yesha Vora | Gour Saha |
+| [HDFS-13525](https://issues.apache.org/jira/browse/HDFS-13525) | RBF: Add unit test TestStateStoreDisabledNameservice |  Major | . | Yiqun Lin | Yiqun Lin |
+| [YARN-5151](https://issues.apache.org/jira/browse/YARN-5151) | [UI2] Support kill application from new YARN UI |  Major | . | Wangda Tan | Gergely Novák |
+| [YARN-8253](https://issues.apache.org/jira/browse/YARN-8253) | HTTPS Ats v2 api call fails with "bad HTTP parsed" |  Critical | ATSv2 | Yesha Vora | Charan Hebri |
+| [YARN-8207](https://issues.apache.org/jira/browse/YARN-8207) | Docker container launch use popen have risk of shell expansion |  Blocker | yarn-native-services | Eric Yang | Eric Yang |
+| [YARN-8261](https://issues.apache.org/jira/browse/YARN-8261) | Docker container launch fails due to .cmd file creation failure |  Blocker | . | Eric Badger | Jason Lowe |
+| [HADOOP-15454](https://issues.apache.org/jira/browse/HADOOP-15454) | TestRollingFileSystemSinkWithLocal fails on Windows |  Major | test | Xiao Liang | Xiao Liang |
+| [HDFS-13346](https://issues.apache.org/jira/browse/HDFS-13346) | RBF: Fix synchronization of router quota and nameservice quota |  Major | . | liuhongtong | Yiqun Lin |
+| [YARN-8243](https://issues.apache.org/jira/browse/YARN-8243) | Flex down should remove instance with largest component instance ID first |  Critical | yarn-native-services | Gour Saha | Gour Saha |
+| [YARN-7654](https://issues.apache.org/jira/browse/YARN-7654) | Support ENTRY\_POINT for docker container |  Blocker | yarn | Eric Yang | Eric Yang |
+| [YARN-8247](https://issues.apache.org/jira/browse/YARN-8247) | Incorrect HTTP status code returned by ATSv2 for non-whitelisted users |  Critical | ATSv2 | Charan Hebri | Rohith Sharma K S |
+| [YARN-8130](https://issues.apache.org/jira/browse/YARN-8130) | Race condition when container events are published for KILLED applications |  Major | ATSv2 | Charan Hebri | Rohith Sharma K S |
+| [YARN-8081](https://issues.apache.org/jira/browse/YARN-8081) | Yarn Service Upgrade: Add support to upgrade a component |  Major | . | Chandni Singh | Chandni Singh |
+| [YARN-8284](https://issues.apache.org/jira/browse/YARN-8284) | get\_docker\_command refactoring |  Minor | . | Jason Lowe | Eric Badger |
+| [HADOOP-15469](https://issues.apache.org/jira/browse/HADOOP-15469) | S3A directory committer commit job fails if \_temporary directory created under dest |  Major | fs/s3 | Steve Loughran | Steve Loughran |
+| [YARN-8206](https://issues.apache.org/jira/browse/YARN-8206) | Sending a kill does not immediately kill docker containers |  Major | . | Eric Badger | Eric Badger |
+| [YARN-7960](https://issues.apache.org/jira/browse/YARN-7960) | Add no-new-privileges flag to docker run |  Major | . | Eric Badger | Eric Badger |
+| [YARN-7530](https://issues.apache.org/jira/browse/YARN-7530) | hadoop-yarn-services-api should be part of hadoop-yarn-services |  Blocker | yarn-native-services | Eric Yang | Chandni Singh |
+| [YARN-6919](https://issues.apache.org/jira/browse/YARN-6919) | Add default volume mount list |  Major | yarn | Eric Badger | Eric Badger |
+| [HADOOP-15498](https://issues.apache.org/jira/browse/HADOOP-15498) | TestHadoopArchiveLogs (#testGenerateScript, #testPrepareWorkingDir) fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [YARN-8329](https://issues.apache.org/jira/browse/YARN-8329) | Docker client configuration can still be set incorrectly |  Major | . | Shane Kumpf | Shane Kumpf |
+| [HDFS-12978](https://issues.apache.org/jira/browse/HDFS-12978) | Fine-grained locking while consuming journal stream. |  Major | namenode | Konstantin Shvachko | Konstantin Shvachko |
+| [YARN-8384](https://issues.apache.org/jira/browse/YARN-8384) | stdout.txt, stderr.txt logs of a launched docker container is coming with primary group of submit user instead of hadoop |  Critical | yarn-native-services | Sunil Govindan | Eric Yang |
+| [YARN-8349](https://issues.apache.org/jira/browse/YARN-8349) | Remove YARN registry entries when a service is killed by the RM |  Critical | yarn-native-services | Shane Kumpf | Billie Rinaldi |
+| [HDFS-13637](https://issues.apache.org/jira/browse/HDFS-13637) | RBF: Router fails when threadIndex (in ConnectionPool) wraps around Integer.MIN\_VALUE |  Critical | federation | CR Hota | CR Hota |
+| [YARN-8342](https://issues.apache.org/jira/browse/YARN-8342) | Using docker image from a non-privileged registry, the launch\_command is not honored |  Critical | . | Wangda Tan | Eric Yang |
+| [HDFS-13281](https://issues.apache.org/jira/browse/HDFS-13281) | Namenode#createFile should be /.reserved/raw/ aware. |  Critical | encryption | Rushabh S Shah | Rushabh S Shah |
+| [YARN-4677](https://issues.apache.org/jira/browse/YARN-4677) | RMNodeResourceUpdateEvent update from scheduler can lead to race condition |  Major | graceful, resourcemanager, scheduler | Brook Zhou | Wilfred Spiegelenburg |
+| [HADOOP-15137](https://issues.apache.org/jira/browse/HADOOP-15137) | ClassNotFoundException: org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol when using hadoop-client-minicluster |  Major | . | Jeff Zhang | Bharat Viswanadham |
+| [HDFS-13547](https://issues.apache.org/jira/browse/HDFS-13547) | Add ingress port based sasl resolver |  Major | security | Chen Liang | Chen Liang |
+| [HADOOP-15514](https://issues.apache.org/jira/browse/HADOOP-15514) | NoClassDefFoundError for TimelineCollectorManager when starting MiniYARNCluster |  Major | . | Jeff Zhang | Rohith Sharma K S |
+| [HADOOP-15516](https://issues.apache.org/jira/browse/HADOOP-15516) | Add test cases to cover FileUtil#readLink |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HADOOP-15506](https://issues.apache.org/jira/browse/HADOOP-15506) | Upgrade Azure Storage Sdk version to 7.0.0 and update corresponding code blocks |  Minor | fs/azure | Esfandiar Manii | Esfandiar Manii |
+| [HADOOP-15529](https://issues.apache.org/jira/browse/HADOOP-15529) | ContainerLaunch#testInvalidEnvVariableSubstitutionType is not supported in Windows |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-8411](https://issues.apache.org/jira/browse/YARN-8411) | Enable stopped system services to be started during RM start |  Critical | . | Billie Rinaldi | Billie Rinaldi |
+| [YARN-8259](https://issues.apache.org/jira/browse/YARN-8259) | Revisit liveliness checks for Docker containers |  Blocker | . | Shane Kumpf | Shane Kumpf |
+| [HADOOP-15533](https://issues.apache.org/jira/browse/HADOOP-15533) | Make WASB listStatus messages consistent |  Trivial | fs/azure | Esfandiar Manii | Esfandiar Manii |
+| [HADOOP-15458](https://issues.apache.org/jira/browse/HADOOP-15458) | TestLocalFileSystem#testFSOutputStreamBuilder fails on Windows |  Minor | test | Xiao Liang | Xiao Liang |
+| [YARN-8465](https://issues.apache.org/jira/browse/YARN-8465) | Dshell docker container gets marked as lost after NM restart |  Major | yarn-native-services | Yesha Vora | Shane Kumpf |
+| [YARN-8485](https://issues.apache.org/jira/browse/YARN-8485) | Priviledged container app launch is failing intermittently |  Major | yarn-native-services | Yesha Vora | Eric Yang |
+| [HDFS-13528](https://issues.apache.org/jira/browse/HDFS-13528) | RBF: If a directory exceeds quota limit then quota usage is not refreshed for other mount entries |  Major | . | Dibyendu Karmakar | Dibyendu Karmakar |
+| [HDFS-13710](https://issues.apache.org/jira/browse/HDFS-13710) | RBF:  setQuota and getQuotaUsage should check the dfs.federation.router.quota.enable |  Major | federation, hdfs | yanghuafeng | yanghuafeng |
+| [HADOOP-15384](https://issues.apache.org/jira/browse/HADOOP-15384) | distcp numListstatusThreads option doesn't get to -delete scan |  Major | tools/distcp | Steve Loughran | Steve Loughran |
+| [HDFS-13726](https://issues.apache.org/jira/browse/HDFS-13726) | RBF: Fix RBF configuration links |  Minor | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [HDFS-13475](https://issues.apache.org/jira/browse/HDFS-13475) | RBF: Admin cannot enforce Router enter SafeMode |  Major | . | Wei Yan | Chao Sun |
+| [HDFS-13733](https://issues.apache.org/jira/browse/HDFS-13733) | RBF: Add Web UI configurations and descriptions to RBF document |  Minor | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-8301](https://issues.apache.org/jira/browse/YARN-8301) | Yarn Service Upgrade: Add documentation |  Critical | . | Chandni Singh | Chandni Singh |
+| [YARN-8546](https://issues.apache.org/jira/browse/YARN-8546) | Resource leak caused by a reserved container being released more than once under async scheduling |  Major | capacity scheduler | Weiwei Yang | Tao Yang |
+
+
+### OTHER:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-8091](https://issues.apache.org/jira/browse/YARN-8091) | Revisit checkUserAccessToQueue RM REST API |  Critical | . | Wangda Tan | Wangda Tan |
+| [YARN-8274](https://issues.apache.org/jira/browse/YARN-8274) | Docker command error during container relaunch |  Critical | . | Billie Rinaldi | Jason Lowe |
+| [YARN-8080](https://issues.apache.org/jira/browse/YARN-8080) | YARN native service should support component restart policy |  Critical | . | Wangda Tan | Suma Shivaprasad |
+| [HADOOP-15483](https://issues.apache.org/jira/browse/HADOOP-15483) | Upgrade jquery to version 3.3.1 |  Major | . | Lokesh Jain | Lokesh Jain |
+| [YARN-8506](https://issues.apache.org/jira/browse/YARN-8506) | Make GetApplicationsRequestPBImpl thread safe |  Critical | . | Wangda Tan | Wangda Tan |
+
+


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


[05/50] [abbrv] hadoop git commit: Make 3.1.1 awared by other branches

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49c68760/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.1.1.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.1.1.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.1.1.xml
new file mode 100644
index 0000000..80c4946
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.1.1.xml
@@ -0,0 +1,676 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Thu Aug 02 05:10:01 UTC 2018 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop HDFS 3.1.1"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-hdfs-project/hadoop-hdfs/target/hadoop-annotations.jar:/build/source/hadoop-hdfs-project/hadoop-hdfs/target/jdiff.jar -verbose -classpath /build/source/hadoop-hdfs-project/hadoop-hdfs/target/classes:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-3.1.1.jar:/usr/lib/jvm/java-8-openjdk-amd64/lib/tools.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-3.1.1.jar:/maven/org/slf4j/slf4j-api/1.7.25/slf4j-api-1.7.25.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/com/nimbusds/nimbus-jose-jwt/4.41.1/nimbus-jose-jwt-4.41.1.jar:/maven/com/github/stephenc/jcip/jcip-annotations/1.0-1/jcip-annotations-1.0-1.jar:/maven/net/minidev/json-smart/2.3/json-smart-2.3.jar:/maven/net/minidev/accessors-smart/1
 .2/accessors-smart-1.2.jar:/maven/org/ow2/asm/asm/5.0.4/asm-5.0.4.jar:/maven/org/apache/zookeeper/zookeeper/3.4.9/zookeeper-3.4.9.jar:/maven/org/apache/curator/curator-framework/2.12.0/curator-framework-2.12.0.jar:/maven/org/apache/kerby/kerb-simplekdc/1.0.1/kerb-simplekdc-1.0.1.jar:/maven/org/apache/kerby/kerb-client/1.0.1/kerb-client-1.0.1.jar:/maven/org/apache/kerby/kerby-config/1.0.1/kerby-config-1.0.1.jar:/maven/org/apache/kerby/kerb-core/1.0.1/kerb-core-1.0.1.jar:/maven/org/apache/kerby/kerby-pkix/1.0.1/kerby-pkix-1.0.1.jar:/maven/org/apache/kerby/kerby-asn1/1.0.1/kerby-asn1-1.0.1.jar:/maven/org/apache/kerby/kerby-util/1.0.1/kerby-util-1.0.1.jar:/maven/org/apache/kerby/kerb-common/1.0.1/kerb-common-1.0.1.jar:/maven/org/apache/kerby/kerb-crypto/1.0.1/kerb-crypto-1.0.1.jar:/maven/org/apache/kerby/kerb-util/1.0.1/kerb-util-1.0.1.jar:/maven/org/apache/kerby/token-provider/1.0.1/token-provider-1.0.1.jar:/maven/org/apache/kerby/kerb-admin/1.0.1/kerb-admin-1.0.1.jar:/maven/org/apache
 /kerby/kerb-server/1.0.1/kerb-server-1.0.1.jar:/maven/org/apache/kerby/kerb-identity/1.0.1/kerb-identity-1.0.1.jar:/maven/org/apache/kerby/kerby-xdr/1.0.1/kerby-xdr-1.0.1.jar:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-3.1.1.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/commons-net/commons-net/3.6/commons-net-3.6.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/maven/org/eclipse/jetty/jetty-servlet/9.3.19.v20170502/jetty-servlet-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-security/9.3.19.v20170502/jetty-security-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-webapp/9.3.19.v20170502/jetty-webapp-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-xml/9.3.19.v20170502/jetty-xml-9.3.19.v20170502.jar:/maven/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/com/sun/jersey/jersey-servlet/1.19/jersey-servlet-1.19.jar:/maven/com/sun/jersey/jersey-json/1.19/jersey-json-1.19.ja
 r:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/maven/javax/xml/bind/jaxb-api/2.2.11/jaxb-api-2.2.11.jar:/maven/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar:/maven/org/apache/commons/commons-configuration2/2.1.1/commons-configuration2-2.1.1.jar:/maven/org/apache/commons/commons-lang3/3.4/commons-lang3-3.4.jar:/maven/org/apache/avro/avro/1.7.7/avro-1.7.7.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy-java/1.0.5/snappy-java-1.0.5.jar:/maven/com/google/re2j/re2j/1.1/re2j-1.1.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.ja
 r:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.12.0/curator-client-2.12.0.jar:/maven/org/apache/curator/curator-recipes/2.12.0/curator-recipes-2.12.0.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/maven/org/codehaus/woodstox/stax2-api/3.1.4/stax2-api-3.1.4.jar:/maven/com/fasterxml/woodstox/woodstox-core/5.0.3/woodstox-core-5.0.3.jar:/build/source/hadoop-hdfs-project/hadoop-hdfs-client/target/hadoop-hdfs-client-3.1.1.jar:/maven/com/squareup/okhttp/okhttp/2.7.5/okhttp-2.7.5.jar:/maven/com/squareup/okio/okio/1.6.0/okio-1.6.0.jar:/maven/com/fasterxml/jackson/core/jackson-annotations/2.7.8/jackson-annotations-2.7.8.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/org/eclipse/jetty/jetty-server/9.3.19.v20170502/jetty-server-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-http/9.3.19.v20170502/je
 tty-http-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-io/9.3.19.v20170502/jetty-io-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-util/9.3.19.v20170502/jetty-util-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-util-ajax/9.3.19.v20170502/jetty-util-ajax-9.3.19.v20170502.jar:/maven/com/sun/jersey/jersey-core/1.19/jersey-core-1.19.jar:/maven/javax/ws/rs/jsr311-api/1.1.1/jsr311-api-1.1.1.jar:/maven/com/sun/jersey/jersey-server/1.19/jersey-server-1.19.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/commons-codec/commons-codec/1.11/commons-codec-1.11.jar:/maven/commons-io/commons-io/2.5/commons-io-2.5.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/maven/commons-daemon/commons-daemon/1.0.13/commons-daemon-1.0.13.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/javax/servlet/javax.servlet-api/3.1
 .0/javax.servlet-api-3.1.0.jar:/maven/org/slf4j/slf4j-log4j12/1.7.25/slf4j-log4j12-1.7.25.jar:/maven/io/netty/netty/3.10.5.Final/netty-3.10.5.Final.jar:/maven/io/netty/netty-all/4.0.52.Final/netty-all-4.0.52.Final.jar:/maven/org/apache/htrace/htrace-core4/4.1.0-incubating/htrace-core4-4.1.0-incubating.jar:/maven/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/maven/com/fasterxml/jackson/core/jackson-databind/2.7.8/jackson-databind-2.7.8.jar:/maven/com/fasterxml/jackson/core/jackson-core/2.7.8/jackson-core-2.7.8.jar:/maven/xerces/xercesImpl/2.11.0/xercesImpl-2.11.0.jar:/maven/xml-apis/xml-apis/1.4.01/xml-apis-1.4.01.jar -sourcepath /build/source/hadoop-hdfs-project/hadoop-hdfs/src/main/java -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-hdfs-project/hadoop-hdfs/target/hadoop-annotations.jar:/build/source/hadoop-hdfs-project/hadoop-hdfs/target/jdiff.jar -apidir /build/source/hadoop-hdfs-project/h
 adoop-hdfs/target/site/jdiff/xml -apiname Apache Hadoop HDFS 3.1.1 -->
+<package name="org.apache.hadoop.hdfs">
+  <doc>
+  <![CDATA[<p>A distributed implementation of {@link
+org.apache.hadoop.fs.FileSystem}.  This is loosely modelled after
+Google's <a href="http://research.google.com/archive/gfs.html">GFS</a>.</p>
+
+<p>The most important difference is that unlike GFS, Hadoop DFS files 
+have strictly one writer at any one time.  Bytes are always appended 
+to the end of the writer's stream.  There is no notion of "record appends"
+or "mutations" that are then checked or reordered.  Writers simply emit 
+a byte stream.  That byte stream is guaranteed to be stored in the 
+order written.</p>]]>
+  </doc>
+</package>
+<package name="org.apache.hadoop.hdfs.net">
+</package>
+<package name="org.apache.hadoop.hdfs.protocol">
+</package>
+<package name="org.apache.hadoop.hdfs.protocol.datatransfer">
+</package>
+<package name="org.apache.hadoop.hdfs.protocol.datatransfer.sasl">
+</package>
+<package name="org.apache.hadoop.hdfs.protocolPB">
+</package>
+<package name="org.apache.hadoop.hdfs.qjournal.client">
+</package>
+<package name="org.apache.hadoop.hdfs.qjournal.protocol">
+</package>
+<package name="org.apache.hadoop.hdfs.qjournal.protocolPB">
+</package>
+<package name="org.apache.hadoop.hdfs.qjournal.server">
+  <!-- start interface org.apache.hadoop.hdfs.qjournal.server.JournalNodeMXBean -->
+  <interface name="JournalNodeMXBean"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getJournalsStatus" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get status information (e.g., whether formatted) of JournalNode's journals.
+ 
+ @return A string presenting status for each journal]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[This is the JMX management interface for JournalNode information]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.hdfs.qjournal.server.JournalNodeMXBean -->
+</package>
+<package name="org.apache.hadoop.hdfs.security.token.block">
+</package>
+<package name="org.apache.hadoop.hdfs.security.token.delegation">
+</package>
+<package name="org.apache.hadoop.hdfs.server.aliasmap">
+  <!-- start class org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap -->
+  <class name="InMemoryAliasMap" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol"/>
+    <implements name="org.apache.hadoop.conf.Configurable"/>
+    <method name="setConf"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getConf" return="org.apache.hadoop.conf.Configuration"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="init" return="org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="blockPoolID" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="list" return="org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol.IterationResult"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="marker" type="java.util.Optional"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="read" return="java.util.Optional"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="block" type="org.apache.hadoop.hdfs.protocol.Block"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="block" type="org.apache.hadoop.hdfs.protocol.Block"/>
+      <param name="providedStorageLocation" type="org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getBlockPoolId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="fromProvidedStorageLocationBytes" return="org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="providedStorageLocationDbFormat" type="byte[]"/>
+      <exception name="InvalidProtocolBufferException" type="com.google.protobuf.InvalidProtocolBufferException"/>
+    </method>
+    <method name="fromBlockBytes" return="org.apache.hadoop.hdfs.protocol.Block"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="blockDbFormat" type="byte[]"/>
+      <exception name="InvalidProtocolBufferException" type="com.google.protobuf.InvalidProtocolBufferException"/>
+    </method>
+    <method name="toProtoBufBytes" return="byte[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="providedStorageLocation" type="org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="toProtoBufBytes" return="byte[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="block" type="org.apache.hadoop.hdfs.protocol.Block"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <doc>
+    <![CDATA[InMemoryAliasMap is an implementation of the InMemoryAliasMapProtocol for
+ use with LevelDB.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap -->
+</package>
+<package name="org.apache.hadoop.hdfs.server.balancer">
+</package>
+<package name="org.apache.hadoop.hdfs.server.blockmanagement">
+</package>
+<package name="org.apache.hadoop.hdfs.server.common">
+  <!-- start interface org.apache.hadoop.hdfs.server.common.BlockAlias -->
+  <interface name="BlockAlias"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getBlock" return="org.apache.hadoop.hdfs.protocol.Block"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Interface used to load provided blocks.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.hdfs.server.common.BlockAlias -->
+  <!-- start class org.apache.hadoop.hdfs.server.common.FileRegion -->
+  <class name="FileRegion" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.hdfs.server.common.BlockAlias"/>
+    <constructor name="FileRegion" type="long, org.apache.hadoop.fs.Path, long, long, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="FileRegion" type="long, org.apache.hadoop.fs.Path, long, long, long, byte[]"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="FileRegion" type="long, org.apache.hadoop.fs.Path, long, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="FileRegion" type="org.apache.hadoop.hdfs.protocol.Block, org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getBlock" return="org.apache.hadoop.hdfs.protocol.Block"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProvidedStorageLocation" return="org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="o" type="java.lang.Object"/>
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[This class is used to represent provided blocks that are file regions,
+ i.e., can be described using (path, offset, length).]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.hdfs.server.common.FileRegion -->
+</package>
+<package name="org.apache.hadoop.hdfs.server.common.blockaliasmap">
+  <!-- start class org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap -->
+  <class name="BlockAliasMap" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="BlockAliasMap"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getReader" return="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Reader"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="opts" type="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Reader.Options"/>
+      <param name="blockPoolID" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns a reader to the alias map.
+ @param opts reader options
+ @param blockPoolID block pool id to use
+ @return {@link Reader} to the alias map. If a Reader for the blockPoolID
+ cannot be created, this will return null.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getWriter" return="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Writer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="opts" type="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Writer.Options"/>
+      <param name="blockPoolID" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns the writer for the alias map.
+ @param opts writer options.
+ @param blockPoolID block pool id to use
+ @return {@link Writer} to the alias map.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="refresh"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Refresh the alias map.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="close"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <doc>
+    <![CDATA[An abstract class used to read and write block maps for provided blocks.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap -->
+</package>
+<package name="org.apache.hadoop.hdfs.server.common.blockaliasmap.impl">
+  <!-- start class org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap -->
+  <class name="LevelDBFileRegionAliasMap" extends="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.conf.Configurable"/>
+    <constructor name="LevelDBFileRegionAliasMap"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="setConf"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getConf" return="org.apache.hadoop.conf.Configuration"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getReader" return="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Reader"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="opts" type="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Reader.Options"/>
+      <param name="blockPoolID" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getWriter" return="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Writer"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="opts" type="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Writer.Options"/>
+      <param name="blockPoolID" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="refresh"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <field name="LOG" type="org.slf4j.Logger"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[A LevelDB based implementation of {@link BlockAliasMap}.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap -->
+  <!-- start class org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap -->
+  <class name="TextFileRegionAliasMap" extends="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.conf.Configurable"/>
+    <constructor name="TextFileRegionAliasMap"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="setConf"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getConf" return="org.apache.hadoop.conf.Configuration"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getReader" return="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Reader"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="opts" type="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Reader.Options"/>
+      <param name="blockPoolID" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getWriter" return="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Writer"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="opts" type="org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap.Writer.Options"/>
+      <param name="blockPoolID" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="refresh"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="blockPoolIDFromFileName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="file" type="org.apache.hadoop.fs.Path"/>
+    </method>
+    <method name="fileNameFromBlockPoolID" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="blockPoolID" type="java.lang.String"/>
+    </method>
+    <field name="LOG" type="org.slf4j.Logger"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[This class is used for block maps stored as text files,
+ with a specified delimiter.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap -->
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.fsdataset">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.fsdataset.impl">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.metrics">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.web">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.web.webhdfs">
+</package>
+<package name="org.apache.hadoop.hdfs.server.diskbalancer">
+</package>
+<package name="org.apache.hadoop.hdfs.server.diskbalancer.command">
+</package>
+<package name="org.apache.hadoop.hdfs.server.diskbalancer.connectors">
+</package>
+<package name="org.apache.hadoop.hdfs.server.diskbalancer.datamodel">
+</package>
+<package name="org.apache.hadoop.hdfs.server.diskbalancer.planner">
+</package>
+<package name="org.apache.hadoop.hdfs.server.mover">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode">
+  <!-- start interface org.apache.hadoop.hdfs.server.namenode.AuditLogger -->
+  <interface name="AuditLogger"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="initialize"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Called during initialization of the logger.
+
+ @param conf The configuration object.]]>
+      </doc>
+    </method>
+    <method name="logAuditEvent"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="succeeded" type="boolean"/>
+      <param name="userName" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetAddress"/>
+      <param name="cmd" type="java.lang.String"/>
+      <param name="src" type="java.lang.String"/>
+      <param name="dst" type="java.lang.String"/>
+      <param name="stat" type="org.apache.hadoop.fs.FileStatus"/>
+      <doc>
+      <![CDATA[Called to log an audit event.
+ <p>
+ This method must return as quickly as possible, since it's called
+ in a critical section of the NameNode's operation.
+
+ @param succeeded Whether authorization succeeded.
+ @param userName Name of the user executing the request.
+ @param addr Remote address of the request.
+ @param cmd The requested command.
+ @param src Path of affected source file.
+ @param dst Path of affected destination file (if any).
+ @param stat File information for operations that change the file's
+             metadata (permissions, owner, times, etc).]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Interface defining an audit logger.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.hdfs.server.namenode.AuditLogger -->
+  <!-- start class org.apache.hadoop.hdfs.server.namenode.HdfsAuditLogger -->
+  <class name="HdfsAuditLogger" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.hdfs.server.namenode.AuditLogger"/>
+    <constructor name="HdfsAuditLogger"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="logAuditEvent"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="succeeded" type="boolean"/>
+      <param name="userName" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetAddress"/>
+      <param name="cmd" type="java.lang.String"/>
+      <param name="src" type="java.lang.String"/>
+      <param name="dst" type="java.lang.String"/>
+      <param name="status" type="org.apache.hadoop.fs.FileStatus"/>
+    </method>
+    <method name="logAuditEvent"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="succeeded" type="boolean"/>
+      <param name="userName" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetAddress"/>
+      <param name="cmd" type="java.lang.String"/>
+      <param name="src" type="java.lang.String"/>
+      <param name="dst" type="java.lang.String"/>
+      <param name="stat" type="org.apache.hadoop.fs.FileStatus"/>
+      <param name="callerContext" type="org.apache.hadoop.ipc.CallerContext"/>
+      <param name="ugi" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="dtSecretManager" type="org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager"/>
+      <doc>
+      <![CDATA[Same as
+ {@link #logAuditEvent(boolean, String, InetAddress, String, String, String,
+ FileStatus)} with additional parameters related to logging delegation token
+ tracking IDs.
+ 
+ @param succeeded Whether authorization succeeded.
+ @param userName Name of the user executing the request.
+ @param addr Remote address of the request.
+ @param cmd The requested command.
+ @param src Path of affected source file.
+ @param dst Path of affected destination file (if any).
+ @param stat File information for operations that change the file's metadata
+          (permissions, owner, times, etc).
+ @param callerContext Context information of the caller
+ @param ugi UserGroupInformation of the current user, or null if not logging
+          token tracking information
+ @param dtSecretManager The token secret manager, or null if not logging
+          token tracking information]]>
+      </doc>
+    </method>
+    <method name="logAuditEvent"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="succeeded" type="boolean"/>
+      <param name="userName" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetAddress"/>
+      <param name="cmd" type="java.lang.String"/>
+      <param name="src" type="java.lang.String"/>
+      <param name="dst" type="java.lang.String"/>
+      <param name="stat" type="org.apache.hadoop.fs.FileStatus"/>
+      <param name="ugi" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="dtSecretManager" type="org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager"/>
+      <doc>
+      <![CDATA[Same as
+ {@link #logAuditEvent(boolean, String, InetAddress, String, String,
+ String, FileStatus, CallerContext, UserGroupInformation,
+ DelegationTokenSecretManager)} without {@link CallerContext} information.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Extension of {@link AuditLogger}.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.hdfs.server.namenode.HdfsAuditLogger -->
+  <!-- start class org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider -->
+  <class name="INodeAttributeProvider" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="INodeAttributeProvider"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="start"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Initialize the provider. This method is called at NameNode startup
+ time.]]>
+      </doc>
+    </method>
+    <method name="stop"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Shutdown the provider. This method is called at NameNode shutdown time.]]>
+      </doc>
+    </method>
+    <method name="getAttributes" return="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="fullPath" type="java.lang.String"/>
+      <param name="inode" type="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"/>
+    </method>
+    <method name="getAttributes" return="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="pathElements" type="java.lang.String[]"/>
+      <param name="inode" type="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"/>
+    </method>
+    <method name="getAttributes" return="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="components" type="byte[][]"/>
+      <param name="inode" type="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"/>
+    </method>
+    <method name="getExternalAccessControlEnforcer" return="org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="defaultEnforcer" type="org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer"/>
+      <doc>
+      <![CDATA[Can be over-ridden by implementations to provide a custom Access Control
+ Enforcer that can provide an alternate implementation of the
+ default permission checking logic.
+ @param defaultEnforcer The Default AccessControlEnforcer
+ @return The AccessControlEnforcer to use]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider -->
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.ha">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.metrics">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.snapshot">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.top">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.top.metrics">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.top.window">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.web.resources">
+</package>
+<package name="org.apache.hadoop.hdfs.server.protocol">
+</package>
+<package name="org.apache.hadoop.hdfs.tools">
+</package>
+<package name="org.apache.hadoop.hdfs.tools.offlineEditsViewer">
+</package>
+<package name="org.apache.hadoop.hdfs.tools.offlineImageViewer">
+</package>
+<package name="org.apache.hadoop.hdfs.tools.snapshot">
+</package>
+<package name="org.apache.hadoop.hdfs.util">
+</package>
+<package name="org.apache.hadoop.hdfs.web">
+</package>
+<package name="org.apache.hadoop.hdfs.web.resources">
+</package>
+
+</api>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49c68760/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index 5e21b4a..66afc14 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -145,7 +145,7 @@
         <activeByDefault>false</activeByDefault>
       </activation>
       <properties>
-        <jdiff.stable.api>3.0.3</jdiff.stable.api>
+        <jdiff.stable.api>3.1.1</jdiff.stable.api>
         <jdiff.stability>-unstable</jdiff.stability>
         <!-- Commented out for HADOOP-11776 -->
         <!-- Uncomment param name="${jdiff.compatibility}" in javadoc doclet if compatibility is not empty -->


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


[21/50] [abbrv] hadoop git commit: YARN-8559. Expose mutable-conf scheduler's configuration in RM /scheduler-conf endpoint. Contributed by Weiwei Yang.

Posted by tm...@apache.org.
YARN-8559. Expose mutable-conf scheduler's configuration in RM /scheduler-conf endpoint. 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/d352f167
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d352f167
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d352f167

Branch: refs/heads/HADOOP-15407
Commit: d352f167ebb865a6486afbbdac8e2a5e97a7bbad
Parents: cd04e95
Author: Weiwei Yang <ww...@apache.org>
Authored: Thu Aug 9 23:46:53 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Thu Aug 9 23:46:53 2018 +0800

----------------------------------------------------------------------
 .../scheduler/MutableConfigurationProvider.java |  7 ++
 .../conf/MutableCSConfigurationProvider.java    |  5 ++
 .../resourcemanager/webapp/RMWebServices.java   | 34 +++++++++
 .../resourcemanager/webapp/dao/ConfInfo.java    | 72 ++++++++++++++++++++
 .../TestRMWebServicesConfigurationMutation.java | 40 +++++++++++
 .../src/site/markdown/ResourceManagerRest.md    | 40 +++++++++++
 6 files changed, 198 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d352f167/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.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/MutableConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
index 2b9b25a..6e56f3d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
@@ -59,6 +60,12 @@ public interface MutableConfigurationProvider {
   void confirmPendingMutation(boolean isValid) throws Exception;
 
   /**
+   * Returns scheduler configuration cached in this provider.
+   * @return scheduler configuration.
+   */
+  Configuration getConfiguration();
+
+  /**
    * Closes the configuration provider, releasing any required resources.
    * @throws IOException on failure to close
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d352f167/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
index 9c3bf9d..51de437 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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
@@ -126,6 +126,11 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
   }
 
   @Override
+  public Configuration getConfiguration() {
+    return new Configuration(schedConf);
+  }
+
+  @Override
   public ConfigurationMutationACLPolicy getAclMutationPolicy() {
     return aclMutationPolicy;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d352f167/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.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/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 7752fa2..0117376 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -188,6 +188,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ConfInfo;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.webapp.WebServices;
@@ -2373,6 +2374,39 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
   }
 
   @GET
+  @Path(RMWSConsts.SCHEDULER_CONF)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  public Response getSchedulerConfiguration(@Context HttpServletRequest hsr)
+      throws AuthorizationException {
+    // Only admin user is allowed to read scheduler conf,
+    // in order to avoid leaking sensitive info, such as ACLs
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    initForWritableEndpoints(callerUGI, true);
+
+    ResourceScheduler scheduler = rm.getResourceScheduler();
+    if (scheduler instanceof MutableConfScheduler
+        && ((MutableConfScheduler) scheduler).isConfigurationMutable()) {
+      MutableConfigurationProvider mutableConfigurationProvider =
+          ((MutableConfScheduler) scheduler).getMutableConfProvider();
+      // We load the cached configuration from configuration store,
+      // this should be the conf properties used by the scheduler.
+      Configuration schedulerConf = mutableConfigurationProvider
+          .getConfiguration();
+      return Response.status(Status.OK)
+          .entity(new ConfInfo(schedulerConf))
+          .build();
+    } else {
+      return Response.status(Status.BAD_REQUEST).entity(
+          "This API only supports to retrieve scheduler configuration"
+              + " from a mutable-conf scheduler, underneath scheduler "
+              + scheduler.getClass().getSimpleName()
+              + " is not an instance of MutableConfScheduler")
+          .build();
+    }
+  }
+
+  @GET
   @Path(RMWSConsts.CHECK_USER_ACCESS_TO_QUEUE)
   @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
                 MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d352f167/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ConfInfo.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/webapp/dao/ConfInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ConfInfo.java
new file mode 100644
index 0000000..bfc450b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ConfInfo.java
@@ -0,0 +1,72 @@
+/**
+ * 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.webapp.dao;
+
+import org.apache.hadoop.conf.Configuration;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+
+@XmlRootElement(name = "configuration")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ConfInfo {
+
+  protected ArrayList<ConfItem> property = new ArrayList<>();
+
+  public ConfInfo() {
+  } // JAXB needs this
+
+  public ConfInfo(Configuration conf) {
+    conf.forEach(entry ->
+        add(new ConfItem(entry.getKey(), entry.getValue())));
+  }
+
+  public void add(ConfItem confItem) {
+    property.add(confItem);
+  }
+
+  public ArrayList<ConfItem> getItems() {
+    return property;
+  }
+
+  @XmlAccessorType(XmlAccessType.FIELD)
+  public static class ConfItem {
+
+    private String name;
+    private String value;
+
+    public ConfItem() {
+      // JAXB needs this
+    }
+
+    public ConfItem(String name, String value){
+      this.name = name;
+      this.value = value;
+    }
+
+    public String getKey() {
+      return name;
+    }
+
+    public String getValue() {
+      return value;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d352f167/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.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/webapp/TestRMWebServicesConfigurationMutation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
index 99b5648..3e2542c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
@@ -39,6 +39,9 @@ import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo;
 import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 import org.apache.hadoop.yarn.webapp.util.YarnWebServiceUtils;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -56,6 +59,7 @@ import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
 
 /**
  * Test scheduler configuration mutation via REST API.
@@ -157,8 +161,40 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         .contextPath("jersey-guice-filter").servletPath("/").build());
   }
 
+  private CapacitySchedulerConfiguration getSchedulerConf()
+      throws JSONException {
+    WebResource r = resource();
+    ClientResponse response =
+        r.path("ws").path("v1").path("cluster")
+            .queryParam("user.name", userName).path("scheduler-conf")
+            .accept(MediaType.APPLICATION_JSON)
+            .get(ClientResponse.class);
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    JSONObject json = response.getEntity(JSONObject.class);
+    JSONArray items = (JSONArray) json.get("property");
+    CapacitySchedulerConfiguration parsedConf =
+        new CapacitySchedulerConfiguration();
+    for (int i=0; i<items.length(); i++) {
+      JSONObject obj = (JSONObject) items.get(i);
+      parsedConf.set(obj.get("name").toString(),
+          obj.get("value").toString());
+    }
+    return parsedConf;
+  }
+
+  @Test
+  public void testGetSchedulerConf() throws Exception {
+    CapacitySchedulerConfiguration orgConf = getSchedulerConf();
+    assertNotNull(orgConf);
+    assertEquals(3, orgConf.getQueues("root").length);
+  }
+
   @Test
   public void testAddNestedQueue() throws Exception {
+    CapacitySchedulerConfiguration orgConf = getSchedulerConf();
+    assertNotNull(orgConf);
+    assertEquals(3, orgConf.getQueues("root").length);
+
     WebResource r = resource();
 
     ClientResponse response;
@@ -198,6 +234,10 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         0.01f);
     assertEquals(75.0f, newCSConf.getNonLabeledQueueCapacity("root.d.d2"),
         0.01f);
+
+    CapacitySchedulerConfiguration newConf = getSchedulerConf();
+    assertNotNull(newConf);
+    assertEquals(4, newConf.getQueues("root").length);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d352f167/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
index 24c2319..041af4c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
@@ -4964,6 +4964,7 @@ Please note that this feature is currently in the alpha stage and is subject to
 
 ### HTTP Operations Supported
 
+      * GET
       * PUT
 
 ### Elements of the *sched-conf* object
@@ -4975,6 +4976,45 @@ Please note that this feature is currently in the alpha stage and is subject to
 | remove-queue | string | Full path name of a queue to remove |
 | global-updates | map | Map of key value pairs to update scheduler's global configuration |
 
+### GET Request Examples
+
+Get requests are used to retrieve the scheduler's configuration that is currently loaded into scheduler's context.
+
+**XML response**
+
+HTTP Request:
+
+      Accept: application/xml
+      Content-Type: application/xml
+      GET http://rm-http-address:port/ws/v1/cluster/scheduler-conf
+
+Response Header:
+
+      TTP/1.1 200 OK
+      Content-Type: application/xml; charset=utf-8
+      Transfer-Encoding: chunked
+
+Response Body:
+
+
+```xml
+      <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+      <configuration>
+        <property>
+          <name>yarn.scheduler.capacity.root.queues</name>
+          <value>default</value>
+        </property>
+        <property>
+          <name>yarn.scheduler.capacity.maximum-applications</name>
+          <value>10000</value>
+        </property>
+        <property>
+          <name>yarn.scheduler.capacity.root.default.capacity</name>
+          <value>100</value>
+        </property>
+      </configuration>
+```
+
 ### PUT Request Examples
 
 Put requests are used to modify the scheduler configuration. A successful mutation results in a 200 response. A malformed request or one which resulted in an invalid scheduler configuration results in a 400 response.


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


[25/50] [abbrv] hadoop git commit: YARN-4946. RM should not consider an application as COMPLETED when log aggregation is not in a terminal state (snemeth via rkanter)

Posted by tm...@apache.org.
YARN-4946. RM should not consider an application as COMPLETED when log aggregation is not in a terminal state (snemeth via rkanter)


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

Branch: refs/heads/HADOOP-15407
Commit: b2517dd66b3c88fdd478411cf208921bd3023755
Parents: 8244abb
Author: Robert Kanter <rk...@apache.org>
Authored: Thu Aug 9 14:58:04 2018 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Thu Aug 9 14:58:04 2018 -0700

----------------------------------------------------------------------
 .../server/resourcemanager/RMAppManager.java    |  81 +++++--
 .../server/resourcemanager/rmapp/RMApp.java     |   6 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |   8 +-
 .../server/resourcemanager/TestAppManager.java  | 241 +++++++++++++++----
 .../applicationsmanager/MockAsm.java            |  11 +
 .../server/resourcemanager/rmapp/MockRMApp.java |  20 ++
 6 files changed, 294 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2517dd6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.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/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 7011aaa..ee78c08 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -86,7 +86,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
   private int maxCompletedAppsInMemory;
   private int maxCompletedAppsInStateStore;
   protected int completedAppsInStateStore = 0;
-  private LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
+  protected LinkedList<ApplicationId> completedApps = new LinkedList<>();
 
   private final RMContext rmContext;
   private final ApplicationMasterService masterService;
@@ -284,31 +284,72 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
    * check to see if hit the limit for max # completed apps kept
    */
   protected synchronized void checkAppNumCompletedLimit() {
-    // check apps kept in state store.
-    while (completedAppsInStateStore > this.maxCompletedAppsInStateStore) {
-      ApplicationId removeId =
-          completedApps.get(completedApps.size() - completedAppsInStateStore);
+    if (completedAppsInStateStore > maxCompletedAppsInStateStore) {
+      removeCompletedAppsFromStateStore();
+    }
+
+    if (completedApps.size() > maxCompletedAppsInMemory) {
+      removeCompletedAppsFromMemory();
+    }
+  }
+
+  private void removeCompletedAppsFromStateStore() {
+    int numDelete = completedAppsInStateStore - maxCompletedAppsInStateStore;
+    for (int i = 0; i < numDelete; i++) {
+      ApplicationId removeId = completedApps.get(i);
       RMApp removeApp = rmContext.getRMApps().get(removeId);
-      LOG.info("Max number of completed apps kept in state store met:"
-          + " maxCompletedAppsInStateStore = " + maxCompletedAppsInStateStore
-          + ", removing app " + removeApp.getApplicationId()
-          + " from state store.");
-      rmContext.getStateStore().removeApplication(removeApp);
-      completedAppsInStateStore--;
+      boolean deleteApp = shouldDeleteApp(removeApp);
+
+      if (deleteApp) {
+        LOG.info("Max number of completed apps kept in state store met:"
+            + " maxCompletedAppsInStateStore = "
+            + maxCompletedAppsInStateStore + ", removing app " + removeId
+            + " from state store.");
+        rmContext.getStateStore().removeApplication(removeApp);
+        completedAppsInStateStore--;
+      } else {
+        LOG.info("Max number of completed apps kept in state store met:"
+            + " maxCompletedAppsInStateStore = "
+            + maxCompletedAppsInStateStore + ", but not removing app "
+            + removeId
+            + " from state store as log aggregation have not finished yet.");
+      }
     }
+  }
 
-    // check apps kept in memorty.
-    while (completedApps.size() > this.maxCompletedAppsInMemory) {
-      ApplicationId removeId = completedApps.remove();
-      LOG.info("Application should be expired, max number of completed apps"
-          + " kept in memory met: maxCompletedAppsInMemory = "
-          + this.maxCompletedAppsInMemory + ", removing app " + removeId
-          + " from memory: ");
-      rmContext.getRMApps().remove(removeId);
-      this.applicationACLsManager.removeApplication(removeId);
+  private void removeCompletedAppsFromMemory() {
+    int numDelete = completedApps.size() - maxCompletedAppsInMemory;
+    int offset = 0;
+    for (int i = 0; i < numDelete; i++) {
+      int deletionIdx = i - offset;
+      ApplicationId removeId = completedApps.get(deletionIdx);
+      RMApp removeApp = rmContext.getRMApps().get(removeId);
+      boolean deleteApp = shouldDeleteApp(removeApp);
+
+      if (deleteApp) {
+        ++offset;
+        LOG.info("Application should be expired, max number of completed apps"
+                + " kept in memory met: maxCompletedAppsInMemory = "
+                + this.maxCompletedAppsInMemory + ", removing app " + removeId
+                + " from memory: ");
+        completedApps.remove(deletionIdx);
+        rmContext.getRMApps().remove(removeId);
+        this.applicationACLsManager.removeApplication(removeId);
+      } else {
+        LOG.info("Application should be expired, max number of completed apps"
+                + " kept in memory met: maxCompletedAppsInMemory = "
+                + this.maxCompletedAppsInMemory + ", but not removing app "
+                + removeId
+                + " from memory as log aggregation have not finished yet.");
+      }
     }
   }
 
+  private boolean shouldDeleteApp(RMApp app) {
+    return !app.isLogAggregationEnabled()
+            || app.isLogAggregationFinished();
+  }
+
   @SuppressWarnings("unchecked")
   protected void submitApplication(
       ApplicationSubmissionContext submissionContext, long submitTime,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2517dd6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.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/rmapp/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index 99cce87..535888c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -242,7 +242,11 @@ public interface RMApp extends EventHandler<RMAppEvent> {
    * @return the number of max attempts of the application.
    */
   int getMaxAppAttempts();
-  
+
+  boolean isLogAggregationEnabled();
+
+  boolean isLogAggregationFinished();
+
   /**
    * Returns the application type
    * @return the application type.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2517dd6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 9f1ea44..42e2bcf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -1912,7 +1912,13 @@ public class RMAppImpl implements RMApp, Recoverable {
     }
   }
 
-  private boolean isLogAggregationFinished() {
+  @Override
+  public boolean isLogAggregationEnabled() {
+    return logAggregationEnabled;
+  }
+
+  @Override
+  public boolean isLogAggregationFinished() {
     return this.logAggregationStatusForAppReport
       .equals(LogAggregationStatus.SUCCEEDED)
         || this.logAggregationStatusForAppReport

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2517dd6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.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/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
index 6a6f9cf..27e87bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
@@ -19,28 +19,9 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Matchers.matches;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -88,28 +69,48 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptI
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
-    .CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
-    .CapacitySchedulerConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
-    .ManagedParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.ArgumentCaptor;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+
+import static java.util.stream.Collectors.toSet;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Matchers.matches;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 /**
  * Testing applications being retired from RM.
@@ -131,7 +132,7 @@ public class TestAppManager{
   } 
 
 
-  public static List<RMApp> newRMApps(int n, long time, RMAppState state) {
+  private static List<RMApp> newRMApps(int n, long time, RMAppState state) {
     List<RMApp> list = Lists.newArrayList();
     for (int i = 0; i < n; ++i) {
       list.add(new MockRMApp(i, time, state));
@@ -139,23 +140,63 @@ public class TestAppManager{
     return list;
   }
 
+  private static List<RMApp> newRMAppsMixedLogAggregationStatus(int n,
+      long time, RMAppState state) {
+    List<RMApp> list = Lists.newArrayList();
+    for (int i = 0; i < n; ++i) {
+      MockRMApp rmApp = new MockRMApp(i, time, state);
+      rmApp.setLogAggregationEnabled(true);
+      rmApp.setLogAggregationFinished(i % 2 == 0);
+      list.add(rmApp);
+    }
+    return list;
+  }
+
   public RMContext mockRMContext(int n, long time) {
+    final ConcurrentMap<ApplicationId, RMApp> map = createRMAppsMap(n, time);
+    return createMockRMContextInternal(map);
+  }
+
+  public RMContext mockRMContextWithMixedLogAggregationStatus(int n,
+      long time) {
+    final ConcurrentMap<ApplicationId, RMApp> map =
+        createRMAppsMapMixedLogAggStatus(n, time);
+    return createMockRMContextInternal(map);
+  }
+
+  private ConcurrentMap<ApplicationId, RMApp> createRMAppsMap(int n,
+      long time) {
     final List<RMApp> apps = newRMApps(n, time, RMAppState.FINISHED);
     final ConcurrentMap<ApplicationId, RMApp> map = Maps.newConcurrentMap();
     for (RMApp app : apps) {
       map.put(app.getApplicationId(), app);
     }
+    return map;
+  }
+
+  private ConcurrentMap<ApplicationId, RMApp> createRMAppsMapMixedLogAggStatus(
+      int n, long time) {
+    final List<RMApp> apps =
+        newRMAppsMixedLogAggregationStatus(n, time, RMAppState.FINISHED);
+    final ConcurrentMap<ApplicationId, RMApp> map = Maps.newConcurrentMap();
+    for (RMApp app : apps) {
+      map.put(app.getApplicationId(), app);
+    }
+    return map;
+  }
+
+  private RMContext createMockRMContextInternal(ConcurrentMap<ApplicationId, RMApp> map) {
     Dispatcher rmDispatcher = new AsyncDispatcher();
     ContainerAllocationExpirer containerAllocationExpirer = new ContainerAllocationExpirer(
-        rmDispatcher);
+            rmDispatcher);
     AMLivelinessMonitor amLivelinessMonitor = new AMLivelinessMonitor(
-        rmDispatcher);
+            rmDispatcher);
     AMLivelinessMonitor amFinishingMonitor = new AMLivelinessMonitor(
-        rmDispatcher);
+            rmDispatcher);
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
     RMContext context = new RMContextImpl(rmDispatcher,
-        containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
-        null, null, null, null, null) {
+            containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
+            null, null, null, null, null) {
       @Override
       public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
         return map;
@@ -198,9 +239,11 @@ public class TestAppManager{
 
   // Extend and make the functions we want to test public
   public class TestRMAppManager extends RMAppManager {
+    private final RMStateStore stateStore;
 
     public TestRMAppManager(RMContext context, Configuration conf) {
       super(context, null, null, new ApplicationACLsManager(conf), conf);
+      this.stateStore = context.getStateStore();
     }
 
     public TestRMAppManager(RMContext context,
@@ -208,6 +251,7 @@ public class TestAppManager{
         YarnScheduler scheduler, ApplicationMasterService masterService,
         ApplicationACLsManager applicationACLsManager, Configuration conf) {
       super(context, scheduler, masterService, applicationACLsManager, conf);
+      this.stateStore = context.getStateStore();
     }
 
     public void checkAppNumCompletedLimit() {
@@ -222,10 +266,32 @@ public class TestAppManager{
       return super.getCompletedAppsListSize();
     }
 
-    public int getCompletedAppsInStateStore() {
+    public int getNumberOfCompletedAppsInStateStore() {
       return this.completedAppsInStateStore;
     }
 
+    List<ApplicationId> getCompletedApps() {
+      return completedApps;
+    }
+
+    Set<ApplicationId> getFirstNCompletedApps(int n) {
+      return getCompletedApps().stream().limit(n).collect(toSet());
+    }
+
+    Set<ApplicationId> getCompletedAppsWithEvenIdsInRange(int n) {
+      return getCompletedApps().stream().limit(n)
+          .filter(app -> app.getId() % 2 == 0).collect(toSet());
+    }
+
+    Set<ApplicationId> getRemovedAppsFromStateStore(int numRemoves) {
+      ArgumentCaptor<RMApp> argumentCaptor =
+          ArgumentCaptor.forClass(RMApp.class);
+      verify(stateStore, times(numRemoves))
+          .removeApplication(argumentCaptor.capture());
+      return argumentCaptor.getAllValues().stream().map(RMApp::getApplicationId)
+          .collect(toSet());
+    }
+
     public void submitApplication(
         ApplicationSubmissionContext submissionContext, String user)
             throws YarnException, IOException {
@@ -234,10 +300,14 @@ public class TestAppManager{
     }
   }
 
-  protected void addToCompletedApps(TestRMAppManager appMonitor, RMContext rmContext) {
-    for (RMApp app : rmContext.getRMApps().values()) {
+  private void addToCompletedApps(TestRMAppManager appMonitor,
+          RMContext rmContext) {
+    // ensure applications are finished in order by their IDs
+    List<RMApp> sortedApps = new ArrayList<>(rmContext.getRMApps().values());
+    sortedApps.sort(Comparator.comparingInt(o -> o.getApplicationId().getId()));
+    for (RMApp app : sortedApps) {
       if (app.getState() == RMAppState.FINISHED
-          || app.getState() == RMAppState.KILLED 
+          || app.getState() == RMAppState.KILLED
           || app.getState() == RMAppState.FAILED) {
         appMonitor.finishApplication(app.getApplicationId());
       }
@@ -631,7 +701,8 @@ public class TestAppManager{
   @Test
   public void testStateStoreAppLimitLessThanMemoryAppLimit() {
     long now = System.currentTimeMillis();
-    RMContext rmContext = mockRMContext(10, now - 20000);
+    final int allApps = 10;
+    RMContext rmContext = mockRMContext(allApps, now - 20000);
     Configuration conf = new YarnConfiguration();
     int maxAppsInMemory = 8;
     int maxAppsInStateStore = 4;
@@ -641,39 +712,57 @@ public class TestAppManager{
     TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
 
     addToCompletedApps(appMonitor, rmContext);
-    Assert.assertEquals("Number of completed apps incorrect", 10,
+    Assert.assertEquals("Number of completed apps incorrect", allApps,
         appMonitor.getCompletedAppsListSize());
+
+    int numRemoveAppsFromStateStore = allApps - maxAppsInStateStore;
+    Set<ApplicationId> appsShouldBeRemovedFromStateStore = appMonitor
+            .getFirstNCompletedApps(numRemoveAppsFromStateStore);
     appMonitor.checkAppNumCompletedLimit();
 
+    Set<ApplicationId> removedAppsFromStateStore = appMonitor
+            .getRemovedAppsFromStateStore(numRemoveAppsFromStateStore);
+
     Assert.assertEquals("Number of apps incorrect after # completed check",
       maxAppsInMemory, rmContext.getRMApps().size());
     Assert.assertEquals("Number of completed apps incorrect after check",
       maxAppsInMemory, appMonitor.getCompletedAppsListSize());
 
-    int numRemoveAppsFromStateStore = 10 - maxAppsInStateStore;
     verify(rmContext.getStateStore(), times(numRemoveAppsFromStateStore))
       .removeApplication(isA(RMApp.class));
     Assert.assertEquals(maxAppsInStateStore,
-      appMonitor.getCompletedAppsInStateStore());
+      appMonitor.getNumberOfCompletedAppsInStateStore());
+
+    List<ApplicationId> completedApps = appMonitor.getCompletedApps();
+    Assert.assertEquals(maxAppsInMemory, completedApps.size());
+    Assert.assertEquals(numRemoveAppsFromStateStore,
+        removedAppsFromStateStore.size());
+    Assert.assertEquals(numRemoveAppsFromStateStore,
+        Sets.intersection(appsShouldBeRemovedFromStateStore,
+            removedAppsFromStateStore).size());
   }
 
   @Test
-  public void testStateStoreAppLimitLargerThanMemoryAppLimit() {
+  public void testStateStoreAppLimitGreaterThanMemoryAppLimit() {
     long now = System.currentTimeMillis();
-    RMContext rmContext = mockRMContext(10, now - 20000);
+    final int allApps = 10;
+    RMContext rmContext = mockRMContext(allApps, now - 20000);
     Configuration conf = new YarnConfiguration();
     int maxAppsInMemory = 8;
     conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, maxAppsInMemory);
-    // larger than maxCompletedAppsInMemory, reset to RM_MAX_COMPLETED_APPLICATIONS.
+    // greater than maxCompletedAppsInMemory, reset to RM_MAX_COMPLETED_APPLICATIONS.
     conf.setInt(YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS, 1000);
     TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
 
     addToCompletedApps(appMonitor, rmContext);
-    Assert.assertEquals("Number of completed apps incorrect", 10,
+    Assert.assertEquals("Number of completed apps incorrect", allApps,
         appMonitor.getCompletedAppsListSize());
+
+    int numRemoveApps = allApps - maxAppsInMemory;
+    Set<ApplicationId> appsShouldBeRemoved = appMonitor
+            .getFirstNCompletedApps(numRemoveApps);
     appMonitor.checkAppNumCompletedLimit();
 
-    int numRemoveApps = 10 - maxAppsInMemory;
     Assert.assertEquals("Number of apps incorrect after # completed check",
       maxAppsInMemory, rmContext.getRMApps().size());
     Assert.assertEquals("Number of completed apps incorrect after check",
@@ -681,7 +770,57 @@ public class TestAppManager{
     verify(rmContext.getStateStore(), times(numRemoveApps)).removeApplication(
       isA(RMApp.class));
     Assert.assertEquals(maxAppsInMemory,
-      appMonitor.getCompletedAppsInStateStore());
+      appMonitor.getNumberOfCompletedAppsInStateStore());
+
+    List<ApplicationId> completedApps = appMonitor.getCompletedApps();
+    Assert.assertEquals(maxAppsInMemory, completedApps.size());
+    Assert.assertEquals(numRemoveApps, appsShouldBeRemoved.size());
+    assertTrue(Collections.disjoint(completedApps, appsShouldBeRemoved));
+  }
+
+  @Test
+  public void testStateStoreAppLimitSomeAppsHaveNotFinishedLogAggregation() {
+    long now = System.currentTimeMillis();
+    final int allApps = 10;
+    RMContext rmContext =
+        mockRMContextWithMixedLogAggregationStatus(allApps, now - 20000);
+    Configuration conf = new YarnConfiguration();
+    int maxAppsInMemory = 2;
+    conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS,
+        maxAppsInMemory);
+    // greater than maxCompletedAppsInMemory, reset to
+    // RM_MAX_COMPLETED_APPLICATIONS.
+    conf.setInt(YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS,
+        1000);
+    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
+
+    addToCompletedApps(appMonitor, rmContext);
+    Assert.assertEquals("Number of completed apps incorrect", allApps,
+            appMonitor.getCompletedAppsListSize());
+
+    int numRemoveApps = allApps - maxAppsInMemory;
+    int effectiveNumRemoveApps = numRemoveApps / 2;
+    //only apps with even ID would be deleted due to log aggregation status
+    int expectedNumberOfAppsInMemory = maxAppsInMemory + effectiveNumRemoveApps;
+
+    Set<ApplicationId> appsShouldBeRemoved = appMonitor
+            .getCompletedAppsWithEvenIdsInRange(numRemoveApps);
+    appMonitor.checkAppNumCompletedLimit();
+
+    Assert.assertEquals("Number of apps incorrect after # completed check",
+        expectedNumberOfAppsInMemory, rmContext.getRMApps().size());
+    Assert.assertEquals("Number of completed apps incorrect after check",
+        expectedNumberOfAppsInMemory, appMonitor.getCompletedAppsListSize());
+    verify(rmContext.getStateStore(), times(effectiveNumRemoveApps))
+        .removeApplication(isA(RMApp.class));
+    Assert.assertEquals(expectedNumberOfAppsInMemory,
+        appMonitor.getNumberOfCompletedAppsInStateStore());
+
+    List<ApplicationId> completedApps = appMonitor.getCompletedApps();
+
+    Assert.assertEquals(expectedNumberOfAppsInMemory, completedApps.size());
+    Assert.assertEquals(effectiveNumRemoveApps, appsShouldBeRemoved.size());
+    assertTrue(Collections.disjoint(completedApps, appsShouldBeRemoved));
   }
 
   protected void setupDispatcher(RMContext rmContext, Configuration conf) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2517dd6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.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/applicationsmanager/MockAsm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
index 6c6c4b4..342dab8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
@@ -146,6 +146,17 @@ public abstract class MockAsm extends MockApps {
     public int getMaxAppAttempts() {
       throw new UnsupportedOperationException("Not supported yet.");
     }
+
+    @Override
+    public boolean isLogAggregationEnabled() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    @Override
+    public boolean isLogAggregationFinished() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
+
     @Override
     public ApplicationReport createAndGetApplicationReport(
         String clientUserName,boolean allowAccess) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2517dd6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.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/rmapp/MockRMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
index ad29d27..32ece34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
@@ -70,6 +70,8 @@ public class MockRMApp implements RMApp {
   int maxAppAttempts = 1;
   List<ResourceRequest> amReqs;
   private Set<String> applicationTags = null;
+  private boolean logAggregationEnabled;
+  private boolean logAggregationFinished;
 
   public MockRMApp(int newid, long time, RMAppState newState) {
     finish = time;
@@ -236,6 +238,24 @@ public class MockRMApp implements RMApp {
     return maxAppAttempts;
   }
 
+  @Override
+  public boolean isLogAggregationEnabled() {
+    return logAggregationEnabled;
+  }
+
+  @Override
+  public boolean isLogAggregationFinished() {
+    return logAggregationFinished;
+  }
+
+  public void setLogAggregationEnabled(boolean enabled) {
+    this.logAggregationEnabled = enabled;
+  }
+
+  public void setLogAggregationFinished(boolean finished) {
+    this.logAggregationFinished = finished;
+  }
+
   public void setNumMaxRetries(int maxAppAttempts) {
     this.maxAppAttempts = maxAppAttempts;
   }


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


[18/50] [abbrv] hadoop git commit: HDDS-219. Genearate version-info.properties for hadoop and ozone. Contributed by Sandeep Nemuri.

Posted by tm...@apache.org.
HDDS-219. Genearate version-info.properties for hadoop and ozone. Contributed by Sandeep Nemuri.


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

Branch: refs/heads/HADOOP-15407
Commit: 3d96bc6e5ff098900cf07e4b30c642e961a39427
Parents: 00013d6
Author: Márton Elek <el...@apache.org>
Authored: Thu Aug 9 11:06:03 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Thu Aug 9 11:06:03 2018 +0200

----------------------------------------------------------------------
 hadoop-hdds/common/pom.xml                      |  34 +++
 .../apache/hadoop/utils/HddsVersionInfo.java    | 182 ++++++++++++++++
 .../main/resources/hdds-version-info.properties |  26 +++
 hadoop-ozone/common/pom.xml                     |  35 +++
 hadoop-ozone/common/src/main/bin/ozone          |   2 +-
 .../hadoop/ozone/util/OzoneVersionInfo.java     | 213 +++++++++++++++++++
 .../resources/ozone-version-info.properties     |  27 +++
 7 files changed, 518 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d96bc6e/hadoop-hdds/common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/pom.xml b/hadoop-hdds/common/pom.xml
index 4068522..ed29d31 100644
--- a/hadoop-hdds/common/pom.xml
+++ b/hadoop-hdds/common/pom.xml
@@ -29,10 +29,12 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <packaging>jar</packaging>
 
   <properties>
+    <hdds.version>0.2.1-SNAPSHOT</hdds.version>
     <hadoop.component>hdds</hadoop.component>
     <is.hadoop.component>true</is.hadoop.component>
     <log4j2.version>2.11.0</log4j2.version>
     <disruptor.version>3.4.2</disruptor.version>
+    <declared.hdds.version>${hdds.version}</declared.hdds.version>
   </properties>
 
   <dependencies>
@@ -102,6 +104,22 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   </dependencies>
 
   <build>
+    <resources>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <excludes>
+          <exclude>hdds-version-info.properties</exclude>
+        </excludes>
+        <filtering>false</filtering>
+      </resource>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <includes>
+          <include>hdds-version-info.properties</include>
+        </includes>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
     <extensions>
       <extension>
         <groupId>kr.motd.maven</groupId>
@@ -170,6 +188,22 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
         <artifactId>hadoop-maven-plugins</artifactId>
         <executions>
           <execution>
+            <id>version-info</id>
+            <phase>generate-resources</phase>
+            <goals>
+              <goal>version-info</goal>
+            </goals>
+            <configuration>
+              <source>
+                <directory>${basedir}/../</directory>
+                <includes>
+                  <include>*/src/main/java/**/*.java</include>
+                  <include>*/src/main/proto/*.proto</include>
+                </includes>
+              </source>
+            </configuration>
+          </execution>
+          <execution>
             <id>compile-protoc</id>
             <goals>
               <goal>protoc</goal>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d96bc6e/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java
new file mode 100644
index 0000000..59b9de6
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java
@@ -0,0 +1,182 @@
+/*
+ * 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.utils;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.ClassUtil;
+import org.apache.hadoop.util.ThreadUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+
+/**
+ * This class returns build information about Hadoop components.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class HddsVersionInfo {
+  private static final Logger LOG = LoggerFactory.getLogger(HddsVersionInfo.class);
+
+  private Properties info;
+
+  protected HddsVersionInfo(String component) {
+    info = new Properties();
+    String versionInfoFile = component + "-version-info.properties";
+    InputStream is = null;
+    try {
+      is = ThreadUtil.getResourceAsStream(HddsVersionInfo.class.getClassLoader(),
+          versionInfoFile);
+      info.load(is);
+    } catch (IOException ex) {
+      LoggerFactory.getLogger(getClass()).warn("Could not read '" +
+          versionInfoFile + "', " + ex.toString(), ex);
+    } finally {
+      IOUtils.closeStream(is);
+    }
+  }
+
+  protected String _getVersion() {
+    return info.getProperty("version", "Unknown");
+  }
+
+  protected String _getRevision() {
+    return info.getProperty("revision", "Unknown");
+  }
+
+  protected String _getBranch() {
+    return info.getProperty("branch", "Unknown");
+  }
+
+  protected String _getDate() {
+    return info.getProperty("date", "Unknown");
+  }
+
+  protected String _getUser() {
+    return info.getProperty("user", "Unknown");
+  }
+
+  protected String _getUrl() {
+    return info.getProperty("url", "Unknown");
+  }
+
+  protected String _getSrcChecksum() {
+    return info.getProperty("srcChecksum", "Unknown");
+  }
+
+  protected String _getBuildVersion(){
+    return _getVersion() +
+      " from " + _getRevision() +
+      " by " + _getUser() +
+      " source checksum " + _getSrcChecksum();
+  }
+
+  protected String _getProtocVersion() {
+    return info.getProperty("protocVersion", "Unknown");
+  }
+
+  private static HddsVersionInfo HDDS_VERSION_INFO = new HddsVersionInfo("hdds");
+  /**
+   * Get the HDDS version.
+   * @return the Hdds version string, eg. "0.6.3-dev"
+   */
+  public static String getVersion() {
+    return HDDS_VERSION_INFO._getVersion();
+  }
+
+  /**
+   * Get the Git commit hash of the repository when compiled.
+   * @return the commit hash, eg. "18f64065d5db6208daf50b02c1b5ed4ee3ce547a"
+   */
+  public static String getRevision() {
+    return HDDS_VERSION_INFO._getRevision();
+  }
+
+  /**
+   * Get the branch on which this originated.
+   * @return The branch name, e.g. "trunk" or "branches/branch-0.20"
+   */
+  public static String getBranch() {
+    return HDDS_VERSION_INFO._getBranch();
+  }
+
+  /**
+   * The date that HDDS was compiled.
+   * @return the compilation date in unix date format
+   */
+  public static String getDate() {
+    return HDDS_VERSION_INFO._getDate();
+  }
+
+  /**
+   * The user that compiled HDDS.
+   * @return the username of the user
+   */
+  public static String getUser() {
+    return HDDS_VERSION_INFO._getUser();
+  }
+
+  /**
+   * Get the URL for the HDDS repository.
+   * @return the URL of the Hdds repository
+   */
+  public static String getUrl() {
+    return HDDS_VERSION_INFO._getUrl();
+  }
+
+  /**
+   * Get the checksum of the source files from which HDDS was built.
+   * @return the checksum of the source files
+   */
+  public static String getSrcChecksum() {
+    return HDDS_VERSION_INFO._getSrcChecksum();
+  }
+
+  /**
+   * Returns the buildVersion which includes version,
+   * revision, user and date.
+   * @return the buildVersion
+   */
+  public static String getBuildVersion(){
+    return HDDS_VERSION_INFO._getBuildVersion();
+  }
+
+  /**
+   * Returns the protoc version used for the build.
+   * @return the protoc version
+   */
+  public static String getProtocVersion(){
+    return HDDS_VERSION_INFO._getProtocVersion();
+  }
+
+  public static void main(String[] args) {
+    System.out.println("Using HDDS " + getVersion());
+    System.out.println("Source code repository " + getUrl() + " -r " +
+        getRevision());
+    System.out.println("Compiled by " + getUser() + " on " + getDate());
+    System.out.println("Compiled with protoc " + getProtocVersion());
+    System.out.println("From source with checksum " + getSrcChecksum());
+    LOG.debug("This command was run using " +
+        ClassUtil.findContainingJar(HddsVersionInfo.class));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d96bc6e/hadoop-hdds/common/src/main/resources/hdds-version-info.properties
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/hdds-version-info.properties b/hadoop-hdds/common/src/main/resources/hdds-version-info.properties
new file mode 100644
index 0000000..2cbd817
--- /dev/null
+++ b/hadoop-hdds/common/src/main/resources/hdds-version-info.properties
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+
+version=${declared.hdds.version}
+revision=${version-info.scm.commit}
+branch=${version-info.scm.branch}
+user=${user.name}
+date=${version-info.build.time}
+url=${version-info.scm.uri}
+srcChecksum=${version-info.source.md5}
+protocVersion=${protobuf.version}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d96bc6e/hadoop-ozone/common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/pom.xml b/hadoop-ozone/common/pom.xml
index 83d023e..ea5eb46 100644
--- a/hadoop-ozone/common/pom.xml
+++ b/hadoop-ozone/common/pom.xml
@@ -29,8 +29,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <packaging>jar</packaging>
 
   <properties>
+    <ozone.version>0.2.1-SNAPSHOT</ozone.version>
+    <ozone.release>Acadia</ozone.release>
     <hadoop.component>ozone</hadoop.component>
     <is.hadoop.component>true</is.hadoop.component>
+    <declared.ozone.version>${ozone.version}</declared.ozone.version>
   </properties>
 
   <dependencies>
@@ -38,12 +41,44 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   </dependencies>
 
   <build>
+    <resources>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <excludes>
+          <exclude>ozone-version-info.properties</exclude>
+        </excludes>
+        <filtering>false</filtering>
+      </resource>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <includes>
+          <include>ozone-version-info.properties</include>
+        </includes>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
     <plugins>
       <plugin>
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-maven-plugins</artifactId>
         <executions>
           <execution>
+            <id>version-info</id>
+            <phase>generate-resources</phase>
+            <goals>
+              <goal>version-info</goal>
+            </goals>
+            <configuration>
+              <source>
+                <directory>${basedir}/../</directory>
+                <includes>
+                  <include>*/src/main/java/**/*.java</include>
+                  <include>*/src/main/proto/*.proto</include>
+                </includes>
+              </source>
+            </configuration>
+          </execution>
+          <execution>
             <id>compile-protoc</id>
             <goals>
               <goal>protoc</goal>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d96bc6e/hadoop-ozone/common/src/main/bin/ozone
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/bin/ozone b/hadoop-ozone/common/src/main/bin/ozone
index 5d1b6bc..75ceeb7 100755
--- a/hadoop-ozone/common/src/main/bin/ozone
+++ b/hadoop-ozone/common/src/main/bin/ozone
@@ -117,7 +117,7 @@ function ozonecmd_case
       HADOOP_CLASSNAME=org.apache.hadoop.ozone.scm.cli.SCMCLI
     ;;
     version)
-      HADOOP_CLASSNAME=org.apache.hadoop.util.VersionInfo
+      HADOOP_CLASSNAME=org.apache.hadoop.ozone.util.OzoneVersionInfo
     ;;
     genconf)
       HADOOP_CLASSNAME=org.apache.hadoop.ozone.genconf.GenerateOzoneRequiredConfigurations

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d96bc6e/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/OzoneVersionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/OzoneVersionInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/OzoneVersionInfo.java
new file mode 100644
index 0000000..d476748
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/OzoneVersionInfo.java
@@ -0,0 +1,213 @@
+/*
+ * 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.ozone.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.ClassUtil;
+import org.apache.hadoop.util.ThreadUtil;
+import org.apache.hadoop.utils.HddsVersionInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+
+/**
+ * This class returns build information about Hadoop components.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class OzoneVersionInfo {
+  private static final Logger LOG = LoggerFactory.getLogger(OzoneVersionInfo.class);
+
+  private Properties info;
+
+  protected OzoneVersionInfo(String component) {
+    info = new Properties();
+    String versionInfoFile = component + "-version-info.properties";
+    InputStream is = null;
+    try {
+      is = ThreadUtil.getResourceAsStream(OzoneVersionInfo.class.getClassLoader(),
+          versionInfoFile);
+      info.load(is);
+    } catch (IOException ex) {
+      LoggerFactory.getLogger(getClass()).warn("Could not read '" +
+          versionInfoFile + "', " + ex.toString(), ex);
+    } finally {
+      IOUtils.closeStream(is);
+    }
+  }
+
+  protected String _getVersion() {
+    return info.getProperty("version", "Unknown");
+  }
+
+  protected String _getRelease() {
+    return info.getProperty("release", "Unknown");
+  }
+
+  protected String _getRevision() {
+    return info.getProperty("revision", "Unknown");
+  }
+
+  protected String _getBranch() {
+    return info.getProperty("branch", "Unknown");
+  }
+
+  protected String _getDate() {
+    return info.getProperty("date", "Unknown");
+  }
+
+  protected String _getUser() {
+    return info.getProperty("user", "Unknown");
+  }
+
+  protected String _getUrl() {
+    return info.getProperty("url", "Unknown");
+  }
+
+  protected String _getSrcChecksum() {
+    return info.getProperty("srcChecksum", "Unknown");
+  }
+
+  protected String _getBuildVersion(){
+    return _getVersion() +
+      " from " + _getRevision() +
+      " by " + _getUser() +
+      " source checksum " + _getSrcChecksum();
+  }
+
+  protected String _getProtocVersion() {
+    return info.getProperty("protocVersion", "Unknown");
+  }
+
+  private static OzoneVersionInfo OZONE_VERSION_INFO = new OzoneVersionInfo("ozone");
+  /**
+   * Get the Ozone version.
+   * @return the Ozone version string, eg. "0.6.3-dev"
+   */
+  public static String getVersion() {
+    return OZONE_VERSION_INFO._getVersion();
+  }
+
+  /**
+   * Get the Ozone release name.
+   * @return the Ozone release string, eg. "Acadia"
+   */
+  public static String getRelease() {
+    return OZONE_VERSION_INFO._getRelease();
+  }
+
+  /**
+   * Get the Git commit hash of the repository when compiled.
+   * @return the commit hash, eg. "18f64065d5db6208daf50b02c1b5ed4ee3ce547a"
+   */
+  public static String getRevision() {
+    return OZONE_VERSION_INFO._getRevision();
+  }
+
+  /**
+   * Get the branch on which this originated.
+   * @return The branch name, e.g. "trunk" or "branches/branch-0.20"
+   */
+  public static String getBranch() {
+    return OZONE_VERSION_INFO._getBranch();
+  }
+
+  /**
+   * The date that Ozone was compiled.
+   * @return the compilation date in unix date format
+   */
+  public static String getDate() {
+    return OZONE_VERSION_INFO._getDate();
+  }
+
+  /**
+   * The user that compiled Ozone.
+   * @return the username of the user
+   */
+  public static String getUser() {
+    return OZONE_VERSION_INFO._getUser();
+  }
+
+  /**
+   * Get the URL for the Ozone repository.
+   * @return the URL of the Ozone repository
+   */
+  public static String getUrl() {
+    return OZONE_VERSION_INFO._getUrl();
+  }
+
+  /**
+   * Get the checksum of the source files from which Ozone was built.
+   * @return the checksum of the source files
+   */
+  public static String getSrcChecksum() {
+    return OZONE_VERSION_INFO._getSrcChecksum();
+  }
+
+  /**
+   * Returns the buildVersion which includes version,
+   * revision, user and date.
+   * @return the buildVersion
+   */
+  public static String getBuildVersion(){
+    return OZONE_VERSION_INFO._getBuildVersion();
+  }
+
+  /**
+   * Returns the protoc version used for the build.
+   * @return the protoc version
+   */
+  public static String getProtocVersion(){
+    return OZONE_VERSION_INFO._getProtocVersion();
+  }
+
+  public static void main(String[] args) {
+    System.out.println(
+        "                  //////////////                 \n" +
+        "               ////////////////////              \n" +
+        "            ////////     ////////////////        \n" +
+        "           //////      ////////////////          \n" +
+        "          /////      ////////////////  /         \n" +
+        "         /////            ////////   ///         \n" +
+        "         ////           ////////    /////        \n" +
+        "        /////         ////////////////           \n" +
+        "        /////       ////////////////   //        \n" +
+        "         ////     ///////////////   /////        \n" +
+        "         /////  ///////////////     ////         \n" +
+        "          /////       //////      /////          \n" +
+        "           //////   //////       /////           \n" +
+        "             ///////////     ////////            \n" +
+        "               //////  ////////////              \n" +
+        "               ///   //////////                  \n" +
+        "              /    "+ getVersion() + "("+ getRelease() +")\n");
+    System.out.println("Source code repository " + getUrl() + " -r " +
+        getRevision());
+    System.out.println("Compiled by " + getUser() + " on " + getDate());
+    System.out.println("Compiled with protoc " + getProtocVersion());
+    System.out.println("From source with checksum " + getSrcChecksum() + "\n");
+    LOG.debug("This command was run using " +
+        ClassUtil.findContainingJar(OzoneVersionInfo.class));
+    HddsVersionInfo.main(args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d96bc6e/hadoop-ozone/common/src/main/resources/ozone-version-info.properties
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/resources/ozone-version-info.properties b/hadoop-ozone/common/src/main/resources/ozone-version-info.properties
new file mode 100644
index 0000000..599f14d
--- /dev/null
+++ b/hadoop-ozone/common/src/main/resources/ozone-version-info.properties
@@ -0,0 +1,27 @@
+#
+# 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.
+#
+
+version=${declared.ozone.version}
+release=${ozone.release}
+revision=${version-info.scm.commit}
+branch=${version-info.scm.branch}
+user=${user.name}
+date=${version-info.build.time}
+url=${version-info.scm.uri}
+srcChecksum=${version-info.source.md5}
+protocVersion=${protobuf.version}


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


[20/50] [abbrv] hadoop git commit: YARN-8331. Race condition in NM container launched after done. Contributed by Pradeep Ambati

Posted by tm...@apache.org.
YARN-8331. Race condition in NM container launched after done. Contributed by Pradeep Ambati


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

Branch: refs/heads/HADOOP-15407
Commit: cd04e954d2db27f0a15b7d1c492b7cdb656a51db
Parents: 778369e
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Aug 9 10:17:34 2018 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Aug 9 10:17:34 2018 -0500

----------------------------------------------------------------------
 .../container/ContainerImpl.java                | 13 +++++-
 .../launcher/ContainerLaunch.java               | 12 ++---
 .../launcher/ContainersLauncher.java            | 14 +++++-
 .../container/TestContainer.java                | 46 ++++++++++++++++++--
 4 files changed, 71 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd04e954/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index f76e682..e4cbfdc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -384,7 +384,7 @@ public class ContainerImpl implements Container {
        UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.SCHEDULED, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER,
-        new KillBeforeRunningTransition())
+        new KillTransition())
     .addTransition(ContainerState.SCHEDULED, ContainerState.SCHEDULED,
         ContainerEventType.UPDATE_CONTAINER_TOKEN,
         new NotifyContainerSchedulerOfUpdateTransition())
@@ -618,6 +618,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.EXITED_WITH_SUCCESS,
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerEventType.UPDATE_CONTAINER_TOKEN)
+    .addTransition(ContainerState.EXITED_WITH_SUCCESS,
+        ContainerState.EXITED_WITH_SUCCESS,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST)
 
     // From EXITED_WITH_FAILURE State
     .addTransition(ContainerState.EXITED_WITH_FAILURE, ContainerState.DONE,
@@ -635,6 +638,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.EXITED_WITH_FAILURE,
         ContainerState.EXITED_WITH_FAILURE,
         ContainerEventType.UPDATE_CONTAINER_TOKEN)
+    .addTransition(ContainerState.EXITED_WITH_FAILURE,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST)
 
     // From KILLING State.
     .addTransition(ContainerState.KILLING,
@@ -694,6 +700,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
         ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
         ContainerEventType.UPDATE_CONTAINER_TOKEN)
+    .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
+        ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST)
 
     // From DONE
     .addTransition(ContainerState.DONE, ContainerState.DONE,
@@ -714,6 +723,8 @@ public class ContainerImpl implements Container {
     // No transition - assuming container is on its way to completion
     .addTransition(ContainerState.DONE, ContainerState.DONE,
         ContainerEventType.UPDATE_CONTAINER_TOKEN)
+    .addTransition(ContainerState.DONE, ContainerState.DONE,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST)
 
     // create the topology tables
     .installTopology();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd04e954/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
index 04295e1..23ad408 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
@@ -556,14 +556,10 @@ public class ContainerLaunch implements Callable<Integer> {
         || exitCode == ExitCode.TERMINATED.getExitCode()) {
       // If the process was killed, Send container_cleanedup_after_kill and
       // just break out of this method.
-
-      // If Container was killed before starting... NO need to do this.
-      if (!killedBeforeStart) {
-        dispatcher.getEventHandler().handle(
-            new ContainerExitEvent(containerId,
-                ContainerEventType.CONTAINER_KILLED_ON_REQUEST, exitCode,
-                diagnosticInfo.toString()));
-      }
+      dispatcher.getEventHandler().handle(
+          new ContainerExitEvent(containerId,
+              ContainerEventType.CONTAINER_KILLED_ON_REQUEST, exitCode,
+              diagnosticInfo.toString()));
     } else if (exitCode != 0) {
       handleContainerExitWithFailure(containerId, exitCode, containerLogDir,
           diagnosticInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd04e954/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
index cfd5d6a..7870f86 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
@@ -23,6 +23,11 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ExecutorService;
+
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -151,7 +156,14 @@ public class ContainersLauncher extends AbstractService
       case CLEANUP_CONTAINER_FOR_REINIT:
         ContainerLaunch launcher = running.remove(containerId);
         if (launcher == null) {
-          // Container not launched. So nothing needs to be done.
+          // Container not launched.
+          // triggering KILLING to CONTAINER_CLEANEDUP_AFTER_KILL transition.
+          dispatcher.getEventHandler().handle(
+              new ContainerExitEvent(containerId,
+                  ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+                  Shell.WINDOWS ? ContainerExecutor.ExitCode.FORCE_KILLED.getExitCode() :
+                  ContainerExecutor.ExitCode.TERMINATED.getExitCode(),
+                  "Container terminated before launch."));
           return;
         }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd04e954/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
index edf26d4..71cabdd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.refEq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.times;
@@ -664,6 +665,17 @@ public class TestContainer {
       ContainerLaunch launcher = wc.launcher.running.get(wc.c.getContainerId());
       wc.killContainer();
       assertEquals(ContainerState.KILLING, wc.c.getContainerState());
+
+      // check that container cleanup hasn't started at this point.
+      LocalizationCleanupMatcher cleanupResources =
+          new LocalizationCleanupMatcher(wc.c);
+      verify(wc.localizerBus, times(0)).handle(argThat(cleanupResources));
+
+      // check if containerlauncher cleans up the container launch.
+      verify(wc.launcherBus)
+          .handle(refEq(new ContainersLauncherEvent(wc.c,
+              ContainersLauncherEventType.CLEANUP_CONTAINER), "timestamp"));
+
       launcher.call();
       wc.drainDispatcherEvents();
       assertEquals(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
@@ -676,6 +688,7 @@ public class TestContainer {
       assertEquals(ContainerState.DONE, wc.c.getContainerState());
       assertEquals(killed + 1, metrics.getKilledContainers());
       assertEquals(0, metrics.getRunningContainers());
+      assertEquals(0, wc.launcher.running.size());
     } finally {
       if (wc != null) {
         wc.finished();
@@ -1145,7 +1158,7 @@ public class TestContainer {
     ResourcesReleasedMatcher matchesReq =
         new ResourcesReleasedMatcher(wc.localResources, EnumSet.of(
             LocalResourceVisibility.PUBLIC, LocalResourceVisibility.PRIVATE,
-            LocalResourceVisibility.APPLICATION));
+            LocalResourceVisibility.APPLICATION), wc.c);
     verify(wc.localizerBus, atLeastOnce()).handle(argThat(matchesReq));
   }
 
@@ -1161,13 +1174,35 @@ public class TestContainer {
             wc.c.getContainerId().toString())));
   }
 
-  private static class ResourcesReleasedMatcher extends
+  // Argument matcher for matching container localization cleanup event.
+  private static class LocalizationCleanupMatcher extends
       ArgumentMatcher<LocalizationEvent> {
+    Container c;
+
+    LocalizationCleanupMatcher(Container c){
+      this.c = c;
+    }
+
+    @Override
+    public boolean matches(Object o) {
+      if (!(o instanceof ContainerLocalizationCleanupEvent)) {
+        return false;
+      }
+      ContainerLocalizationCleanupEvent evt =
+          (ContainerLocalizationCleanupEvent) o;
+
+      return (evt.getContainer() == c);
+    }
+  }
+
+  private static class ResourcesReleasedMatcher extends
+      LocalizationCleanupMatcher {
     final HashSet<LocalResourceRequest> resources =
         new HashSet<LocalResourceRequest>();
 
     ResourcesReleasedMatcher(Map<String, LocalResource> allResources,
-        EnumSet<LocalResourceVisibility> vis) throws URISyntaxException {
+        EnumSet<LocalResourceVisibility> vis, Container c) throws URISyntaxException {
+      super(c);
       for (Entry<String, LocalResource> e : allResources.entrySet()) {
         if (vis.contains(e.getValue().getVisibility())) {
           resources.add(new LocalResourceRequest(e.getValue()));
@@ -1177,9 +1212,12 @@ public class TestContainer {
 
     @Override
     public boolean matches(Object o) {
-      if (!(o instanceof ContainerLocalizationCleanupEvent)) {
+      // match event type and container.
+      if(!super.matches(o)){
         return false;
       }
+
+      // match resources.
       ContainerLocalizationCleanupEvent evt =
           (ContainerLocalizationCleanupEvent) o;
       final HashSet<LocalResourceRequest> expected =


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


[49/50] [abbrv] hadoop git commit: HADOOP-15560. ABFS: removed dependency injection and unnecessary dependencies. Contributed by Da Zhou.

Posted by tm...@apache.org.
HADOOP-15560. ABFS: removed dependency injection and unnecessary dependencies.
Contributed by Da Zhou.


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

Branch: refs/heads/HADOOP-15407
Commit: 62ea8ad9839eb84a9efcc69bdd5dd8d9b1e5f5dd
Parents: fdf5f4c
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 3 18:55:10 2018 +0200
Committer: Thomas Marquardt <tm...@microsoft.com>
Committed: Sat Aug 11 03:42:27 2018 +0000

----------------------------------------------------------------------
 hadoop-tools/hadoop-azure/pom.xml               |  18 -
 .../src/config/checkstyle-suppressions.xml      |   2 +-
 .../hadoop/fs/azurebfs/AzureBlobFileSystem.java |  88 ++-
 .../fs/azurebfs/AzureBlobFileSystemStore.java   | 701 +++++++++++++++++++
 .../exceptions/ServiceResolutionException.java  |  36 -
 .../services/AbfsHttpClientFactory.java         |  39 --
 .../contracts/services/AbfsHttpService.java     | 162 -----
 .../contracts/services/AbfsServiceProvider.java |  40 --
 .../services/ConfigurationService.java          | 143 ----
 .../contracts/services/InjectableService.java   |  30 -
 .../contracts/services/TracingService.java      |  66 --
 .../hadoop/fs/azurebfs/services/AbfsClient.java |   7 +-
 .../fs/azurebfs/services/AbfsConfiguration.java | 297 ++++++++
 .../services/AbfsHttpClientFactoryImpl.java     | 116 ---
 .../azurebfs/services/AbfsHttpServiceImpl.java  | 693 ------------------
 .../services/AbfsServiceInjectorImpl.java       |  81 ---
 .../services/AbfsServiceProviderImpl.java       |  96 ---
 .../services/ConfigurationServiceImpl.java      | 317 ---------
 .../services/ExponentialRetryPolicy.java        |   9 +-
 .../azurebfs/services/LoggerSpanReceiver.java   |  74 --
 .../azurebfs/services/TracingServiceImpl.java   | 134 ----
 .../fs/azurebfs/DependencyInjectedTest.java     |  59 +-
 .../azurebfs/ITestAzureBlobFileSystemE2E.java   |   7 +-
 .../ITestAzureBlobFileSystemRandomRead.java     |   7 +-
 .../azurebfs/ITestFileSystemInitialization.java |  23 +-
 .../fs/azurebfs/ITestFileSystemProperties.java  | 126 ++++
 .../azurebfs/ITestFileSystemRegistration.java   |  23 +-
 .../ITestAzureBlobFileSystemBasics.java         |  11 +-
 .../services/ITestAbfsHttpServiceImpl.java      | 122 ----
 .../services/ITestReadWriteAndSeek.java         |   8 +-
 .../services/ITestTracingServiceImpl.java       |  79 ---
 .../services/MockAbfsHttpClientFactoryImpl.java |  69 --
 .../services/MockAbfsServiceInjectorImpl.java   |  50 --
 .../services/MockServiceProviderImpl.java       |  36 -
 .../TestAbfsConfigurationFieldsValidation.java  | 149 ++++
 ...estConfigurationServiceFieldsValidation.java | 149 ----
 .../utils/CleanUpAbfsTestContainer.java         |  68 ++
 37 files changed, 1432 insertions(+), 2703 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml
index d4046ef..cbd4dfb 100644
--- a/hadoop-tools/hadoop-azure/pom.xml
+++ b/hadoop-tools/hadoop-azure/pom.xml
@@ -150,12 +150,6 @@
     </dependency>
 
     <dependency>
-      <groupId>org.threadly</groupId>
-      <artifactId>threadly</artifactId>
-      <scope>compile</scope>
-    </dependency>
-
-    <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-core</artifactId>
       <scope>compile</scope>
@@ -186,18 +180,6 @@
     </dependency>
 
     <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core</artifactId>
-      <scope>compile</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
-      <scope>compile</scope>
-    </dependency>
-
-    <dependency>
       <groupId>com.google.inject</groupId>
       <artifactId>guice</artifactId>
       <scope>compile</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
index 0204355..751a227 100644
--- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
+++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
@@ -43,5 +43,5 @@
 
 <suppressions>
     <suppress checks="ParameterNumber|MagicNumber"
-              files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]services[\\/]AbfsHttpServiceImpl.java"/>
+              files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]AzureBlobFileSystemStore.java"/>
 </suppressions>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
index 707c81e..cf5acbb 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
@@ -39,10 +39,8 @@ import com.google.common.base.Preconditions;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.threadly.util.ExceptionUtils;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
-import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -58,10 +56,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException;
@@ -70,7 +64,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
-import org.apache.htrace.core.TraceScope;
 
 /**
  * A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
@@ -85,10 +78,7 @@ public class AzureBlobFileSystem extends FileSystem {
   private UserGroupInformation userGroupInformation;
   private String user;
   private String primaryUserGroup;
-  private AbfsServiceProvider abfsServiceProvider;
-  private TracingService tracingService;
-  private AbfsHttpService abfsHttpService;
-  private ConfigurationService configurationService;
+  private AzureBlobFileSystemStore abfsStore;
   private boolean isClosed;
 
   @Override
@@ -96,18 +86,8 @@ public class AzureBlobFileSystem extends FileSystem {
       throws IOException {
     uri = ensureAuthority(uri, configuration);
     super.initialize(uri, configuration);
-
     setConf(configuration);
 
-    try {
-      this.abfsServiceProvider = AbfsServiceProviderImpl.create(configuration);
-      this.tracingService = abfsServiceProvider.get(TracingService.class);
-      this.abfsHttpService = abfsServiceProvider.get(AbfsHttpService.class);
-      this.configurationService = abfsServiceProvider.get(ConfigurationService.class);
-    } catch (AzureBlobFileSystemException exception) {
-      throw new IOException(exception);
-    }
-
     this.LOG.debug(
         "Initializing AzureBlobFileSystem for {}", uri);
 
@@ -115,13 +95,14 @@ public class AzureBlobFileSystem extends FileSystem {
     this.userGroupInformation = UserGroupInformation.getCurrentUser();
     this.user = userGroupInformation.getUserName();
     this.primaryUserGroup = userGroupInformation.getPrimaryGroupName();
+    this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation);
 
     this.LOG.debug(
         "Initializing NativeAzureFileSystem for {}", uri);
 
     this.setWorkingDirectory(this.getHomeDirectory());
 
-    if (this.configurationService.getCreateRemoteFileSystemDuringInitialization()) {
+    if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) {
       this.createFileSystem();
     }
 
@@ -143,7 +124,7 @@ public class AzureBlobFileSystem extends FileSystem {
         "AzureBlobFileSystem.open path: {} bufferSize: {}", path.toString(), bufferSize);
 
     try {
-      InputStream inputStream = abfsHttpService.openFileForRead(this, makeQualified(path), statistics);
+      InputStream inputStream = abfsStore.openFileForRead(makeQualified(path), statistics);
       return new FSDataInputStream(inputStream);
     } catch(AzureBlobFileSystemException ex) {
       checkException(path, ex);
@@ -162,7 +143,7 @@ public class AzureBlobFileSystem extends FileSystem {
         blockSize);
 
     try {
-      OutputStream outputStream = abfsHttpService.createFile(this, makeQualified(f), overwrite);
+      OutputStream outputStream = abfsStore.createFile(makeQualified(f), overwrite);
       return new FSDataOutputStream(outputStream, statistics);
     } catch(AzureBlobFileSystemException ex) {
       checkException(f, ex);
@@ -221,7 +202,7 @@ public class AzureBlobFileSystem extends FileSystem {
         bufferSize);
 
     try {
-      OutputStream outputStream = abfsHttpService.openFileForWrite(this, makeQualified(f), false);
+      OutputStream outputStream = abfsStore.openFileForWrite(makeQualified(f), false);
       return new FSDataOutputStream(outputStream, statistics);
     } catch(AzureBlobFileSystemException ex) {
       checkException(f, ex);
@@ -251,7 +232,7 @@ public class AzureBlobFileSystem extends FileSystem {
         adjustedDst = new Path(dst, sourceFileName);
       }
 
-      abfsHttpService.rename(this, makeQualified(src), makeQualified(adjustedDst));
+      abfsStore.rename(makeQualified(src), makeQualified(adjustedDst));
       return true;
     } catch(AzureBlobFileSystemException ex) {
       checkException(
@@ -281,7 +262,7 @@ public class AzureBlobFileSystem extends FileSystem {
     }
 
     try {
-      abfsHttpService.delete(this, makeQualified(f), recursive);
+      abfsStore.delete(makeQualified(f), recursive);
       return true;
     } catch (AzureBlobFileSystemException ex) {
       checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND);
@@ -296,7 +277,7 @@ public class AzureBlobFileSystem extends FileSystem {
         "AzureBlobFileSystem.listStatus path: {}", f.toString());
 
     try {
-      FileStatus[] result = abfsHttpService.listStatus(this, makeQualified(f));
+      FileStatus[] result = abfsStore.listStatus(makeQualified(f));
       return result;
     } catch (AzureBlobFileSystemException ex) {
       checkException(f, ex);
@@ -316,7 +297,7 @@ public class AzureBlobFileSystem extends FileSystem {
     }
 
     try {
-      abfsHttpService.createDirectory(this, makeQualified(f));
+      abfsStore.createDirectory(makeQualified(f));
       return true;
     } catch (AzureBlobFileSystemException ex) {
       checkException(f, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS);
@@ -332,13 +313,7 @@ public class AzureBlobFileSystem extends FileSystem {
 
     super.close();
     this.LOG.debug("AzureBlobFileSystem.close");
-
-    try {
-      abfsHttpService.closeFileSystem(this);
-    } catch (AzureBlobFileSystemException ex) {
-      checkException(null, ex);
-      this.isClosed = true;
-    }
+    this.isClosed = true;
   }
 
   @Override
@@ -346,7 +321,7 @@ public class AzureBlobFileSystem extends FileSystem {
     this.LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f.toString());
 
     try {
-      return abfsHttpService.getFileStatus(this, makeQualified(f));
+      return abfsStore.getFileStatus(makeQualified(f));
     } catch(AzureBlobFileSystemException ex) {
       checkException(f, ex);
       return null;
@@ -397,7 +372,7 @@ public class AzureBlobFileSystem extends FileSystem {
     if (file.getLen() < start) {
       return new BlockLocation[0];
     }
-    final String blobLocationHost = this.configurationService.getAzureBlockLocationHost();
+    final String blobLocationHost = this.abfsStore.getAbfsConfiguration().getAzureBlockLocationHost();
 
     final String[] name = { blobLocationHost };
     final String[] host = { blobLocationHost };
@@ -477,12 +452,10 @@ public class AzureBlobFileSystem extends FileSystem {
     this.LOG.debug(
         "AzureBlobFileSystem.createFileSystem uri: {}", uri);
     try {
-      abfsHttpService.createFilesystem(this);
+      this.abfsStore.createFilesystem();
     } catch (AzureBlobFileSystemException ex) {
       checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS);
     }
-
-
   }
 
   private URI ensureAuthority(URI uri, final Configuration conf) {
@@ -540,25 +513,19 @@ public class AzureBlobFileSystem extends FileSystem {
       final Callable<T> callableFileOperation,
       T defaultResultValue) throws IOException {
 
-    final TraceScope traceScope = tracingService.traceBegin(scopeDescription);
     try {
       final T executionResult = callableFileOperation.call();
       return new FileSystemOperation(executionResult, null);
     } catch (AbfsRestOperationException abfsRestOperationException) {
       return new FileSystemOperation(defaultResultValue, abfsRestOperationException);
     } catch (AzureBlobFileSystemException azureBlobFileSystemException) {
-      tracingService.traceException(traceScope, azureBlobFileSystemException);
       throw new IOException(azureBlobFileSystemException);
     } catch (Exception exception) {
       if (exception instanceof ExecutionException) {
-        exception = (Exception) ExceptionUtils.getRootCause(exception);
+        exception = (Exception) getRootCause(exception);
       }
-
       final FileSystemOperationUnhandledException fileSystemOperationUnhandledException = new FileSystemOperationUnhandledException(exception);
-      tracingService.traceException(traceScope, fileSystemOperationUnhandledException);
       throw new IOException(fileSystemOperationUnhandledException);
-    } finally {
-      tracingService.traceEnd(traceScope);
     }
   }
 
@@ -590,6 +557,26 @@ public class AzureBlobFileSystem extends FileSystem {
     }
   }
 
+  /**
+   * Gets the root cause of a provided {@link Throwable}.  If there is no cause for the
+   * {@link Throwable} provided into this function, the original {@link Throwable} is returned.
+   *
+   * @param throwable starting {@link Throwable}
+   * @return root cause {@link Throwable}
+   */
+  private Throwable getRootCause(Throwable throwable) {
+    if (throwable == null) {
+      throw new IllegalArgumentException("throwable can not be null");
+    }
+
+    Throwable result = throwable;
+    while (result.getCause() != null) {
+      result = result.getCause();
+    }
+
+    return result;
+  }
+
   @VisibleForTesting
   FileSystem.Statistics getFsStatistics() {
     return this.statistics;
@@ -609,4 +596,9 @@ public class AzureBlobFileSystem extends FileSystem {
       return this.exception != null;
     }
   }
+
+  @VisibleForTesting
+  AzureBlobFileSystemStore getAbfsStore() {
+    return this.abfsStore;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
new file mode 100644
index 0000000..134277f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
@@ -0,0 +1,701 @@
+/**
+ * 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.azurebfs;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CharsetEncoder;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Hashtable;
+import java.util.Map;
+import java.util.Set;
+import javax.xml.bind.DatatypeConverter;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy;
+import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.http.client.utils.URIBuilder;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.util.Time.now;
+
+/**
+ * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class AzureBlobFileSystemStore {
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystemStore.class);
+
+  private AbfsClient client;
+  private URI uri;
+  private final UserGroupInformation userGroupInformation;
+  private static final String DATE_TIME_PATTERN = "E, dd MMM yyyy HH:mm:ss 'GMT'";
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int LIST_MAX_RESULTS = 5000;
+  private static final int DELETE_DIRECTORY_TIMEOUT_MILISECONDS = 180000;
+  private static final int RENAME_TIMEOUT_MILISECONDS = 180000;
+
+  private final AbfsConfiguration abfsConfiguration;
+  private final Set<String> azureAtomicRenameDirSet;
+
+
+  public AzureBlobFileSystemStore(URI uri, boolean isSeure, Configuration configuration, UserGroupInformation userGroupInformation)
+          throws AzureBlobFileSystemException {
+    this.uri = uri;
+    try {
+      this.abfsConfiguration = new AbfsConfiguration(configuration);
+    } catch (IllegalAccessException exception) {
+      throw new FileSystemOperationUnhandledException(exception);
+    }
+
+    this.userGroupInformation = userGroupInformation;
+    this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
+
+    initializeClient(uri, isSeure);
+  }
+
+  @VisibleForTesting
+  URIBuilder getURIBuilder(final String hostName, boolean isSecure) {
+    String scheme = isSecure ? FileSystemUriSchemes.HTTPS_SCHEME : FileSystemUriSchemes.HTTP_SCHEME;
+
+    final URIBuilder uriBuilder = new URIBuilder();
+    uriBuilder.setScheme(scheme);
+    uriBuilder.setHost(hostName);
+
+    return uriBuilder;
+  }
+
+  public AbfsConfiguration getAbfsConfiguration() {
+    return this.abfsConfiguration;
+  }
+
+  public Hashtable<String, String> getFilesystemProperties() throws AzureBlobFileSystemException {
+    this.LOG.debug(
+            "getFilesystemProperties for filesystem: {}",
+            client.getFileSystem());
+
+    final Hashtable<String, String> parsedXmsProperties;
+
+    final AbfsRestOperation op = client.getFilesystemProperties();
+    final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
+
+    parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
+
+    return parsedXmsProperties;
+  }
+
+  public void setFilesystemProperties(final Hashtable<String, String> properties) throws AzureBlobFileSystemException {
+    if (properties == null || properties.size() == 0) {
+      return;
+    }
+
+    this.LOG.debug(
+            "setFilesystemProperties for filesystem: {} with properties: {}",
+            client.getFileSystem(),
+            properties);
+
+    final String commaSeparatedProperties;
+    try {
+      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
+    } catch (CharacterCodingException ex) {
+      throw new InvalidAbfsRestOperationException(ex);
+    }
+
+    client.setFilesystemProperties(commaSeparatedProperties);
+  }
+
+  public Hashtable<String, String> getPathProperties(final Path path) throws AzureBlobFileSystemException {
+    this.LOG.debug(
+            "getPathProperties for filesystem: {} path: {}",
+            client.getFileSystem(),
+            path.toString());
+
+    final Hashtable<String, String> parsedXmsProperties;
+    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+    final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
+
+    parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
+
+    return parsedXmsProperties;
+  }
+
+  public void setPathProperties(final Path path, final Hashtable<String, String> properties) throws AzureBlobFileSystemException {
+    this.LOG.debug(
+            "setFilesystemProperties for filesystem: {} path: {} with properties: {}",
+            client.getFileSystem(),
+            path.toString(),
+            properties);
+
+    final String commaSeparatedProperties;
+    try {
+      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
+    } catch (CharacterCodingException ex) {
+      throw new InvalidAbfsRestOperationException(ex);
+    }
+    client.setPathProperties("/" + getRelativePath(path), commaSeparatedProperties);
+  }
+
+  public void createFilesystem() throws AzureBlobFileSystemException {
+    this.LOG.debug(
+            "createFilesystem for filesystem: {}",
+            client.getFileSystem());
+
+    client.createFilesystem();
+  }
+
+  public void deleteFilesystem() throws AzureBlobFileSystemException {
+    this.LOG.debug(
+            "deleteFilesystem for filesystem: {}",
+            client.getFileSystem());
+
+    client.deleteFilesystem();
+  }
+
+  public OutputStream createFile(final Path path, final boolean overwrite) throws AzureBlobFileSystemException {
+    this.LOG.debug(
+            "createFile filesystem: {} path: {} overwrite: {}",
+            client.getFileSystem(),
+            path.toString(),
+            overwrite);
+
+    client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite);
+
+    final OutputStream outputStream;
+    outputStream = new FSDataOutputStream(
+            new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0,
+                    abfsConfiguration.getWriteBufferSize()), null);
+    return outputStream;
+  }
+
+  public Void createDirectory(final Path path) throws AzureBlobFileSystemException {
+    this.LOG.debug(
+            "createDirectory filesystem: {} path: {} overwrite: {}",
+            client.getFileSystem(),
+            path.toString());
+
+    client.createPath("/" + getRelativePath(path), false, true);
+
+    return null;
+  }
+
+  public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException {
+
+    this.LOG.debug(
+            "openFileForRead filesystem: {} path: {}",
+            client.getFileSystem(),
+            path.toString());
+
+    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+    final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+    final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
+    final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+
+    if (parseIsDirectory(resourceType)) {
+      throw new AbfsRestOperationException(
+              AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+              AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+              "openFileForRead must be used with files and not directories",
+              null);
+    }
+
+    // Add statistics for InputStream
+    return new FSDataInputStream(
+            new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
+                    abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag));
+  }
+
+  public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws
+          AzureBlobFileSystemException {
+    this.LOG.debug(
+            "openFileForWrite filesystem: {} path: {} overwrite: {}",
+            client.getFileSystem(),
+            path.toString(),
+            overwrite);
+
+    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+    final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+    final Long contentLength = Long.valueOf(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
+
+    if (parseIsDirectory(resourceType)) {
+      throw new AbfsRestOperationException(
+              AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+              AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+              "openFileForRead must be used with files and not directories",
+              null);
+    }
+
+    final long offset = overwrite ? 0 : contentLength;
+
+    final OutputStream outputStream;
+    outputStream = new FSDataOutputStream(
+            new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
+                    offset, abfsConfiguration.getWriteBufferSize()), null);
+    return outputStream;
+  }
+
+  public void rename(final Path source, final Path destination) throws
+          AzureBlobFileSystemException {
+
+    if (isAtomicRenameKey(source.getName())) {
+      this.LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename,"
+              +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account.");
+    }
+
+    this.LOG.debug(
+            "renameAsync filesystem: {} source: {} destination: {}",
+            client.getFileSystem(),
+            source.toString(),
+            destination.toString());
+
+    String continuation = null;
+    long deadline = now() + RENAME_TIMEOUT_MILISECONDS;
+
+    do {
+      if (now() > deadline) {
+        LOG.debug(
+                "Rename {} to {} timed out.",
+                source,
+                destination);
+
+        throw new TimeoutException("Rename timed out.");
+      }
+
+      AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source),
+              AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation);
+      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+
+    } while (continuation != null && !continuation.isEmpty());
+  }
+
+  public void delete(final Path path, final boolean recursive) throws
+          AzureBlobFileSystemException {
+
+    this.LOG.debug(
+            "delete filesystem: {} path: {} recursive: {}",
+            client.getFileSystem(),
+            path.toString(),
+            String.valueOf(recursive));
+
+    String continuation = null;
+    long deadline = now() + DELETE_DIRECTORY_TIMEOUT_MILISECONDS;
+
+    do {
+      if (now() > deadline) {
+        this.LOG.debug(
+                "Delete directory {} timed out.", path);
+
+        throw new TimeoutException("Delete directory timed out.");
+      }
+
+      AbfsRestOperation op = client.deletePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation);
+      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+
+    } while (continuation != null && !continuation.isEmpty());
+  }
+
+  public FileStatus getFileStatus(final Path path) throws IOException {
+
+    this.LOG.debug(
+            "getFileStatus filesystem: {} path: {}",
+            client.getFileSystem(),
+            path.toString());
+
+    if (path.isRoot()) {
+      AbfsRestOperation op = client.getFilesystemProperties();
+      final long blockSize = abfsConfiguration.getAzureBlockSize();
+      final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+      final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
+      return new VersionedFileStatus(
+              userGroupInformation.getUserName(),
+              userGroupInformation.getPrimaryGroupName(),
+              0,
+              true,
+              1,
+              blockSize,
+              parseLastModifiedTime(lastModified).getMillis(),
+              path,
+              eTag);
+    } else {
+      AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+      final long blockSize = abfsConfiguration.getAzureBlockSize();
+      final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+      final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
+      final String contentLength = op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH);
+      final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+
+      return new VersionedFileStatus(
+              userGroupInformation.getUserName(),
+              userGroupInformation.getPrimaryGroupName(),
+              parseContentLength(contentLength),
+              parseIsDirectory(resourceType),
+              1,
+              blockSize,
+              parseLastModifiedTime(lastModified).getMillis(),
+              path,
+              eTag);
+    }
+  }
+
+  public FileStatus[] listStatus(final Path path) throws IOException {
+    this.LOG.debug(
+            "listStatus filesystem: {} path: {}",
+            client.getFileSystem(),
+            path.toString());
+
+    String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path);
+    String continuation = null;
+    ArrayList<FileStatus> fileStatuses = new ArrayList<>();
+
+    do {
+      AbfsRestOperation op = client.listPath(relativePath, false, LIST_MAX_RESULTS, continuation);
+      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+      ListResultSchema retrievedSchema = op.getResult().getListResultSchema();
+      if (retrievedSchema == null) {
+        throw new AbfsRestOperationException(
+                AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+                AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+                "listStatusAsync path not found",
+                null, op.getResult());
+      }
+
+      long blockSize = abfsConfiguration.getAzureBlockSize();
+
+      for (ListResultEntrySchema entry : retrievedSchema.paths()) {
+        long lastModifiedMillis = 0;
+        long contentLength = entry.contentLength() == null ? 0 : entry.contentLength();
+        boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory();
+        if (entry.lastModified() != null && !entry.lastModified().isEmpty()) {
+          final DateTime dateTime = DateTime.parse(
+                  entry.lastModified(),
+                  DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
+          lastModifiedMillis = dateTime.getMillis();
+        }
+
+        Path entryPath = new Path(File.separator + entry.name());
+        entryPath = entryPath.makeQualified(this.uri, entryPath);
+
+        fileStatuses.add(
+                new VersionedFileStatus(
+                        userGroupInformation.getUserName(),
+                        userGroupInformation.getPrimaryGroupName(),
+                        contentLength,
+                        isDirectory,
+                        1,
+                        blockSize,
+                        lastModifiedMillis,
+                        entryPath,
+                        entry.eTag()));
+      }
+
+    } while (continuation != null && !continuation.isEmpty());
+
+    return fileStatuses.toArray(new FileStatus[0]);
+  }
+
+  public boolean isAtomicRenameKey(String key) {
+    return isKeyForDirectorySet(key, azureAtomicRenameDirSet);
+  }
+
+  private void initializeClient(URI uri, boolean isSeure) throws AzureBlobFileSystemException {
+    if (this.client != null) {
+      return;
+    }
+
+    final String authority = uri.getRawAuthority();
+    if (null == authority) {
+      throw new InvalidUriAuthorityException(uri.toString());
+    }
+
+    if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) {
+      throw new InvalidUriAuthorityException(uri.toString());
+    }
+
+    final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
+
+    if (authorityParts.length < 2 || "".equals(authorityParts[0])) {
+      final String errMsg = String
+              .format("URI '%s' has a malformed authority, expected container name. "
+                              + "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
+                      uri.toString());
+      throw new InvalidUriException(errMsg);
+    }
+
+    final String fileSystemName = authorityParts[0];
+    final String accountName = authorityParts[1];
+
+    final URIBuilder uriBuilder = getURIBuilder(accountName, isSeure);
+
+    final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName;
+
+    URL baseUrl;
+    try {
+      baseUrl = new URL(url);
+    } catch (MalformedURLException e) {
+      throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString()));
+    }
+
+    SharedKeyCredentials creds =
+            new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)),
+                    this.abfsConfiguration.getStorageAccountKey(accountName));
+
+    this.client =  new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy());
+  }
+
+  private String getRelativePath(final Path path) {
+    Preconditions.checkNotNull(path, "path");
+    final String relativePath = path.toUri().getPath();
+
+    if (relativePath.length() == 0) {
+      return relativePath;
+    }
+
+    if (relativePath.charAt(0) == Path.SEPARATOR_CHAR) {
+      if (relativePath.length() == 1) {
+        return AbfsHttpConstants.EMPTY_STRING;
+      }
+
+      return relativePath.substring(1);
+    }
+
+    return relativePath;
+  }
+
+  private long parseContentLength(final String contentLength) {
+    if (contentLength == null) {
+      return -1;
+    }
+
+    return Long.parseLong(contentLength);
+  }
+
+  private boolean parseIsDirectory(final String resourceType) {
+    return resourceType == null ? false : resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
+  }
+
+  private DateTime parseLastModifiedTime(final String lastModifiedTime) {
+    return DateTime.parse(
+            lastModifiedTime,
+            DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
+  }
+
+  private String convertXmsPropertiesToCommaSeparatedString(final Hashtable<String, String> properties) throws
+          CharacterCodingException {
+    StringBuilder commaSeparatedProperties = new StringBuilder();
+
+    final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING).newEncoder();
+
+    for (Map.Entry<String, String> propertyEntry : properties.entrySet()) {
+      String key = propertyEntry.getKey();
+      String value = propertyEntry.getValue();
+
+      Boolean canEncodeValue = encoder.canEncode(value);
+      if (!canEncodeValue) {
+        throw new CharacterCodingException();
+      }
+
+      String encodedPropertyValue = DatatypeConverter.printBase64Binary(encoder.encode(CharBuffer.wrap(value)).array());
+      commaSeparatedProperties.append(key)
+              .append(AbfsHttpConstants.EQUAL)
+              .append(encodedPropertyValue);
+
+      commaSeparatedProperties.append(AbfsHttpConstants.COMMA);
+    }
+
+    if (commaSeparatedProperties.length() != 0) {
+      commaSeparatedProperties.deleteCharAt(commaSeparatedProperties.length() - 1);
+    }
+
+    return commaSeparatedProperties.toString();
+  }
+
+  private Hashtable<String, String> parseCommaSeparatedXmsProperties(String xMsProperties) throws
+          InvalidFileSystemPropertyException, InvalidAbfsRestOperationException {
+    Hashtable<String, String> properties = new Hashtable<>();
+
+    final CharsetDecoder decoder = Charset.forName(XMS_PROPERTIES_ENCODING).newDecoder();
+
+    if (xMsProperties != null && !xMsProperties.isEmpty()) {
+      String[] userProperties = xMsProperties.split(AbfsHttpConstants.COMMA);
+
+      if (userProperties.length == 0) {
+        return properties;
+      }
+
+      for (String property : userProperties) {
+        if (property.isEmpty()) {
+          throw new InvalidFileSystemPropertyException(xMsProperties);
+        }
+
+        String[] nameValue = property.split(AbfsHttpConstants.EQUAL, 2);
+        if (nameValue.length != 2) {
+          throw new InvalidFileSystemPropertyException(xMsProperties);
+        }
+
+        byte[] decodedValue = DatatypeConverter.parseBase64Binary(nameValue[1]);
+
+        final String value;
+        try {
+          value = decoder.decode(ByteBuffer.wrap(decodedValue)).toString();
+        } catch (CharacterCodingException ex) {
+          throw new InvalidAbfsRestOperationException(ex);
+        }
+        properties.put(nameValue[0], value);
+      }
+    }
+
+    return properties;
+  }
+
+  private boolean isKeyForDirectorySet(String key, Set<String> dirSet) {
+    for (String dir : dirSet) {
+      if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) {
+        return true;
+      }
+
+      try {
+        URI uri = new URI(dir);
+        if (null == uri.getAuthority()) {
+          if (key.startsWith(dir + "/")){
+            return true;
+          }
+        }
+      } catch (URISyntaxException e) {
+        this.LOG.info("URI syntax error creating URI for {}", dir);
+      }
+    }
+
+    return false;
+  }
+
+  private static class VersionedFileStatus extends FileStatus {
+    private final String version;
+
+    VersionedFileStatus(
+            final String owner, final String group,
+            final long length, final boolean isdir, final int blockReplication,
+            final long blocksize, final long modificationTime, final Path path,
+            String version) {
+      super(length, isdir, blockReplication, blocksize, modificationTime, 0,
+              new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL),
+              owner,
+              group,
+              path);
+
+      this.version = version;
+    }
+
+    /** Compare if this object is equal to another object.
+     * @param   obj the object to be compared.
+     * @return  true if two file status has the same path name; false if not.
+     */
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      }
+
+      if (obj == null) {
+        return false;
+      }
+
+      if (this.getClass() == obj.getClass()) {
+        VersionedFileStatus other = (VersionedFileStatus) obj;
+        return this.getPath().equals(other.getPath()) && this.version.equals(other.version);
+      }
+
+      return false;
+    }
+
+    /**
+     * Returns a hash code value for the object, which is defined as
+     * the hash code of the path name.
+     *
+     * @return  a hash code value for the path name and version
+     */
+    @Override
+    public int hashCode() {
+      int hash = getPath().hashCode();
+      hash = 89 * hash + (this.version != null ? this.version.hashCode() : 0);
+      return hash;
+    }
+
+    /**
+     * Returns the version of this FileStatus
+     *
+     * @return  a string value for the FileStatus version
+     */
+    public String getVersion() {
+      return this.version;
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java
deleted file mode 100644
index 694d902..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.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.fs.azurebfs.contracts.exceptions;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
-
-/**
- * Thrown a service is either not configured to be injected or the service is not existing.
- * For service registration
- * @see AbfsServiceProviderImpl
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class ServiceResolutionException extends AzureBlobFileSystemException {
-  public ServiceResolutionException(String serviceName, Exception innerException) {
-    super(String.format("%s cannot be resolved.", serviceName), innerException);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java
deleted file mode 100644
index c433f9a..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java
+++ /dev/null
@@ -1,39 +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.fs.azurebfs.contracts.services;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
-import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
-
-/**
- * AbfsClient factory.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public interface AbfsHttpClientFactory extends InjectableService {
-  /**
-   * Creates and configures an instance of new AbfsClient
-   * @return AbfsClient instance
-   */
-  AbfsClient create(AzureBlobFileSystem fs) throws AzureBlobFileSystemException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java
deleted file mode 100644
index 3107fa3..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java
+++ /dev/null
@@ -1,162 +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.fs.azurebfs.contracts.services;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Hashtable;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
-
-/**
- * File System http service to provide network calls for file system operations.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public interface AbfsHttpService extends InjectableService {
-  /**
-   * Gets filesystem properties on the Azure service.
-   * @param azureBlobFileSystem filesystem to get the properties.
-   * @return Hashtable<String, String> hash table containing all the filesystem properties.
-   */
-  Hashtable<String, String> getFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
-
-
-  /**
-   * Sets filesystem properties on the Azure service.
-   * @param azureBlobFileSystem filesystem to get the properties.
-   * @param properties file system properties to set.
-   */
-  void setFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem, Hashtable<String, String> properties) throws
-      AzureBlobFileSystemException;
-
-  /**
-   * Gets path properties on the Azure service.
-   * @param azureBlobFileSystem filesystem to get the properties of the path.
-   * @param path path to get properties.
-   * @return Hashtable<String, String> hash table containing all the path properties.
-   */
-  Hashtable<String, String> getPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
-
-  /**
-   * Sets path properties on the Azure service.
-   * @param azureBlobFileSystem filesystem to get the properties of the path.
-   * @param path path to set properties.
-   * @param properties hash table containing all the path properties.
-   */
-  void setPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path, Hashtable<String, String> properties) throws
-      AzureBlobFileSystemException;
-
-  /**
-   * Creates filesystem on the Azure service.
-   * @param azureBlobFileSystem filesystem to be created.
-   */
-  void createFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
-
-  /**
-   * Deletes filesystem on the Azure service.
-   * @param azureBlobFileSystem filesystem to be deleted.
-   */
-  void deleteFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
-
-  /**
-   * Creates a file on the Azure service.
-   * @param azureBlobFileSystem filesystem to create file or directory.
-   * @param path path of the file to be created.
-   * @param overwrite should overwrite.
-   * @return OutputStream stream to the file.
-   */
-  OutputStream createFile(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException;
-
-  /**
-   * Creates a directory on the Azure service.
-   * @param azureBlobFileSystem filesystem to create file or directory.
-   * @param path path of the directory to be created.
-   * @return OutputStream stream to the file.
-   */
-  Void createDirectory(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
-
-  /**
-   * Opens a file to read and returns the stream.
-   * @param azureBlobFileSystem filesystem to read a file from.
-   * @param path file path to read.
-   * @return InputStream a stream to the file to read.
-   */
-  InputStream openFileForRead(AzureBlobFileSystem azureBlobFileSystem, Path path, FileSystem.Statistics statistics) throws AzureBlobFileSystemException;
-
-  /**
-   * Opens a file to write and returns the stream.
-   * @param azureBlobFileSystem filesystem to write a file to.
-   * @param path file path to write.
-   * @param overwrite should overwrite.
-   * @return OutputStream a stream to the file to write.
-   */
-  OutputStream openFileForWrite(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException;
-
-  /**
-   * Renames a file or directory from source to destination.
-   * @param azureBlobFileSystem filesystem to rename a path.
-   * @param source source path.
-   * @param destination destination path.
-   */
-  void rename(AzureBlobFileSystem azureBlobFileSystem, Path source, Path destination) throws AzureBlobFileSystemException;
-
-  /**
-   * Deletes a file or directory.
-   * @param azureBlobFileSystem filesystem to delete the path.
-   * @param path file path to be deleted.
-   * @param recursive true if path is a directory and recursive deletion is desired.
-   */
-  void delete(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean recursive) throws AzureBlobFileSystemException;
-
-  /**
-   * Gets path's status under the provided path on the Azure service.
-   * @param azureBlobFileSystem filesystem to perform the get file status operation.
-   * @param path path delimiter.
-   * @return FileStatus FileStatus of the path in the file system.
-   */
-  FileStatus getFileStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
-
-  /**
-   * Lists all the paths under the provided path on the Azure service.
-   * @param azureBlobFileSystem filesystem to perform the list operation.
-   * @param path path delimiter.
-   * @return FileStatus[] list of all paths in the file system.
-   */
-  FileStatus[] listStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
-
-  /**
-   * Closes the client to filesystem to Azure service.
-   * @param azureBlobFileSystem filesystem to perform the list operation.
-   */
-  void closeFileSystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
-
-  /**
-   * Checks for the given path if it is marked as atomic rename directory or not.
-   * @param key
-   * @return True if the given path is listed under atomic rename property otherwise False.
-   */
-  boolean isAtomicRenameKey(String key);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java
deleted file mode 100644
index bd98bae..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java
+++ /dev/null
@@ -1,40 +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.fs.azurebfs.contracts.services;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException;
-
-/**
- * Dependency injected Azure Storage services provider interface.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public interface AbfsServiceProvider {
-  /**
-   * Returns an instance of resolved injectable service by class name.
-   * The injectable service must be configured first to be resolvable.
-   * @param clazz the injectable service which is expected to be returned.
-   * @param <T> The type of injectable service.
-   * @return T instance
-   * @throws ServiceResolutionException if the service is not resolvable.
-   */
-  <T extends InjectableService> T get(Class<T> clazz) throws ServiceResolutionException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java
deleted file mode 100644
index ee40c9d..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java
+++ /dev/null
@@ -1,143 +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.fs.azurebfs.contracts.services;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
-
-/**
- * Configuration service collects required Azure Hadoop configurations and provides it to the consumers.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public interface ConfigurationService extends InjectableService {
-  /**
-   * Checks if ABFS is running from Emulator;
-   * @return is emulator mode.
-   */
-  boolean isEmulator();
-
-  /**
-   * Retrieves storage secure mode from Hadoop configuration;
-   * @return storage secure mode;
-   */
-  boolean isSecureMode();
-
-  /**
-   * Retrieves storage account key for provided account name from Hadoop configuration.
-   * @param accountName the account name to retrieve the key.
-   * @return storage account key;
-   */
-  String getStorageAccountKey(String accountName) throws ConfigurationPropertyNotFoundException;
-
-  /**
-   * Returns Hadoop configuration.
-   * @return Hadoop configuration.
-   */
-  Configuration getConfiguration();
-
-  /**
-   * Retrieves configured write buffer size
-   * @return the size of the write buffer
-   */
-  int getWriteBufferSize();
-
-  /**
-   * Retrieves configured read buffer size
-   * @return the size of the read buffer
-   */
-  int getReadBufferSize();
-
-  /**
-   * Retrieves configured min backoff interval
-   * @return min backoff interval
-   */
-  int getMinBackoffIntervalMilliseconds();
-
-  /**
-   * Retrieves configured max backoff interval
-   * @return max backoff interval
-   */
-  int getMaxBackoffIntervalMilliseconds();
-
-  /**
-   * Retrieves configured backoff interval
-   * @return backoff interval
-   */
-  int getBackoffIntervalMilliseconds();
-
-  /**
-   * Retrieves configured num of retries
-   * @return num of retries
-   */
-  int getMaxIoRetries();
-
-  /**
-   * Retrieves configured azure block size
-   * @return azure block size
-   */
-  long getAzureBlockSize();
-
-  /**
-   * Retrieves configured azure block location host
-   * @return azure block location host
-   */
-  String getAzureBlockLocationHost();
-
-  /**
-   * Retrieves configured number of concurrent threads
-   * @return number of concurrent write threads
-   */
-  int getMaxConcurrentWriteThreads();
-
-  /**
-   * Retrieves configured number of concurrent threads
-   * @return number of concurrent read threads
-   */
-  int getMaxConcurrentReadThreads();
-
-  /**
-   * Retrieves configured boolean for tolerating out of band writes to files
-   * @return configured boolean for tolerating out of band writes to files
-   */
-  boolean getTolerateOobAppends();
-
-  /**
-   * Retrieves the comma-separated list of directories to receive special treatment so that folder
-   * rename is made atomic. The default value for this setting is just '/hbase'.
-   * Example directories list : <value>/hbase,/data</value>
-   * @see <a href="https://hadoop.apache.org/docs/stable/hadoop-azure/index.html#Configuring_Credentials">AtomicRenameProperty</a>
-   * @return atomic rename directories
-   */
-  String getAzureAtomicRenameDirs();
-
-  /**
-   * Retrieves configured boolean for creating remote file system during initialization
-   * @return configured boolean for creating remote file system during initialization
-   */
-  boolean getCreateRemoteFileSystemDuringInitialization();
-
-  /**
-   * Retrieves configured value of read ahead queue
-   * @return depth of read ahead
-   */
-  int getReadAheadQueueDepth();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java
deleted file mode 100644
index 8b3801f..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java
+++ /dev/null
@@ -1,30 +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.fs.azurebfs.contracts.services;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Marker interface for all the injectable services.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public interface InjectableService {
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java
deleted file mode 100644
index 267d11f..0000000
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java
+++ /dev/null
@@ -1,66 +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.fs.azurebfs.contracts.services;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.TraceScope;
-
-/**
- * Azure Blob File System tracing service.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public interface TracingService extends InjectableService {
-  /**
-   * Creates a {@link TraceScope} object with the provided description.
-   * @param description the trace description.
-   * @return created traceScope.
-   */
-  TraceScope traceBegin(String description);
-
-  /**
-   * Creates a {@link TraceScope} object with the provided description.
-   * @param description the trace description.
-   * @param parentSpanId the span id of the parent trace scope.
-   * @return create traceScope
-   */
-  TraceScope traceBegin(String description, SpanId parentSpanId);
-
-  /**
-   * Gets current thread latest generated traceScope id.
-   * @return current thread latest generated traceScope id.
-   */
-  SpanId getCurrentTraceScopeSpanId();
-
-  /**
-   * Appends the provided exception to the trace scope.
-   * @param traceScope the scope which exception needs to be attached to.
-   * @param azureBlobFileSystemException the exception to be attached to the scope.
-   */
-  void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException);
-
-  /**
-   * Ends the provided traceScope.
-   * @param traceScope the scope that needs to be ended.
-   */
-  void traceEnd(TraceScope traceScope);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
index c17a5c1..a78e7af 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -28,7 +28,6 @@ import java.util.Locale;
 
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
@@ -45,17 +44,17 @@ public class AbfsClient {
   private final String xMsVersion = "2018-03-28";
   private final ExponentialRetryPolicy retryPolicy;
   private final String filesystem;
-  private final ConfigurationService configurationService;
+  private final AbfsConfiguration abfsConfiguration;
   private final String userAgent;
 
   public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
-                    final ConfigurationService configurationService,
+                    final AbfsConfiguration abfsConfiguration,
                     final ExponentialRetryPolicy exponentialRetryPolicy) {
     this.baseUrl = baseUrl;
     this.sharedKeyCredentials = sharedKeyCredentials;
     String baseUrlString = baseUrl.toString();
     this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(AbfsHttpConstants.FORWARD_SLASH) + 1);
-    this.configurationService = configurationService;
+    this.abfsConfiguration = abfsConfiguration;
     this.retryPolicy = exponentialRetryPolicy;
     this.userAgent = initializeUserAgent();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java
new file mode 100644
index 0000000..8def1bb
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java
@@ -0,0 +1,297 @@
+/**
+ * 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.azurebfs.services;
+
+import java.lang.reflect.Field;
+import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
+
+/**
+ * Configuration for Azure Blob FileSystem.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class AbfsConfiguration{
+  private final Configuration configuration;
+  private final boolean isSecure;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE,
+      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
+      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
+      DefaultValue = FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE)
+  private int writeBufferSize;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_READ_BUFFER_SIZE,
+      MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
+      MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
+      DefaultValue = FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE)
+  private int readBufferSize;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL,
+      DefaultValue = FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL)
+  private int minBackoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL,
+      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL)
+  private int maxBackoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BACKOFF_INTERVAL,
+      DefaultValue = FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL)
+  private int backoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_IO_RETRIES,
+      MinValue = 0,
+      DefaultValue = FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS)
+  private int maxIoRetries;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_SIZE_PROPERTY_NAME,
+      MinValue = 0,
+      MaxValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE,
+      DefaultValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE)
+  private long azureBlockSize;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
+      DefaultValue = FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT)
+  private String azureBlockLocationHost;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_OUT,
+      MinValue = 1,
+      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_WRITE_THREADS)
+  private int maxConcurrentWriteThreads;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_IN,
+      MinValue = 1,
+      DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_READ_THREADS)
+  private int maxConcurrentReadThreads;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_TOLERATE_CONCURRENT_APPEND,
+      DefaultValue = FileSystemConfigurations.DEFAULT_READ_TOLERATE_CONCURRENT_APPEND)
+  private boolean tolerateOobAppends;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ATOMIC_RENAME_KEY,
+          DefaultValue = FileSystemConfigurations.DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES)
+  private String azureAtomicDirs;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
+      DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION)
+  private boolean createRemoteFileSystemDuringInitialization;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH,
+      DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH)
+  private int readAheadQueueDepth;
+
+  private Map<String, String> storageAccountKeys;
+
+  public AbfsConfiguration(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException {
+    this.configuration = configuration;
+    this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false);
+
+    validateStorageAccountKeys();
+    Field[] fields = this.getClass().getDeclaredFields();
+    for (Field field : fields) {
+      field.setAccessible(true);
+      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateInt(field));
+      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateLong(field));
+      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateString(field));
+      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateBase64String(field));
+      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateBoolean(field));
+      }
+    }
+  }
+
+  public boolean isEmulator() {
+    return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
+  }
+
+  public boolean isSecureMode() {
+    return this.isSecure;
+  }
+
+  public String getStorageAccountKey(final String accountName) throws ConfigurationPropertyNotFoundException {
+    String accountKey = this.storageAccountKeys.get(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName);
+    if (accountKey == null) {
+      throw new ConfigurationPropertyNotFoundException(accountName);
+    }
+
+    return accountKey;
+  }
+
+  public Configuration getConfiguration() {
+    return this.configuration;
+  }
+
+  public int getWriteBufferSize() {
+    return this.writeBufferSize;
+  }
+
+  public int getReadBufferSize() {
+    return this.readBufferSize;
+  }
+
+  public int getMinBackoffIntervalMilliseconds() {
+    return this.minBackoffInterval;
+  }
+
+  public int getMaxBackoffIntervalMilliseconds() {
+    return this.maxBackoffInterval;
+  }
+
+  public int getBackoffIntervalMilliseconds() {
+    return this.backoffInterval;
+  }
+
+  public int getMaxIoRetries() {
+    return this.maxIoRetries;
+  }
+
+  public long getAzureBlockSize() {
+    return this.azureBlockSize;
+  }
+
+  public String getAzureBlockLocationHost() {
+    return this.azureBlockLocationHost;
+  }
+
+  public int getMaxConcurrentWriteThreads() {
+    return this.maxConcurrentWriteThreads;
+  }
+
+  public int getMaxConcurrentReadThreads() {
+    return this.maxConcurrentReadThreads;
+  }
+
+  public boolean getTolerateOobAppends() {
+    return this.tolerateOobAppends;
+  }
+
+  public String getAzureAtomicRenameDirs() {
+    return this.azureAtomicDirs;
+  }
+
+  public boolean getCreateRemoteFileSystemDuringInitialization() {
+    return this.createRemoteFileSystemDuringInitialization;
+  }
+
+  public int getReadAheadQueueDepth() {
+    return this.readAheadQueueDepth;
+  }
+
+  void validateStorageAccountKeys() throws InvalidConfigurationValueException {
+    Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator(
+        ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true);
+    this.storageAccountKeys = this.configuration.getValByRegex(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX);
+
+    for (Map.Entry<String, String> account : this.storageAccountKeys.entrySet()) {
+      validator.validate(account.getValue());
+    }
+  }
+
+  int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new IntegerConfigurationBasicValidator(
+        validator.MinValue(),
+        validator.MaxValue(),
+        validator.DefaultValue(),
+        validator.ConfigurationKey(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new LongConfigurationBasicValidator(
+        validator.MinValue(),
+        validator.MaxValue(),
+        validator.DefaultValue(),
+        validator.ConfigurationKey(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new StringConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class));
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new Base64StringConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class);
+    String value = this.configuration.get(validator.ConfigurationKey());
+
+    // validate
+    return new BooleanConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  @VisibleForTesting
+  void setReadBufferSize(int bufferSize) {
+    this.readBufferSize = bufferSize;
+  }
+
+  @VisibleForTesting
+  void setWriteBufferSize(int bufferSize) {
+    this.writeBufferSize = bufferSize;
+  }
+}
\ 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


[02/50] [abbrv] hadoop git commit: YARN-8601. Print ExecutionType in Container report CLI. Contributed by Bilwa S T.

Posted by tm...@apache.org.
YARN-8601. Print ExecutionType in Container report CLI. Contributed by Bilwa S T.


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

Branch: refs/heads/HADOOP-15407
Commit: ff06bd1be83a2a6d2ee39cb002e91499720a7243
Parents: 36c0d74
Author: bibinchundatt <bi...@apache.org>
Authored: Wed Aug 8 22:42:52 2018 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Wed Aug 8 22:42:52 2018 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java     | 2 ++
 .../test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java   | 1 +
 2 files changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff06bd1b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index 14710a4..807938c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -881,6 +881,8 @@ public class ApplicationCLI extends YarnCLI {
       containerReportStr.println(containerReport.getFinishTime());
       containerReportStr.print("\tState : ");
       containerReportStr.println(containerReport.getContainerState());
+      containerReportStr.print("\tExecution-Type : ");
+      containerReportStr.println(containerReport.getExecutionType());
       containerReportStr.print("\tLOG-URL : ");
       containerReportStr.println(containerReport.getLogUrl());
       containerReportStr.print("\tHost : ");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff06bd1b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index 6b823b2..526adfd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -292,6 +292,7 @@ public class TestYarnCLI {
     pw.println("\tStart-Time : 1234");
     pw.println("\tFinish-Time : 5678");
     pw.println("\tState : COMPLETE");
+    pw.println("\tExecution-Type : GUARANTEED");
     pw.println("\tLOG-URL : logURL");
     pw.println("\tHost : host:1234");
     pw.println("\tNodeHttpAddress : http://host:2345");


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


[14/50] [abbrv] hadoop git commit: YARN-8633. Update DataTables version in yarn-common in line with JQuery 3 upgrade. Contributed by Akhil PB.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js
deleted file mode 100644
index 61acb9b..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/js/jquery.dataTables.min.js
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * File:        jquery.dataTables.min.js
- * Version:     1.9.4
- * Author:      Allan Jardine (www.sprymedia.co.uk)
- * Info:        www.datatables.net
- *
- * Copyright 2008-2012 Allan Jardine, all rights reserved.
- *
- * This source file is free software, under either the GPL v2 license or a
- * BSD style license, available at:
- *   http://datatables.net/license_gpl2
- *   http://datatables.net/license_bsd
- *
- * This source file is distributed in the hope that it will be useful, but
- * WITHOUT ANY WARRANTY; without even the implied warranty of MERCHANTABILITY
- * or FITNESS FOR A PARTICULAR PURPOSE. See the license files for details.
- */
-(function(la,s,p){(function(i){if(typeof define==="function"&&define.amd)define(["jquery"],i);else jQuery&&!jQuery.fn.dataTable&&i(jQuery)})(function(i){var l=function(h){function n(a,b){var c=l.defaults.columns,d=a.aoColumns.length;b=i.extend({},l.models.oColumn,c,{sSortingClass:a.oClasses.sSortable,sSortingClassJUI:a.oClasses.sSortJUI,nTh:b?b:s.createElement("th"),sTitle:c.sTitle?c.sTitle:b?b.innerHTML:"",aDataSort:c.aDataSort?c.aDataSort:[d],mData:c.mData?c.oDefaults:d});a.aoColumns.push(b);if(a.aoPreSearchCols[d]===
-p||a.aoPreSearchCols[d]===null)a.aoPreSearchCols[d]=i.extend({},l.models.oSearch);else{b=a.aoPreSearchCols[d];if(b.bRegex===p)b.bRegex=true;if(b.bSmart===p)b.bSmart=true;if(b.bCaseInsensitive===p)b.bCaseInsensitive=true}q(a,d,null)}function q(a,b,c){var d=a.aoColumns[b];if(c!==p&&c!==null){if(c.mDataProp&&!c.mData)c.mData=c.mDataProp;if(c.sType!==p){d.sType=c.sType;d._bAutoType=false}i.extend(d,c);r(d,c,"sWidth","sWidthOrig");if(c.iDataSort!==p)d.aDataSort=[c.iDataSort];r(d,c,"aDataSort")}var e=d.mRender?
-ca(d.mRender):null,f=ca(d.mData);d.fnGetData=function(g,j){var k=f(g,j);if(d.mRender&&j&&j!=="")return e(k,j,g);return k};d.fnSetData=Ja(d.mData);if(!a.oFeatures.bSort)d.bSortable=false;if(!d.bSortable||i.inArray("asc",d.asSorting)==-1&&i.inArray("desc",d.asSorting)==-1){d.sSortingClass=a.oClasses.sSortableNone;d.sSortingClassJUI=""}else if(i.inArray("asc",d.asSorting)==-1&&i.inArray("desc",d.asSorting)==-1){d.sSortingClass=a.oClasses.sSortable;d.sSortingClassJUI=a.oClasses.sSortJUI}else if(i.inArray("asc",
-d.asSorting)!=-1&&i.inArray("desc",d.asSorting)==-1){d.sSortingClass=a.oClasses.sSortableAsc;d.sSortingClassJUI=a.oClasses.sSortJUIAscAllowed}else if(i.inArray("asc",d.asSorting)==-1&&i.inArray("desc",d.asSorting)!=-1){d.sSortingClass=a.oClasses.sSortableDesc;d.sSortingClassJUI=a.oClasses.sSortJUIDescAllowed}}function o(a){if(a.oFeatures.bAutoWidth===false)return false;ta(a);for(var b=0,c=a.aoColumns.length;b<c;b++)a.aoColumns[b].nTh.style.width=a.aoColumns[b].sWidth}function v(a,b){a=A(a,"bVisible");
-return typeof a[b]==="number"?a[b]:null}function w(a,b){a=A(a,"bVisible");b=i.inArray(b,a);return b!==-1?b:null}function D(a){return A(a,"bVisible").length}function A(a,b){var c=[];i.map(a.aoColumns,function(d,e){d[b]&&c.push(e)});return c}function G(a){for(var b=l.ext.aTypes,c=b.length,d=0;d<c;d++){var e=b[d](a);if(e!==null)return e}return"string"}function E(a,b){b=b.split(",");for(var c=[],d=0,e=a.aoColumns.length;d<e;d++)for(var f=0;f<e;f++)if(a.aoColumns[d].sName==b[f]){c.push(f);break}return c}
-function Y(a){for(var b="",c=0,d=a.aoColumns.length;c<d;c++)b+=a.aoColumns[c].sName+",";if(b.length==d)return"";return b.slice(0,-1)}function ma(a,b,c,d){var e,f,g,j,k;if(b)for(e=b.length-1;e>=0;e--){var m=b[e].aTargets;i.isArray(m)||O(a,1,"aTargets must be an array of targets, not a "+typeof m);f=0;for(g=m.length;f<g;f++)if(typeof m[f]==="number"&&m[f]>=0){for(;a.aoColumns.length<=m[f];)n(a);d(m[f],b[e])}else if(typeof m[f]==="number"&&m[f]<0)d(a.aoColumns.length+m[f],b[e]);else if(typeof m[f]===
-"string"){j=0;for(k=a.aoColumns.length;j<k;j++)if(m[f]=="_all"||i(a.aoColumns[j].nTh).hasClass(m[f]))d(j,b[e])}}if(c){e=0;for(a=c.length;e<a;e++)d(e,c[e])}}function R(a,b){var c;c=i.isArray(b)?b.slice():i.extend(true,{},b);b=a.aoData.length;var d=i.extend(true,{},l.models.oRow);d._aData=c;a.aoData.push(d);var e;d=0;for(var f=a.aoColumns.length;d<f;d++){c=a.aoColumns[d];typeof c.fnRender==="function"&&c.bUseRendered&&c.mData!==null?S(a,b,d,da(a,b,d)):S(a,b,d,F(a,b,d));if(c._bAutoType&&c.sType!="string"){e=
-F(a,b,d,"type");if(e!==null&&e!==""){e=G(e);if(c.sType===null)c.sType=e;else if(c.sType!=e&&c.sType!="html")c.sType="string"}}}a.aiDisplayMaster.push(b);a.oFeatures.bDeferRender||ua(a,b);return b}function ea(a){var b,c,d,e,f,g,j;if(a.bDeferLoading||a.sAjaxSource===null)for(b=a.nTBody.firstChild;b;){if(b.nodeName.toUpperCase()=="TR"){c=a.aoData.length;b._DT_RowIndex=c;a.aoData.push(i.extend(true,{},l.models.oRow,{nTr:b}));a.aiDisplayMaster.push(c);f=b.firstChild;for(d=0;f;){g=f.nodeName.toUpperCase();
-if(g=="TD"||g=="TH"){S(a,c,d,i.trim(f.innerHTML));d++}f=f.nextSibling}}b=b.nextSibling}e=fa(a);d=[];b=0;for(c=e.length;b<c;b++)for(f=e[b].firstChild;f;){g=f.nodeName.toUpperCase();if(g=="TD"||g=="TH")d.push(f);f=f.nextSibling}c=0;for(e=a.aoColumns.length;c<e;c++){j=a.aoColumns[c];if(j.sTitle===null)j.sTitle=j.nTh.innerHTML;var k=j._bAutoType,m=typeof j.fnRender==="function",u=j.sClass!==null,x=j.bVisible,y,B;if(k||m||u||!x){g=0;for(b=a.aoData.length;g<b;g++){f=a.aoData[g];y=d[g*e+c];if(k&&j.sType!=
-"string"){B=F(a,g,c,"type");if(B!==""){B=G(B);if(j.sType===null)j.sType=B;else if(j.sType!=B&&j.sType!="html")j.sType="string"}}if(j.mRender)y.innerHTML=F(a,g,c,"display");else if(j.mData!==c)y.innerHTML=F(a,g,c,"display");if(m){B=da(a,g,c);y.innerHTML=B;j.bUseRendered&&S(a,g,c,B)}if(u)y.className+=" "+j.sClass;if(x)f._anHidden[c]=null;else{f._anHidden[c]=y;y.parentNode.removeChild(y)}j.fnCreatedCell&&j.fnCreatedCell.call(a.oInstance,y,F(a,g,c,"display"),f._aData,g,c)}}}if(a.aoRowCreatedCallback.length!==
-0){b=0;for(c=a.aoData.length;b<c;b++){f=a.aoData[b];K(a,"aoRowCreatedCallback",null,[f.nTr,f._aData,b])}}}function V(a,b){return b._DT_RowIndex!==p?b._DT_RowIndex:null}function va(a,b,c){b=W(a,b);var d=0;for(a=a.aoColumns.length;d<a;d++)if(b[d]===c)return d;return-1}function na(a,b,c,d){for(var e=[],f=0,g=d.length;f<g;f++)e.push(F(a,b,d[f],c));return e}function F(a,b,c,d){var e=a.aoColumns[c];if((c=e.fnGetData(a.aoData[b]._aData,d))===p){if(a.iDrawError!=a.iDraw&&e.sDefaultContent===null){O(a,0,"Requested unknown parameter "+
-(typeof e.mData=="function"?"{mData function}":"'"+e.mData+"'")+" from the data source for row "+b);a.iDrawError=a.iDraw}return e.sDefaultContent}if(c===null&&e.sDefaultContent!==null)c=e.sDefaultContent;else if(typeof c==="function")return c();if(d=="display"&&c===null)return"";return c}function S(a,b,c,d){a.aoColumns[c].fnSetData(a.aoData[b]._aData,d)}function ca(a){if(a===null)return function(){return null};else if(typeof a==="function")return function(c,d,e){return a(c,d,e)};else if(typeof a===
-"string"&&(a.indexOf(".")!==-1||a.indexOf("[")!==-1)){var b=function(c,d,e){var f=e.split("."),g;if(e!==""){var j=0;for(g=f.length;j<g;j++){if(e=f[j].match(ga)){f[j]=f[j].replace(ga,"");if(f[j]!=="")c=c[f[j]];g=[];f.splice(0,j+1);f=f.join(".");j=0;for(var k=c.length;j<k;j++)g.push(b(c[j],d,f));c=e[0].substring(1,e[0].length-1);c=c===""?g:g.join(c);break}if(c===null||c[f[j]]===p)return p;c=c[f[j]]}}return c};return function(c,d){return b(c,d,a)}}else return function(c){return c[a]}}function Ja(a){if(a===
-null)return function(){};else if(typeof a==="function")return function(c,d){a(c,"set",d)};else if(typeof a==="string"&&(a.indexOf(".")!==-1||a.indexOf("[")!==-1)){var b=function(c,d,e){e=e.split(".");var f,g,j=0;for(g=e.length-1;j<g;j++){if(f=e[j].match(ga)){e[j]=e[j].replace(ga,"");c[e[j]]=[];f=e.slice();f.splice(0,j+1);g=f.join(".");for(var k=0,m=d.length;k<m;k++){f={};b(f,d[k],g);c[e[j]].push(f)}return}if(c[e[j]]===null||c[e[j]]===p)c[e[j]]={};c=c[e[j]]}c[e[e.length-1].replace(ga,"")]=d};return function(c,
-d){return b(c,d,a)}}else return function(c,d){c[a]=d}}function oa(a){for(var b=[],c=a.aoData.length,d=0;d<c;d++)b.push(a.aoData[d]._aData);return b}function wa(a){a.aoData.splice(0,a.aoData.length);a.aiDisplayMaster.splice(0,a.aiDisplayMaster.length);a.aiDisplay.splice(0,a.aiDisplay.length);I(a)}function xa(a,b){for(var c=-1,d=0,e=a.length;d<e;d++)if(a[d]==b)c=d;else a[d]>b&&a[d]--;c!=-1&&a.splice(c,1)}function da(a,b,c){var d=a.aoColumns[c];return d.fnRender({iDataRow:b,iDataColumn:c,oSettings:a,
-aData:a.aoData[b]._aData,mDataProp:d.mData},F(a,b,c,"display"))}function ua(a,b){var c=a.aoData[b],d;if(c.nTr===null){c.nTr=s.createElement("tr");c.nTr._DT_RowIndex=b;if(c._aData.DT_RowId)c.nTr.id=c._aData.DT_RowId;if(c._aData.DT_RowClass)c.nTr.className=c._aData.DT_RowClass;for(var e=0,f=a.aoColumns.length;e<f;e++){var g=a.aoColumns[e];d=s.createElement(g.sCellType);d.innerHTML=typeof g.fnRender==="function"&&(!g.bUseRendered||g.mData===null)?da(a,b,e):F(a,b,e,"display");if(g.sClass!==null)d.className=
-g.sClass;if(g.bVisible){c.nTr.appendChild(d);c._anHidden[e]=null}else c._anHidden[e]=d;g.fnCreatedCell&&g.fnCreatedCell.call(a.oInstance,d,F(a,b,e,"display"),c._aData,b,e)}K(a,"aoRowCreatedCallback",null,[c.nTr,c._aData,b])}}function Ka(a){var b,c,d;if(i("th, td",a.nTHead).length!==0){b=0;for(d=a.aoColumns.length;b<d;b++){c=a.aoColumns[b].nTh;c.setAttribute("role","columnheader");if(a.aoColumns[b].bSortable){c.setAttribute("tabindex",a.iTabIndex);c.setAttribute("aria-controls",a.sTableId)}a.aoColumns[b].sClass!==
-null&&i(c).addClass(a.aoColumns[b].sClass);if(a.aoColumns[b].sTitle!=c.innerHTML)c.innerHTML=a.aoColumns[b].sTitle}}else{var e=s.createElement("tr");b=0;for(d=a.aoColumns.length;b<d;b++){c=a.aoColumns[b].nTh;c.innerHTML=a.aoColumns[b].sTitle;c.setAttribute("tabindex","0");a.aoColumns[b].sClass!==null&&i(c).addClass(a.aoColumns[b].sClass);e.appendChild(c)}i(a.nTHead).html("")[0].appendChild(e);ha(a.aoHeader,a.nTHead)}i(a.nTHead).children("tr").attr("role","row");if(a.bJUI){b=0;for(d=a.aoColumns.length;b<
-d;b++){c=a.aoColumns[b].nTh;e=s.createElement("div");e.className=a.oClasses.sSortJUIWrapper;i(c).contents().appendTo(e);var f=s.createElement("span");f.className=a.oClasses.sSortIcon;e.appendChild(f);c.appendChild(e)}}if(a.oFeatures.bSort)for(b=0;b<a.aoColumns.length;b++)a.aoColumns[b].bSortable!==false?ya(a,a.aoColumns[b].nTh,b):i(a.aoColumns[b].nTh).addClass(a.oClasses.sSortableNone);a.oClasses.sFooterTH!==""&&i(a.nTFoot).children("tr").children("th").addClass(a.oClasses.sFooterTH);if(a.nTFoot!==
-null){c=Z(a,null,a.aoFooter);b=0;for(d=a.aoColumns.length;b<d;b++)if(c[b]){a.aoColumns[b].nTf=c[b];a.aoColumns[b].sClass&&i(c[b]).addClass(a.aoColumns[b].sClass)}}}function ia(a,b,c){var d,e,f,g=[],j=[],k=a.aoColumns.length,m;if(c===p)c=false;d=0;for(e=b.length;d<e;d++){g[d]=b[d].slice();g[d].nTr=b[d].nTr;for(f=k-1;f>=0;f--)!a.aoColumns[f].bVisible&&!c&&g[d].splice(f,1);j.push([])}d=0;for(e=g.length;d<e;d++){if(a=g[d].nTr)for(;f=a.firstChild;)a.removeChild(f);f=0;for(b=g[d].length;f<b;f++){m=k=1;
-if(j[d][f]===p){a.appendChild(g[d][f].cell);for(j[d][f]=1;g[d+k]!==p&&g[d][f].cell==g[d+k][f].cell;){j[d+k][f]=1;k++}for(;g[d][f+m]!==p&&g[d][f].cell==g[d][f+m].cell;){for(c=0;c<k;c++)j[d+c][f+m]=1;m++}g[d][f].cell.rowSpan=k;g[d][f].cell.colSpan=m}}}}function H(a){var b=K(a,"aoPreDrawCallback","preDraw",[a]);if(i.inArray(false,b)!==-1)P(a,false);else{var c,d;b=[];var e=0,f=a.asStripeClasses.length;c=a.aoOpenRows.length;a.bDrawing=true;if(a.iInitDisplayStart!==p&&a.iInitDisplayStart!=-1){a._iDisplayStart=
-a.oFeatures.bServerSide?a.iInitDisplayStart:a.iInitDisplayStart>=a.fnRecordsDisplay()?0:a.iInitDisplayStart;a.iInitDisplayStart=-1;I(a)}if(a.bDeferLoading){a.bDeferLoading=false;a.iDraw++}else if(a.oFeatures.bServerSide){if(!a.bDestroying&&!La(a))return}else a.iDraw++;if(a.aiDisplay.length!==0){var g=a._iDisplayStart;d=a._iDisplayEnd;if(a.oFeatures.bServerSide){g=0;d=a.aoData.length}for(g=g;g<d;g++){var j=a.aoData[a.aiDisplay[g]];j.nTr===null&&ua(a,a.aiDisplay[g]);var k=j.nTr;if(f!==0){var m=a.asStripeClasses[e%
-f];if(j._sRowStripe!=m){i(k).removeClass(j._sRowStripe).addClass(m);j._sRowStripe=m}}K(a,"aoRowCallback",null,[k,a.aoData[a.aiDisplay[g]]._aData,e,g]);b.push(k);e++;if(c!==0)for(j=0;j<c;j++)if(k==a.aoOpenRows[j].nParent){b.push(a.aoOpenRows[j].nTr);break}}}else{b[0]=s.createElement("tr");if(a.asStripeClasses[0])b[0].className=a.asStripeClasses[0];c=a.oLanguage;f=c.sZeroRecords;if(a.iDraw==1&&a.sAjaxSource!==null&&!a.oFeatures.bServerSide)f=c.sLoadingRecords;else if(c.sEmptyTable&&a.fnRecordsTotal()===
-0)f=c.sEmptyTable;c=s.createElement("td");c.setAttribute("valign","top");c.colSpan=D(a);c.className=a.oClasses.sRowEmpty;c.innerHTML=za(a,f);b[e].appendChild(c)}K(a,"aoHeaderCallback","header",[i(a.nTHead).children("tr")[0],oa(a),a._iDisplayStart,a.fnDisplayEnd(),a.aiDisplay]);K(a,"aoFooterCallback","footer",[i(a.nTFoot).children("tr")[0],oa(a),a._iDisplayStart,a.fnDisplayEnd(),a.aiDisplay]);e=s.createDocumentFragment();c=s.createDocumentFragment();if(a.nTBody){f=a.nTBody.parentNode;c.appendChild(a.nTBody);
-if(!a.oScroll.bInfinite||!a._bInitComplete||a.bSorted||a.bFiltered)for(;c=a.nTBody.firstChild;)a.nTBody.removeChild(c);c=0;for(d=b.length;c<d;c++)e.appendChild(b[c]);a.nTBody.appendChild(e);f!==null&&f.appendChild(a.nTBody)}K(a,"aoDrawCallback","draw",[a]);a.bSorted=false;a.bFiltered=false;a.bDrawing=false;if(a.oFeatures.bServerSide){P(a,false);a._bInitComplete||pa(a)}}}function qa(a){if(a.oFeatures.bSort)$(a,a.oPreviousSearch);else if(a.oFeatures.bFilter)X(a,a.oPreviousSearch);else{I(a);H(a)}}function Ma(a){var b=
-i("<div></div>")[0];a.nTable.parentNode.insertBefore(b,a.nTable);a.nTableWrapper=i('<div id="'+a.sTableId+'_wrapper" class="'+a.oClasses.sWrapper+'" role="grid"></div>')[0];a.nTableReinsertBefore=a.nTable.nextSibling;for(var c=a.nTableWrapper,d=a.sDom.split(""),e,f,g,j,k,m,u,x=0;x<d.length;x++){f=0;g=d[x];if(g=="<"){j=i("<div></div>")[0];k=d[x+1];if(k=="'"||k=='"'){m="";for(u=2;d[x+u]!=k;){m+=d[x+u];u++}if(m=="H")m=a.oClasses.sJUIHeader;else if(m=="F")m=a.oClasses.sJUIFooter;if(m.indexOf(".")!=-1){k=
-m.split(".");j.id=k[0].substr(1,k[0].length-1);j.className=k[1]}else if(m.charAt(0)=="#")j.id=m.substr(1,m.length-1);else j.className=m;x+=u}c.appendChild(j);c=j}else if(g==">")c=c.parentNode;else if(g=="l"&&a.oFeatures.bPaginate&&a.oFeatures.bLengthChange){e=Na(a);f=1}else if(g=="f"&&a.oFeatures.bFilter){e=Oa(a);f=1}else if(g=="r"&&a.oFeatures.bProcessing){e=Pa(a);f=1}else if(g=="t"){e=Qa(a);f=1}else if(g=="i"&&a.oFeatures.bInfo){e=Ra(a);f=1}else if(g=="p"&&a.oFeatures.bPaginate){e=Sa(a);f=1}else if(l.ext.aoFeatures.length!==
-0){j=l.ext.aoFeatures;u=0;for(k=j.length;u<k;u++)if(g==j[u].cFeature){if(e=j[u].fnInit(a))f=1;break}}if(f==1&&e!==null){if(typeof a.aanFeatures[g]!=="object")a.aanFeatures[g]=[];a.aanFeatures[g].push(e);c.appendChild(e)}}b.parentNode.replaceChild(a.nTableWrapper,b)}function ha(a,b){b=i(b).children("tr");var c,d,e,f,g,j,k,m,u,x,y=function(B,T,M){for(B=B[T];B[M];)M++;return M};a.splice(0,a.length);e=0;for(j=b.length;e<j;e++)a.push([]);e=0;for(j=b.length;e<j;e++){c=b[e];for(d=c.firstChild;d;){if(d.nodeName.toUpperCase()==
-"TD"||d.nodeName.toUpperCase()=="TH"){m=d.getAttribute("colspan")*1;u=d.getAttribute("rowspan")*1;m=!m||m===0||m===1?1:m;u=!u||u===0||u===1?1:u;k=y(a,e,0);x=m===1?true:false;for(g=0;g<m;g++)for(f=0;f<u;f++){a[e+f][k+g]={cell:d,unique:x};a[e+f].nTr=c}}d=d.nextSibling}}}function Z(a,b,c){var d=[];if(!c){c=a.aoHeader;if(b){c=[];ha(c,b)}}b=0;for(var e=c.length;b<e;b++)for(var f=0,g=c[b].length;f<g;f++)if(c[b][f].unique&&(!d[f]||!a.bSortCellsTop))d[f]=c[b][f].cell;return d}function La(a){if(a.bAjaxDataGet){a.iDraw++;
-P(a,true);var b=Ta(a);Aa(a,b);a.fnServerData.call(a.oInstance,a.sAjaxSource,b,function(c){Ua(a,c)},a);return false}else return true}function Ta(a){var b=a.aoColumns.length,c=[],d,e,f,g;c.push({name:"sEcho",value:a.iDraw});c.push({name:"iColumns",value:b});c.push({name:"sColumns",value:Y(a)});c.push({name:"iDisplayStart",value:a._iDisplayStart});c.push({name:"iDisplayLength",value:a.oFeatures.bPaginate!==false?a._iDisplayLength:-1});for(f=0;f<b;f++){d=a.aoColumns[f].mData;c.push({name:"mDataProp_"+
-f,value:typeof d==="function"?"function":d})}if(a.oFeatures.bFilter!==false){c.push({name:"sSearch",value:a.oPreviousSearch.sSearch});c.push({name:"bRegex",value:a.oPreviousSearch.bRegex});for(f=0;f<b;f++){c.push({name:"sSearch_"+f,value:a.aoPreSearchCols[f].sSearch});c.push({name:"bRegex_"+f,value:a.aoPreSearchCols[f].bRegex});c.push({name:"bSearchable_"+f,value:a.aoColumns[f].bSearchable})}}if(a.oFeatures.bSort!==false){var j=0;d=a.aaSortingFixed!==null?a.aaSortingFixed.concat(a.aaSorting):a.aaSorting.slice();
-for(f=0;f<d.length;f++){e=a.aoColumns[d[f][0]].aDataSort;for(g=0;g<e.length;g++){c.push({name:"iSortCol_"+j,value:e[g]});c.push({name:"sSortDir_"+j,value:d[f][1]});j++}}c.push({name:"iSortingCols",value:j});for(f=0;f<b;f++)c.push({name:"bSortable_"+f,value:a.aoColumns[f].bSortable})}return c}function Aa(a,b){K(a,"aoServerParams","serverParams",[b])}function Ua(a,b){if(b.sEcho!==p)if(b.sEcho*1<a.iDraw)return;else a.iDraw=b.sEcho*1;if(!a.oScroll.bInfinite||a.oScroll.bInfinite&&(a.bSorted||a.bFiltered))wa(a);
-a._iRecordsTotal=parseInt(b.iTotalRecords,10);a._iRecordsDisplay=parseInt(b.iTotalDisplayRecords,10);var c=Y(a);c=b.sColumns!==p&&c!==""&&b.sColumns!=c;var d;if(c)d=E(a,b.sColumns);b=ca(a.sAjaxDataProp)(b);for(var e=0,f=b.length;e<f;e++)if(c){for(var g=[],j=0,k=a.aoColumns.length;j<k;j++)g.push(b[e][d[j]]);R(a,g)}else R(a,b[e]);a.aiDisplay=a.aiDisplayMaster.slice();a.bAjaxDataGet=false;H(a);a.bAjaxDataGet=true;P(a,false)}function Oa(a){var b=a.oPreviousSearch,c=a.oLanguage.sSearch;c=c.indexOf("_INPUT_")!==
--1?c.replace("_INPUT_",'<input type="text" />'):c===""?'<input type="text" />':c+' <input type="text" />';var d=s.createElement("div");d.className=a.oClasses.sFilter;d.innerHTML="<label>"+c+"</label>";if(!a.aanFeatures.f)d.id=a.sTableId+"_filter";c=i('input[type="text"]',d);d._DT_Input=c[0];c.val(b.sSearch.replace('"',"&quot;"));c.bind("keyup.DT",function(){for(var e=a.aanFeatures.f,f=this.value===""?"":this.value,g=0,j=e.length;g<j;g++)e[g]!=i(this).parents("div.dataTables_filter")[0]&&i(e[g]._DT_Input).val(f);
-f!=b.sSearch&&X(a,{sSearch:f,bRegex:b.bRegex,bSmart:b.bSmart,bCaseInsensitive:b.bCaseInsensitive})});c.attr("aria-controls",a.sTableId).bind("keypress.DT",function(e){if(e.keyCode==13)return false});return d}function X(a,b,c){var d=a.oPreviousSearch,e=a.aoPreSearchCols,f=function(g){d.sSearch=g.sSearch;d.bRegex=g.bRegex;d.bSmart=g.bSmart;d.bCaseInsensitive=g.bCaseInsensitive};if(a.oFeatures.bServerSide)f(b);else{Va(a,b.sSearch,c,b.bRegex,b.bSmart,b.bCaseInsensitive);f(b);for(b=0;b<a.aoPreSearchCols.length;b++)Wa(a,
-e[b].sSearch,b,e[b].bRegex,e[b].bSmart,e[b].bCaseInsensitive);Xa(a)}a.bFiltered=true;i(a.oInstance).trigger("filter",a);a._iDisplayStart=0;I(a);H(a);Ba(a,0)}function Xa(a){for(var b=l.ext.afnFiltering,c=A(a,"bSearchable"),d=0,e=b.length;d<e;d++)for(var f=0,g=0,j=a.aiDisplay.length;g<j;g++){var k=a.aiDisplay[g-f];if(!b[d](a,na(a,k,"filter",c),k)){a.aiDisplay.splice(g-f,1);f++}}}function Wa(a,b,c,d,e,f){if(b!==""){var g=0;b=Ca(b,d,e,f);for(d=a.aiDisplay.length-1;d>=0;d--){e=Ya(F(a,a.aiDisplay[d],c,
-"filter"),a.aoColumns[c].sType);if(!b.test(e)){a.aiDisplay.splice(d,1);g++}}}}function Va(a,b,c,d,e,f){d=Ca(b,d,e,f);e=a.oPreviousSearch;c||(c=0);if(l.ext.afnFiltering.length!==0)c=1;if(b.length<=0){a.aiDisplay.splice(0,a.aiDisplay.length);a.aiDisplay=a.aiDisplayMaster.slice()}else if(a.aiDisplay.length==a.aiDisplayMaster.length||e.sSearch.length>b.length||c==1||b.indexOf(e.sSearch)!==0){a.aiDisplay.splice(0,a.aiDisplay.length);Ba(a,1);for(b=0;b<a.aiDisplayMaster.length;b++)d.test(a.asDataSearch[b])&&
-a.aiDisplay.push(a.aiDisplayMaster[b])}else for(b=c=0;b<a.asDataSearch.length;b++)if(!d.test(a.asDataSearch[b])){a.aiDisplay.splice(b-c,1);c++}}function Ba(a,b){if(!a.oFeatures.bServerSide){a.asDataSearch=[];var c=A(a,"bSearchable");b=b===1?a.aiDisplayMaster:a.aiDisplay;for(var d=0,e=b.length;d<e;d++)a.asDataSearch[d]=Da(a,na(a,b[d],"filter",c))}}function Da(a,b){a=b.join("  ");if(a.indexOf("&")!==-1)a=i("<div>").html(a).text();return a.replace(/[\n\r]/g," ")}function Ca(a,b,c,d){if(c){a=b?a.split(" "):
-Ea(a).split(" ");a="^(?=.*?"+a.join(")(?=.*?")+").*$";return new RegExp(a,d?"i":"")}else{a=b?a:Ea(a);return new RegExp(a,d?"i":"")}}function Ya(a,b){if(typeof l.ext.ofnSearch[b]==="function")return l.ext.ofnSearch[b](a);else if(a===null)return"";else if(b=="html")return a.replace(/[\r\n]/g," ").replace(/<.*?>/g,"");else if(typeof a==="string")return a.replace(/[\r\n]/g," ");return a}function Ea(a){return a.replace(new RegExp("(\\/|\\.|\\*|\\+|\\?|\\||\\(|\\)|\\[|\\]|\\{|\\}|\\\\|\\$|\\^|\\-)","g"),
-"\\$1")}function Ra(a){var b=s.createElement("div");b.className=a.oClasses.sInfo;if(!a.aanFeatures.i){a.aoDrawCallback.push({fn:Za,sName:"information"});b.id=a.sTableId+"_info"}a.nTable.setAttribute("aria-describedby",a.sTableId+"_info");return b}function Za(a){if(!(!a.oFeatures.bInfo||a.aanFeatures.i.length===0)){var b=a.oLanguage,c=a._iDisplayStart+1,d=a.fnDisplayEnd(),e=a.fnRecordsTotal(),f=a.fnRecordsDisplay(),g;g=f===0?b.sInfoEmpty:b.sInfo;if(f!=e)g+=" "+b.sInfoFiltered;g+=b.sInfoPostFix;g=za(a,
-g);if(b.fnInfoCallback!==null)g=b.fnInfoCallback.call(a.oInstance,a,c,d,e,f,g);a=a.aanFeatures.i;b=0;for(c=a.length;b<c;b++)i(a[b]).html(g)}}function za(a,b){var c=a.fnFormatNumber(a._iDisplayStart+1),d=a.fnDisplayEnd();d=a.fnFormatNumber(d);var e=a.fnRecordsDisplay();e=a.fnFormatNumber(e);var f=a.fnRecordsTotal();f=a.fnFormatNumber(f);if(a.oScroll.bInfinite)c=a.fnFormatNumber(1);return b.replace(/_START_/g,c).replace(/_END_/g,d).replace(/_TOTAL_/g,e).replace(/_MAX_/g,f)}function ra(a){var b,c,d=
-a.iInitDisplayStart;if(a.bInitialised===false)setTimeout(function(){ra(a)},200);else{Ma(a);Ka(a);ia(a,a.aoHeader);a.nTFoot&&ia(a,a.aoFooter);P(a,true);a.oFeatures.bAutoWidth&&ta(a);b=0;for(c=a.aoColumns.length;b<c;b++)if(a.aoColumns[b].sWidth!==null)a.aoColumns[b].nTh.style.width=t(a.aoColumns[b].sWidth);if(a.oFeatures.bSort)$(a);else if(a.oFeatures.bFilter)X(a,a.oPreviousSearch);else{a.aiDisplay=a.aiDisplayMaster.slice();I(a);H(a)}if(a.sAjaxSource!==null&&!a.oFeatures.bServerSide){c=[];Aa(a,c);a.fnServerData.call(a.oInstance,
-a.sAjaxSource,c,function(e){var f=a.sAjaxDataProp!==""?ca(a.sAjaxDataProp)(e):e;for(b=0;b<f.length;b++)R(a,f[b]);a.iInitDisplayStart=d;if(a.oFeatures.bSort)$(a);else{a.aiDisplay=a.aiDisplayMaster.slice();I(a);H(a)}P(a,false);pa(a,e)},a)}else if(!a.oFeatures.bServerSide){P(a,false);pa(a)}}}function pa(a,b){a._bInitComplete=true;K(a,"aoInitComplete","init",[a,b])}function Fa(a){var b=l.defaults.oLanguage;!a.sEmptyTable&&a.sZeroRecords&&b.sEmptyTable==="No data available in table"&&r(a,a,"sZeroRecords",
-"sEmptyTable");!a.sLoadingRecords&&a.sZeroRecords&&b.sLoadingRecords==="Loading..."&&r(a,a,"sZeroRecords","sLoadingRecords")}function Na(a){if(a.oScroll.bInfinite)return null;var b='<select size="1" '+('name="'+a.sTableId+'_length"')+">",c,d,e=a.aLengthMenu;if(e.length==2&&typeof e[0]==="object"&&typeof e[1]==="object"){c=0;for(d=e[0].length;c<d;c++)b+='<option value="'+e[0][c]+'">'+e[1][c]+"</option>"}else{c=0;for(d=e.length;c<d;c++)b+='<option value="'+e[c]+'">'+e[c]+"</option>"}b+="</select>";
-e=s.createElement("div");if(!a.aanFeatures.l)e.id=a.sTableId+"_length";e.className=a.oClasses.sLength;e.innerHTML="<label>"+a.oLanguage.sLengthMenu.replace("_MENU_",b)+"</label>";i('select option[value="'+a._iDisplayLength+'"]',e).attr("selected",true);i("select",e).bind("change.DT",function(){var f=i(this).val(),g=a.aanFeatures.l;c=0;for(d=g.length;c<d;c++)g[c]!=this.parentNode&&i("select",g[c]).val(f);a._iDisplayLength=parseInt(f,10);I(a);if(a.fnDisplayEnd()==a.fnRecordsDisplay()){a._iDisplayStart=
-a.fnDisplayEnd()-a._iDisplayLength;if(a._iDisplayStart<0)a._iDisplayStart=0}if(a._iDisplayLength==-1)a._iDisplayStart=0;H(a)});i("select",e).attr("aria-controls",a.sTableId);return e}function I(a){a._iDisplayEnd=a.oFeatures.bPaginate===false?a.aiDisplay.length:a._iDisplayStart+a._iDisplayLength>a.aiDisplay.length||a._iDisplayLength==-1?a.aiDisplay.length:a._iDisplayStart+a._iDisplayLength}function Sa(a){if(a.oScroll.bInfinite)return null;var b=s.createElement("div");b.className=a.oClasses.sPaging+
-a.sPaginationType;l.ext.oPagination[a.sPaginationType].fnInit(a,b,function(c){I(c);H(c)});a.aanFeatures.p||a.aoDrawCallback.push({fn:function(c){l.ext.oPagination[c.sPaginationType].fnUpdate(c,function(d){I(d);H(d)})},sName:"pagination"});return b}function Ga(a,b){var c=a._iDisplayStart;if(typeof b==="number"){a._iDisplayStart=b*a._iDisplayLength;if(a._iDisplayStart>a.fnRecordsDisplay())a._iDisplayStart=0}else if(b=="first")a._iDisplayStart=0;else if(b=="previous"){a._iDisplayStart=a._iDisplayLength>=
-0?a._iDisplayStart-a._iDisplayLength:0;if(a._iDisplayStart<0)a._iDisplayStart=0}else if(b=="next")if(a._iDisplayLength>=0){if(a._iDisplayStart+a._iDisplayLength<a.fnRecordsDisplay())a._iDisplayStart+=a._iDisplayLength}else a._iDisplayStart=0;else if(b=="last")if(a._iDisplayLength>=0){b=parseInt((a.fnRecordsDisplay()-1)/a._iDisplayLength,10)+1;a._iDisplayStart=(b-1)*a._iDisplayLength}else a._iDisplayStart=0;else O(a,0,"Unknown paging action: "+b);i(a.oInstance).trigger("page",a);return c!=a._iDisplayStart}
-function Pa(a){var b=s.createElement("div");if(!a.aanFeatures.r)b.id=a.sTableId+"_processing";b.innerHTML=a.oLanguage.sProcessing;b.className=a.oClasses.sProcessing;a.nTable.parentNode.insertBefore(b,a.nTable);return b}function P(a,b){if(a.oFeatures.bProcessing)for(var c=a.aanFeatures.r,d=0,e=c.length;d<e;d++)c[d].style.visibility=b?"visible":"hidden";i(a.oInstance).trigger("processing",[a,b])}function Qa(a){if(a.oScroll.sX===""&&a.oScroll.sY==="")return a.nTable;var b=s.createElement("div"),c=s.createElement("div"),
-d=s.createElement("div"),e=s.createElement("div"),f=s.createElement("div"),g=s.createElement("div"),j=a.nTable.cloneNode(false),k=a.nTable.cloneNode(false),m=a.nTable.getElementsByTagName("thead")[0],u=a.nTable.getElementsByTagName("tfoot").length===0?null:a.nTable.getElementsByTagName("tfoot")[0],x=a.oClasses;c.appendChild(d);f.appendChild(g);e.appendChild(a.nTable);b.appendChild(c);b.appendChild(e);d.appendChild(j);j.appendChild(m);if(u!==null){b.appendChild(f);g.appendChild(k);k.appendChild(u)}b.className=
-x.sScrollWrapper;c.className=x.sScrollHead;d.className=x.sScrollHeadInner;e.className=x.sScrollBody;f.className=x.sScrollFoot;g.className=x.sScrollFootInner;if(a.oScroll.bAutoCss){c.style.overflow="hidden";c.style.position="relative";f.style.overflow="hidden";e.style.overflow="auto"}c.style.border="0";c.style.width="100%";f.style.border="0";d.style.width=a.oScroll.sXInner!==""?a.oScroll.sXInner:"100%";j.removeAttribute("id");j.style.marginLeft="0";a.nTable.style.marginLeft="0";if(u!==null){k.removeAttribute("id");
-k.style.marginLeft="0"}d=i(a.nTable).children("caption");if(d.length>0){d=d[0];if(d._captionSide==="top")j.appendChild(d);else d._captionSide==="bottom"&&u&&k.appendChild(d)}if(a.oScroll.sX!==""){c.style.width=t(a.oScroll.sX);e.style.width=t(a.oScroll.sX);if(u!==null)f.style.width=t(a.oScroll.sX);i(e).scroll(function(){c.scrollLeft=this.scrollLeft;if(u!==null)f.scrollLeft=this.scrollLeft})}if(a.oScroll.sY!=="")e.style.height=t(a.oScroll.sY);a.aoDrawCallback.push({fn:$a,sName:"scrolling"});a.oScroll.bInfinite&&
-i(e).scroll(function(){if(!a.bDrawing&&i(this).scrollTop()!==0)if(i(this).scrollTop()+i(this).height()>i(a.nTable).height()-a.oScroll.iLoadGap)if(a.fnDisplayEnd()<a.fnRecordsDisplay()){Ga(a,"next");I(a);H(a)}});a.nScrollHead=c;a.nScrollFoot=f;return b}function $a(a){var b=a.nScrollHead.getElementsByTagName("div")[0],c=b.getElementsByTagName("table")[0],d=a.nTable.parentNode,e,f,g,j,k,m,u,x,y=[],B=[],T=a.nTFoot!==null?a.nScrollFoot.getElementsByTagName("div")[0]:null,M=a.nTFoot!==null?T.getElementsByTagName("table")[0]:
-null,L=a.oBrowser.bScrollOversize,ja=function(z){u=z.style;u.paddingTop="0";u.paddingBottom="0";u.borderTopWidth="0";u.borderBottomWidth="0";u.height=0};i(a.nTable).children("thead, tfoot").remove();e=i(a.nTHead).clone()[0];a.nTable.insertBefore(e,a.nTable.childNodes[0]);g=a.nTHead.getElementsByTagName("tr");j=e.getElementsByTagName("tr");if(a.nTFoot!==null){k=i(a.nTFoot).clone()[0];a.nTable.insertBefore(k,a.nTable.childNodes[1]);m=a.nTFoot.getElementsByTagName("tr");k=k.getElementsByTagName("tr")}if(a.oScroll.sX===
-""){d.style.width="100%";b.parentNode.style.width="100%"}var U=Z(a,e);e=0;for(f=U.length;e<f;e++){x=v(a,e);U[e].style.width=a.aoColumns[x].sWidth}a.nTFoot!==null&&N(function(z){z.style.width=""},k);if(a.oScroll.bCollapse&&a.oScroll.sY!=="")d.style.height=d.offsetHeight+a.nTHead.offsetHeight+"px";e=i(a.nTable).outerWidth();if(a.oScroll.sX===""){a.nTable.style.width="100%";if(L&&(i("tbody",d).height()>d.offsetHeight||i(d).css("overflow-y")=="scroll"))a.nTable.style.width=t(i(a.nTable).outerWidth()-
-a.oScroll.iBarWidth)}else if(a.oScroll.sXInner!=="")a.nTable.style.width=t(a.oScroll.sXInner);else if(e==i(d).width()&&i(d).height()<i(a.nTable).height()){a.nTable.style.width=t(e-a.oScroll.iBarWidth);if(i(a.nTable).outerWidth()>e-a.oScroll.iBarWidth)a.nTable.style.width=t(e)}else a.nTable.style.width=t(e);e=i(a.nTable).outerWidth();N(ja,j);N(function(z){y.push(t(i(z).width()))},j);N(function(z,Q){z.style.width=y[Q]},g);i(j).height(0);if(a.nTFoot!==null){N(ja,k);N(function(z){B.push(t(i(z).width()))},
-k);N(function(z,Q){z.style.width=B[Q]},m);i(k).height(0)}N(function(z,Q){z.innerHTML="";z.style.width=y[Q]},j);a.nTFoot!==null&&N(function(z,Q){z.innerHTML="";z.style.width=B[Q]},k);if(i(a.nTable).outerWidth()<e){g=d.scrollHeight>d.offsetHeight||i(d).css("overflow-y")=="scroll"?e+a.oScroll.iBarWidth:e;if(L&&(d.scrollHeight>d.offsetHeight||i(d).css("overflow-y")=="scroll"))a.nTable.style.width=t(g-a.oScroll.iBarWidth);d.style.width=t(g);a.nScrollHead.style.width=t(g);if(a.nTFoot!==null)a.nScrollFoot.style.width=
-t(g);if(a.oScroll.sX==="")O(a,1,"The table cannot fit into the current element which will cause column misalignment. The table has been drawn at its minimum possible width.");else a.oScroll.sXInner!==""&&O(a,1,"The table cannot fit into the current element which will cause column misalignment. Increase the sScrollXInner value or remove it to allow automatic calculation")}else{d.style.width=t("100%");a.nScrollHead.style.width=t("100%");if(a.nTFoot!==null)a.nScrollFoot.style.width=t("100%")}if(a.oScroll.sY===
-"")if(L)d.style.height=t(a.nTable.offsetHeight+a.oScroll.iBarWidth);if(a.oScroll.sY!==""&&a.oScroll.bCollapse){d.style.height=t(a.oScroll.sY);L=a.oScroll.sX!==""&&a.nTable.offsetWidth>d.offsetWidth?a.oScroll.iBarWidth:0;if(a.nTable.offsetHeight<d.offsetHeight)d.style.height=t(a.nTable.offsetHeight+L)}L=i(a.nTable).outerWidth();c.style.width=t(L);b.style.width=t(L);c=i(a.nTable).height()>d.clientHeight||i(d).css("overflow-y")=="scroll";b.style.paddingRight=c?a.oScroll.iBarWidth+"px":"0px";if(a.nTFoot!==
-null){M.style.width=t(L);T.style.width=t(L);T.style.paddingRight=c?a.oScroll.iBarWidth+"px":"0px"}i(d).scroll();if(a.bSorted||a.bFiltered)d.scrollTop=0}function N(a,b,c){for(var d=0,e=0,f=b.length,g,j;e<f;){g=b[e].firstChild;for(j=c?c[e].firstChild:null;g;){if(g.nodeType===1){c?a(g,j,d):a(g,d);d++}g=g.nextSibling;j=c?j.nextSibling:null}e++}}function ab(a,b){if(!a||a===null||a==="")return 0;if(!b)b=s.body;var c=s.createElement("div");c.style.width=t(a);b.appendChild(c);a=c.offsetWidth;b.removeChild(c);
-return a}function ta(a){var b=0,c,d=0,e=a.aoColumns.length,f,g,j=i("th",a.nTHead),k=a.nTable.getAttribute("width");g=a.nTable.parentNode;for(f=0;f<e;f++)if(a.aoColumns[f].bVisible){d++;if(a.aoColumns[f].sWidth!==null){c=ab(a.aoColumns[f].sWidthOrig,g);if(c!==null)a.aoColumns[f].sWidth=t(c);b++}}if(e==j.length&&b===0&&d==e&&a.oScroll.sX===""&&a.oScroll.sY==="")for(f=0;f<a.aoColumns.length;f++){c=i(j[f]).width();if(c!==null)a.aoColumns[f].sWidth=t(c)}else{b=a.nTable.cloneNode(false);f=a.nTHead.cloneNode(true);
-d=s.createElement("tbody");c=s.createElement("tr");b.removeAttribute("id");b.appendChild(f);if(a.nTFoot!==null){b.appendChild(a.nTFoot.cloneNode(true));N(function(u){u.style.width=""},b.getElementsByTagName("tr"))}b.appendChild(d);d.appendChild(c);d=i("thead th",b);if(d.length===0)d=i("tbody tr:eq(0)>td",b);j=Z(a,f);for(f=d=0;f<e;f++){var m=a.aoColumns[f];if(m.bVisible&&m.sWidthOrig!==null&&m.sWidthOrig!=="")j[f-d].style.width=t(m.sWidthOrig);else if(m.bVisible)j[f-d].style.width="";else d++}for(f=
-0;f<e;f++)if(a.aoColumns[f].bVisible){d=bb(a,f);if(d!==null){d=d.cloneNode(true);if(a.aoColumns[f].sContentPadding!=="")d.innerHTML+=a.aoColumns[f].sContentPadding;c.appendChild(d)}}g.appendChild(b);if(a.oScroll.sX!==""&&a.oScroll.sXInner!=="")b.style.width=t(a.oScroll.sXInner);else if(a.oScroll.sX!==""){b.style.width="";if(i(b).width()<g.offsetWidth)b.style.width=t(g.offsetWidth)}else if(a.oScroll.sY!=="")b.style.width=t(g.offsetWidth);else if(k)b.style.width=t(k);b.style.visibility="hidden";cb(a,
-b);e=i("tbody tr:eq(0)",b).children();if(e.length===0)e=Z(a,i("thead",b)[0]);if(a.oScroll.sX!==""){for(f=d=g=0;f<a.aoColumns.length;f++)if(a.aoColumns[f].bVisible){g+=a.aoColumns[f].sWidthOrig===null?i(e[d]).outerWidth():parseInt(a.aoColumns[f].sWidth.replace("px",""),10)+(i(e[d]).outerWidth()-i(e[d]).width());d++}b.style.width=t(g);a.nTable.style.width=t(g)}for(f=d=0;f<a.aoColumns.length;f++)if(a.aoColumns[f].bVisible){g=i(e[d]).width();if(g!==null&&g>0)a.aoColumns[f].sWidth=t(g);d++}e=i(b).css("width");
-a.nTable.style.width=e.indexOf("%")!==-1?e:t(i(b).outerWidth());b.parentNode.removeChild(b)}if(k)a.nTable.style.width=t(k)}function cb(a,b){if(a.oScroll.sX===""&&a.oScroll.sY!==""){i(b).width();b.style.width=t(i(b).outerWidth()-a.oScroll.iBarWidth)}else if(a.oScroll.sX!=="")b.style.width=t(i(b).outerWidth())}function bb(a,b){var c=db(a,b);if(c<0)return null;if(a.aoData[c].nTr===null){var d=s.createElement("td");d.innerHTML=F(a,c,b,"");return d}return W(a,c)[b]}function db(a,b){for(var c=-1,d=-1,e=
-0;e<a.aoData.length;e++){var f=F(a,e,b,"display")+"";f=f.replace(/<.*?>/g,"");if(f.length>c){c=f.length;d=e}}return d}function t(a){if(a===null)return"0px";if(typeof a=="number"){if(a<0)return"0px";return a+"px"}var b=a.charCodeAt(a.length-1);if(b<48||b>57)return a;return a+"px"}function eb(){var a=s.createElement("p"),b=a.style;b.width="100%";b.height="200px";b.padding="0px";var c=s.createElement("div");b=c.style;b.position="absolute";b.top="0px";b.left="0px";b.visibility="hidden";b.width="200px";
-b.height="150px";b.padding="0px";b.overflow="hidden";c.appendChild(a);s.body.appendChild(c);b=a.offsetWidth;c.style.overflow="scroll";a=a.offsetWidth;if(b==a)a=c.clientWidth;s.body.removeChild(c);return b-a}function $(a,b){var c,d,e,f,g,j,k=[],m=[],u=l.ext.oSort,x=a.aoData,y=a.aoColumns,B=a.oLanguage.oAria;if(!a.oFeatures.bServerSide&&(a.aaSorting.length!==0||a.aaSortingFixed!==null)){k=a.aaSortingFixed!==null?a.aaSortingFixed.concat(a.aaSorting):a.aaSorting.slice();for(c=0;c<k.length;c++){d=k[c][0];
-e=w(a,d);f=a.aoColumns[d].sSortDataType;if(l.ext.afnSortData[f]){g=l.ext.afnSortData[f].call(a.oInstance,a,d,e);if(g.length===x.length){e=0;for(f=x.length;e<f;e++)S(a,e,d,g[e])}else O(a,0,"Returned data sort array (col "+d+") is the wrong length")}}c=0;for(d=a.aiDisplayMaster.length;c<d;c++)m[a.aiDisplayMaster[c]]=c;var T=k.length,M;c=0;for(d=x.length;c<d;c++)for(e=0;e<T;e++){M=y[k[e][0]].aDataSort;g=0;for(j=M.length;g<j;g++){f=y[M[g]].sType;f=u[(f?f:"string")+"-pre"];x[c]._aSortData[M[g]]=f?f(F(a,
-c,M[g],"sort")):F(a,c,M[g],"sort")}}a.aiDisplayMaster.sort(function(L,ja){var U,z,Q,aa,ka;for(U=0;U<T;U++){ka=y[k[U][0]].aDataSort;z=0;for(Q=ka.length;z<Q;z++){aa=y[ka[z]].sType;aa=u[(aa?aa:"string")+"-"+k[U][1]](x[L]._aSortData[ka[z]],x[ja]._aSortData[ka[z]]);if(aa!==0)return aa}}return u["numeric-asc"](m[L],m[ja])})}if((b===p||b)&&!a.oFeatures.bDeferRender)ba(a);c=0;for(d=a.aoColumns.length;c<d;c++){e=y[c].sTitle.replace(/<.*?>/g,"");b=y[c].nTh;b.removeAttribute("aria-sort");b.removeAttribute("aria-label");
-if(y[c].bSortable)if(k.length>0&&k[0][0]==c){b.setAttribute("aria-sort",k[0][1]=="asc"?"ascending":"descending");b.setAttribute("aria-label",e+((y[c].asSorting[k[0][2]+1]?y[c].asSorting[k[0][2]+1]:y[c].asSorting[0])=="asc"?B.sSortAscending:B.sSortDescending))}else b.setAttribute("aria-label",e+(y[c].asSorting[0]=="asc"?B.sSortAscending:B.sSortDescending));else b.setAttribute("aria-label",e)}a.bSorted=true;i(a.oInstance).trigger("sort",a);if(a.oFeatures.bFilter)X(a,a.oPreviousSearch,1);else{a.aiDisplay=
-a.aiDisplayMaster.slice();a._iDisplayStart=0;I(a);H(a)}}function ya(a,b,c,d){fb(b,{},function(e){if(a.aoColumns[c].bSortable!==false){var f=function(){var g,j;if(e.shiftKey){for(var k=false,m=0;m<a.aaSorting.length;m++)if(a.aaSorting[m][0]==c){k=true;g=a.aaSorting[m][0];j=a.aaSorting[m][2]+1;if(a.aoColumns[g].asSorting[j]){a.aaSorting[m][1]=a.aoColumns[g].asSorting[j];a.aaSorting[m][2]=j}else a.aaSorting.splice(m,1);break}k===false&&a.aaSorting.push([c,a.aoColumns[c].asSorting[0],0])}else if(a.aaSorting.length==
-1&&a.aaSorting[0][0]==c){g=a.aaSorting[0][0];j=a.aaSorting[0][2]+1;a.aoColumns[g].asSorting[j]||(j=0);a.aaSorting[0][1]=a.aoColumns[g].asSorting[j];a.aaSorting[0][2]=j}else{a.aaSorting.splice(0,a.aaSorting.length);a.aaSorting.push([c,a.aoColumns[c].asSorting[0],0])}$(a)};if(a.oFeatures.bProcessing){P(a,true);setTimeout(function(){f();a.oFeatures.bServerSide||P(a,false)},0)}else f();typeof d=="function"&&d(a)}})}function ba(a){var b,c,d,e,f,g=a.aoColumns.length,j=a.oClasses;for(b=0;b<g;b++)a.aoColumns[b].bSortable&&
-i(a.aoColumns[b].nTh).removeClass(j.sSortAsc+" "+j.sSortDesc+" "+a.aoColumns[b].sSortingClass);c=a.aaSortingFixed!==null?a.aaSortingFixed.concat(a.aaSorting):a.aaSorting.slice();for(b=0;b<a.aoColumns.length;b++)if(a.aoColumns[b].bSortable){f=a.aoColumns[b].sSortingClass;e=-1;for(d=0;d<c.length;d++)if(c[d][0]==b){f=c[d][1]=="asc"?j.sSortAsc:j.sSortDesc;e=d;break}i(a.aoColumns[b].nTh).addClass(f);if(a.bJUI){f=i("span."+j.sSortIcon,a.aoColumns[b].nTh);f.removeClass(j.sSortJUIAsc+" "+j.sSortJUIDesc+" "+
-j.sSortJUI+" "+j.sSortJUIAscAllowed+" "+j.sSortJUIDescAllowed);f.addClass(e==-1?a.aoColumns[b].sSortingClassJUI:c[e][1]=="asc"?j.sSortJUIAsc:j.sSortJUIDesc)}}else i(a.aoColumns[b].nTh).addClass(a.aoColumns[b].sSortingClass);f=j.sSortColumn;if(a.oFeatures.bSort&&a.oFeatures.bSortClasses){a=W(a);e=[];for(b=0;b<g;b++)e.push("");b=0;for(d=1;b<c.length;b++){j=parseInt(c[b][0],10);e[j]=f+d;d<3&&d++}f=new RegExp(f+"[123]");var k;b=0;for(c=a.length;b<c;b++){j=b%g;d=a[b].className;k=e[j];j=d.replace(f,k);
-if(j!=d)a[b].className=i.trim(j);else if(k.length>0&&d.indexOf(k)==-1)a[b].className=d+" "+k}}}function Ha(a){if(!(!a.oFeatures.bStateSave||a.bDestroying)){var b,c;b=a.oScroll.bInfinite;var d={iCreate:(new Date).getTime(),iStart:b?0:a._iDisplayStart,iEnd:b?a._iDisplayLength:a._iDisplayEnd,iLength:a._iDisplayLength,aaSorting:i.extend(true,[],a.aaSorting),oSearch:i.extend(true,{},a.oPreviousSearch),aoSearchCols:i.extend(true,[],a.aoPreSearchCols),abVisCols:[]};b=0;for(c=a.aoColumns.length;b<c;b++)d.abVisCols.push(a.aoColumns[b].bVisible);
-K(a,"aoStateSaveParams","stateSaveParams",[a,d]);a.fnStateSave.call(a.oInstance,a,d)}}function gb(a,b){if(a.oFeatures.bStateSave){var c=a.fnStateLoad.call(a.oInstance,a);if(c){var d=K(a,"aoStateLoadParams","stateLoadParams",[a,c]);if(i.inArray(false,d)===-1){a.oLoadedState=i.extend(true,{},c);a._iDisplayStart=c.iStart;a.iInitDisplayStart=c.iStart;a._iDisplayEnd=c.iEnd;a._iDisplayLength=c.iLength;a.aaSorting=c.aaSorting.slice();a.saved_aaSorting=c.aaSorting.slice();i.extend(a.oPreviousSearch,c.oSearch);
-i.extend(true,a.aoPreSearchCols,c.aoSearchCols);b.saved_aoColumns=[];for(d=0;d<c.abVisCols.length;d++){b.saved_aoColumns[d]={};b.saved_aoColumns[d].bVisible=c.abVisCols[d]}K(a,"aoStateLoaded","stateLoaded",[a,c])}}}}function lb(a,b,c,d,e){var f=new Date;f.setTime(f.getTime()+c*1E3);c=la.location.pathname.split("/");a=a+"_"+c.pop().replace(/[\/:]/g,"").toLowerCase();var g;if(e!==null){g=typeof i.parseJSON==="function"?i.parseJSON(b):eval("("+b+")");b=e(a,g,f.toGMTString(),c.join("/")+"/")}else b=a+
-"="+encodeURIComponent(b)+"; expires="+f.toGMTString()+"; path="+c.join("/")+"/";a=s.cookie.split(";");e=b.split(";")[0].length;f=[];if(e+s.cookie.length+10>4096){for(var j=0,k=a.length;j<k;j++)if(a[j].indexOf(d)!=-1){var m=a[j].split("=");try{(g=eval("("+decodeURIComponent(m[1])+")"))&&g.iCreate&&f.push({name:m[0],time:g.iCreate})}catch(u){}}for(f.sort(function(x,y){return y.time-x.time});e+s.cookie.length+10>4096;){if(f.length===0)return;d=f.pop();s.cookie=d.name+"=; expires=Thu, 01-Jan-1970 00:00:01 GMT; path="+
-c.join("/")+"/"}}s.cookie=b}function mb(a){var b=la.location.pathname.split("/");a=a+"_"+b[b.length-1].replace(/[\/:]/g,"").toLowerCase()+"=";b=s.cookie.split(";");for(var c=0;c<b.length;c++){for(var d=b[c];d.charAt(0)==" ";)d=d.substring(1,d.length);if(d.indexOf(a)===0)return decodeURIComponent(d.substring(a.length,d.length))}return null}function C(a){for(var b=0;b<l.settings.length;b++)if(l.settings[b].nTable===a)return l.settings[b];return null}function fa(a){var b=[];a=a.aoData;for(var c=0,d=
-a.length;c<d;c++)a[c].nTr!==null&&b.push(a[c].nTr);return b}function W(a,b){var c=[],d,e,f,g,j;e=0;var k=a.aoData.length;if(b!==p){e=b;k=b+1}for(e=e;e<k;e++){j=a.aoData[e];if(j.nTr!==null){b=[];for(d=j.nTr.firstChild;d;){f=d.nodeName.toLowerCase();if(f=="td"||f=="th")b.push(d);d=d.nextSibling}f=d=0;for(g=a.aoColumns.length;f<g;f++)if(a.aoColumns[f].bVisible)c.push(b[f-d]);else{c.push(j._anHidden[f]);d++}}}return c}function O(a,b,c){a=a===null?"DataTables warning: "+c:"DataTables warning (table id = '"+
-a.sTableId+"'): "+c;if(b===0)if(l.ext.sErrMode=="alert")alert(a);else throw new Error(a);else la.console&&console.log&&console.log(a)}function r(a,b,c,d){if(d===p)d=c;if(b[c]!==p)a[d]=b[c]}function hb(a,b){var c;for(var d in b)if(b.hasOwnProperty(d)){c=b[d];if(typeof h[d]==="object"&&c!==null&&i.isArray(c)===false)i.extend(true,a[d],c);else a[d]=c}return a}function fb(a,b,c){i(a).bind("click.DT",b,function(d){a.blur();c(d)}).bind("keypress.DT",b,function(d){d.which===13&&c(d)}).bind("selectstart.DT",
-function(){return false})}function J(a,b,c,d){c&&a[b].push({fn:c,sName:d})}function K(a,b,c,d){b=a[b];for(var e=[],f=b.length-1;f>=0;f--)e.push(b[f].fn.apply(a.oInstance,d));c!==null&&i(a.oInstance).trigger(c,d);return e}function ib(a){var b=i('<div style="position:absolute; top:0; left:0; height:1px; width:1px; overflow:hidden"><div style="position:absolute; top:1px; left:1px; width:100px; overflow:scroll;"><div id="DT_BrowserTest" style="width:100%; height:10px;"></div></div></div>')[0];s.body.appendChild(b);
-a.oBrowser.bScrollOversize=i("#DT_BrowserTest",b)[0].offsetWidth===100?true:false;s.body.removeChild(b)}function jb(a){return function(){var b=[C(this[l.ext.iApiIndex])].concat(Array.prototype.slice.call(arguments));return l.ext.oApi[a].apply(this,b)}}var ga=/\[.*?\]$/,kb=la.JSON?JSON.stringify:function(a){var b=typeof a;if(b!=="object"||a===null){if(b==="string")a='"'+a+'"';return a+""}var c,d,e=[],f=i.isArray(a);for(c in a){d=a[c];b=typeof d;if(b==="string")d='"'+d+'"';else if(b==="object"&&d!==
-null)d=kb(d);e.push((f?"":'"'+c+'":')+d)}return(f?"[":"{")+e+(f?"]":"}")};this.$=function(a,b){var c,d=[],e;c=C(this[l.ext.iApiIndex]);var f=c.aoData,g=c.aiDisplay,j=c.aiDisplayMaster;b||(b={});b=i.extend({},{filter:"none",order:"current",page:"all"},b);if(b.page=="current"){b=c._iDisplayStart;for(c=c.fnDisplayEnd();b<c;b++)(e=f[g[b]].nTr)&&d.push(e)}else if(b.order=="current"&&b.filter=="none"){b=0;for(c=j.length;b<c;b++)(e=f[j[b]].nTr)&&d.push(e)}else if(b.order=="current"&&b.filter=="applied"){b=
-0;for(c=g.length;b<c;b++)(e=f[g[b]].nTr)&&d.push(e)}else if(b.order=="original"&&b.filter=="none"){b=0;for(c=f.length;b<c;b++)(e=f[b].nTr)&&d.push(e)}else if(b.order=="original"&&b.filter=="applied"){b=0;for(c=f.length;b<c;b++){e=f[b].nTr;i.inArray(b,g)!==-1&&e&&d.push(e)}}else O(c,1,"Unknown selection options");f=i(d);d=f.filter(a);a=f.find(a);return i([].concat(i.makeArray(d),i.makeArray(a)))};this._=function(a,b){var c=[],d=this.$(a,b);a=0;for(b=d.length;a<b;a++)c.push(this.fnGetData(d[a]));return c};
-this.fnAddData=function(a,b){if(a.length===0)return[];var c=[],d,e=C(this[l.ext.iApiIndex]);if(typeof a[0]==="object"&&a[0]!==null)for(var f=0;f<a.length;f++){d=R(e,a[f]);if(d==-1)return c;c.push(d)}else{d=R(e,a);if(d==-1)return c;c.push(d)}e.aiDisplay=e.aiDisplayMaster.slice();if(b===p||b)qa(e);return c};this.fnAdjustColumnSizing=function(a){var b=C(this[l.ext.iApiIndex]);o(b);if(a===p||a)this.fnDraw(false);else if(b.oScroll.sX!==""||b.oScroll.sY!=="")this.oApi._fnScrollDraw(b)};this.fnClearTable=
-function(a){var b=C(this[l.ext.iApiIndex]);wa(b);if(a===p||a)H(b)};this.fnClose=function(a){for(var b=C(this[l.ext.iApiIndex]),c=0;c<b.aoOpenRows.length;c++)if(b.aoOpenRows[c].nParent==a){(a=b.aoOpenRows[c].nTr.parentNode)&&a.removeChild(b.aoOpenRows[c].nTr);b.aoOpenRows.splice(c,1);return 0}return 1};this.fnDeleteRow=function(a,b,c){var d=C(this[l.ext.iApiIndex]),e,f;a=typeof a==="object"?V(d,a):a;var g=d.aoData.splice(a,1);e=0;for(f=d.aoData.length;e<f;e++)if(d.aoData[e].nTr!==null)d.aoData[e].nTr._DT_RowIndex=
-e;e=i.inArray(a,d.aiDisplay);d.asDataSearch.splice(e,1);xa(d.aiDisplayMaster,a);xa(d.aiDisplay,a);typeof b==="function"&&b.call(this,d,g);if(d._iDisplayStart>=d.fnRecordsDisplay()){d._iDisplayStart-=d._iDisplayLength;if(d._iDisplayStart<0)d._iDisplayStart=0}if(c===p||c){I(d);H(d)}return g};this.fnDestroy=function(a){var b=C(this[l.ext.iApiIndex]),c=b.nTableWrapper.parentNode,d=b.nTBody,e,f;a=a===p?false:a;b.bDestroying=true;K(b,"aoDestroyCallback","destroy",[b]);if(!a){e=0;for(f=b.aoColumns.length;e<
-f;e++)b.aoColumns[e].bVisible===false&&this.fnSetColumnVis(e,true)}i(b.nTableWrapper).find("*").andSelf().unbind(".DT");i("tbody>tr>td."+b.oClasses.sRowEmpty,b.nTable).parent().remove();if(b.nTable!=b.nTHead.parentNode){i(b.nTable).children("thead").remove();b.nTable.appendChild(b.nTHead)}if(b.nTFoot&&b.nTable!=b.nTFoot.parentNode){i(b.nTable).children("tfoot").remove();b.nTable.appendChild(b.nTFoot)}b.nTable.parentNode.removeChild(b.nTable);i(b.nTableWrapper).remove();b.aaSorting=[];b.aaSortingFixed=
-[];ba(b);i(fa(b)).removeClass(b.asStripeClasses.join(" "));i("th, td",b.nTHead).removeClass([b.oClasses.sSortable,b.oClasses.sSortableAsc,b.oClasses.sSortableDesc,b.oClasses.sSortableNone].join(" "));if(b.bJUI){i("th span."+b.oClasses.sSortIcon+", td span."+b.oClasses.sSortIcon,b.nTHead).remove();i("th, td",b.nTHead).each(function(){var g=i("div."+b.oClasses.sSortJUIWrapper,this),j=g.contents();i(this).append(j);g.remove()})}if(!a&&b.nTableReinsertBefore)c.insertBefore(b.nTable,b.nTableReinsertBefore);
-else a||c.appendChild(b.nTable);e=0;for(f=b.aoData.length;e<f;e++)b.aoData[e].nTr!==null&&d.appendChild(b.aoData[e].nTr);if(b.oFeatures.bAutoWidth===true)b.nTable.style.width=t(b.sDestroyWidth);if(f=b.asDestroyStripes.length){a=i(d).children("tr");for(e=0;e<f;e++)a.filter(":nth-child("+f+"n + "+e+")").addClass(b.asDestroyStripes[e])}e=0;for(f=l.settings.length;e<f;e++)l.settings[e]==b&&l.settings.splice(e,1);h=b=null};this.fnDraw=function(a){var b=C(this[l.ext.iApiIndex]);if(a===false){I(b);H(b)}else qa(b)};
-this.fnFilter=function(a,b,c,d,e,f){var g=C(this[l.ext.iApiIndex]);if(g.oFeatures.bFilter){if(c===p||c===null)c=false;if(d===p||d===null)d=true;if(e===p||e===null)e=true;if(f===p||f===null)f=true;if(b===p||b===null){X(g,{sSearch:a+"",bRegex:c,bSmart:d,bCaseInsensitive:f},1);if(e&&g.aanFeatures.f){b=g.aanFeatures.f;c=0;for(d=b.length;c<d;c++)try{b[c]._DT_Input!=s.activeElement&&i(b[c]._DT_Input).val(a)}catch(j){i(b[c]._DT_Input).val(a)}}}else{i.extend(g.aoPreSearchCols[b],{sSearch:a+"",bRegex:c,bSmart:d,
-bCaseInsensitive:f});X(g,g.oPreviousSearch,1)}}};this.fnGetData=function(a,b){var c=C(this[l.ext.iApiIndex]);if(a!==p){var d=a;if(typeof a==="object"){var e=a.nodeName.toLowerCase();if(e==="tr")d=V(c,a);else if(e==="td"){d=V(c,a.parentNode);b=va(c,d,a)}}if(b!==p)return F(c,d,b,"");return c.aoData[d]!==p?c.aoData[d]._aData:null}return oa(c)};this.fnGetNodes=function(a){var b=C(this[l.ext.iApiIndex]);if(a!==p)return b.aoData[a]!==p?b.aoData[a].nTr:null;return fa(b)};this.fnGetPosition=function(a){var b=
-C(this[l.ext.iApiIndex]),c=a.nodeName.toUpperCase();if(c=="TR")return V(b,a);else if(c=="TD"||c=="TH"){c=V(b,a.parentNode);a=va(b,c,a);return[c,w(b,a),a]}return null};this.fnIsOpen=function(a){for(var b=C(this[l.ext.iApiIndex]),c=0;c<b.aoOpenRows.length;c++)if(b.aoOpenRows[c].nParent==a)return true;return false};this.fnOpen=function(a,b,c){var d=C(this[l.ext.iApiIndex]),e=fa(d);if(i.inArray(a,e)!==-1){this.fnClose(a);e=s.createElement("tr");var f=s.createElement("td");e.appendChild(f);f.className=
-c;f.colSpan=D(d);if(typeof b==="string")f.innerHTML=b;else i(f).html(b);b=i("tr",d.nTBody);i.inArray(a,b)!=-1&&i(e).insertAfter(a);d.aoOpenRows.push({nTr:e,nParent:a});return e}};this.fnPageChange=function(a,b){var c=C(this[l.ext.iApiIndex]);Ga(c,a);I(c);if(b===p||b)H(c)};this.fnSetColumnVis=function(a,b,c){var d=C(this[l.ext.iApiIndex]),e,f,g=d.aoColumns,j=d.aoData,k,m;if(g[a].bVisible!=b){if(b){for(e=f=0;e<a;e++)g[e].bVisible&&f++;m=f>=D(d);if(!m)for(e=a;e<g.length;e++)if(g[e].bVisible){k=e;break}e=
-0;for(f=j.length;e<f;e++)if(j[e].nTr!==null)m?j[e].nTr.appendChild(j[e]._anHidden[a]):j[e].nTr.insertBefore(j[e]._anHidden[a],W(d,e)[k])}else{e=0;for(f=j.length;e<f;e++)if(j[e].nTr!==null){k=W(d,e)[a];j[e]._anHidden[a]=k;k.parentNode.removeChild(k)}}g[a].bVisible=b;ia(d,d.aoHeader);d.nTFoot&&ia(d,d.aoFooter);e=0;for(f=d.aoOpenRows.length;e<f;e++)d.aoOpenRows[e].nTr.colSpan=D(d);if(c===p||c){o(d);H(d)}Ha(d)}};this.fnSettings=function(){return C(this[l.ext.iApiIndex])};this.fnSort=function(a){var b=
-C(this[l.ext.iApiIndex]);b.aaSorting=a;$(b)};this.fnSortListener=function(a,b,c){ya(C(this[l.ext.iApiIndex]),a,b,c)};this.fnUpdate=function(a,b,c,d,e){var f=C(this[l.ext.iApiIndex]);b=typeof b==="object"?V(f,b):b;if(i.isArray(a)&&c===p){f.aoData[b]._aData=a.slice();for(c=0;c<f.aoColumns.length;c++)this.fnUpdate(F(f,b,c),b,c,false,false)}else if(i.isPlainObject(a)&&c===p){f.aoData[b]._aData=i.extend(true,{},a);for(c=0;c<f.aoColumns.length;c++)this.fnUpdate(F(f,b,c),b,c,false,false)}else{S(f,b,c,a);
-a=F(f,b,c,"display");var g=f.aoColumns[c];if(g.fnRender!==null){a=da(f,b,c);g.bUseRendered&&S(f,b,c,a)}if(f.aoData[b].nTr!==null)W(f,b)[c].innerHTML=a}c=i.inArray(b,f.aiDisplay);f.asDataSearch[c]=Da(f,na(f,b,"filter",A(f,"bSearchable")));if(e===p||e)o(f);if(d===p||d)qa(f);return 0};this.fnVersionCheck=l.ext.fnVersionCheck;this.oApi={_fnExternApiFunc:jb,_fnInitialise:ra,_fnInitComplete:pa,_fnLanguageCompat:Fa,_fnAddColumn:n,_fnColumnOptions:q,_fnAddData:R,_fnCreateTr:ua,_fnGatherData:ea,_fnBuildHead:Ka,
-_fnDrawHead:ia,_fnDraw:H,_fnReDraw:qa,_fnAjaxUpdate:La,_fnAjaxParameters:Ta,_fnAjaxUpdateDraw:Ua,_fnServerParams:Aa,_fnAddOptionsHtml:Ma,_fnFeatureHtmlTable:Qa,_fnScrollDraw:$a,_fnAdjustColumnSizing:o,_fnFeatureHtmlFilter:Oa,_fnFilterComplete:X,_fnFilterCustom:Xa,_fnFilterColumn:Wa,_fnFilter:Va,_fnBuildSearchArray:Ba,_fnBuildSearchRow:Da,_fnFilterCreateSearch:Ca,_fnDataToSearch:Ya,_fnSort:$,_fnSortAttachListener:ya,_fnSortingClasses:ba,_fnFeatureHtmlPaginate:Sa,_fnPageChange:Ga,_fnFeatureHtmlInfo:Ra,
-_fnUpdateInfo:Za,_fnFeatureHtmlLength:Na,_fnFeatureHtmlProcessing:Pa,_fnProcessingDisplay:P,_fnVisibleToColumnIndex:v,_fnColumnIndexToVisible:w,_fnNodeToDataIndex:V,_fnVisbleColumns:D,_fnCalculateEnd:I,_fnConvertToWidth:ab,_fnCalculateColumnWidths:ta,_fnScrollingWidthAdjust:cb,_fnGetWidestNode:bb,_fnGetMaxLenString:db,_fnStringToCss:t,_fnDetectType:G,_fnSettingsFromNode:C,_fnGetDataMaster:oa,_fnGetTrNodes:fa,_fnGetTdNodes:W,_fnEscapeRegex:Ea,_fnDeleteIndex:xa,_fnReOrderIndex:E,_fnColumnOrdering:Y,
-_fnLog:O,_fnClearTable:wa,_fnSaveState:Ha,_fnLoadState:gb,_fnCreateCookie:lb,_fnReadCookie:mb,_fnDetectHeader:ha,_fnGetUniqueThs:Z,_fnScrollBarWidth:eb,_fnApplyToChildren:N,_fnMap:r,_fnGetRowData:na,_fnGetCellData:F,_fnSetCellData:S,_fnGetObjectDataFn:ca,_fnSetObjectDataFn:Ja,_fnApplyColumnDefs:ma,_fnBindAction:fb,_fnExtend:hb,_fnCallbackReg:J,_fnCallbackFire:K,_fnJsonString:kb,_fnRender:da,_fnNodeToColumnIndex:va,_fnInfoMacros:za,_fnBrowserDetect:ib,_fnGetColumns:A};i.extend(l.ext.oApi,this.oApi);
-for(var Ia in l.ext.oApi)if(Ia)this[Ia]=jb(Ia);var sa=this;this.each(function(){var a=0,b,c,d;c=this.getAttribute("id");var e=false,f=false;if(this.nodeName.toLowerCase()!="table")O(null,0,"Attempted to initialise DataTables on a node which is not a table: "+this.nodeName);else{a=0;for(b=l.settings.length;a<b;a++){if(l.settings[a].nTable==this)if(h===p||h.bRetrieve)return l.settings[a].oInstance;else if(h.bDestroy){l.settings[a].oInstance.fnDestroy();break}else{O(l.settings[a],0,"Cannot reinitialise DataTable.\n\nTo retrieve the DataTables object for this table, pass no arguments or see the docs for bRetrieve and bDestroy");
-return}if(l.settings[a].sTableId==this.id){l.settings.splice(a,1);break}}if(c===null||c==="")this.id=c="DataTables_Table_"+l.ext._oExternConfig.iNextUnique++;var g=i.extend(true,{},l.models.oSettings,{nTable:this,oApi:sa.oApi,oInit:h,sDestroyWidth:i(this).width(),sInstance:c,sTableId:c});l.settings.push(g);g.oInstance=sa.length===1?sa:i(this).dataTable();h||(h={});h.oLanguage&&Fa(h.oLanguage);h=hb(i.extend(true,{},l.defaults),h);r(g.oFeatures,h,"bPaginate");r(g.oFeatures,h,"bLengthChange");r(g.oFeatures,
-h,"bFilter");r(g.oFeatures,h,"bSort");r(g.oFeatures,h,"bInfo");r(g.oFeatures,h,"bProcessing");r(g.oFeatures,h,"bAutoWidth");r(g.oFeatures,h,"bSortClasses");r(g.oFeatures,h,"bServerSide");r(g.oFeatures,h,"bDeferRender");r(g.oScroll,h,"sScrollX","sX");r(g.oScroll,h,"sScrollXInner","sXInner");r(g.oScroll,h,"sScrollY","sY");r(g.oScroll,h,"bScrollCollapse","bCollapse");r(g.oScroll,h,"bScrollInfinite","bInfinite");r(g.oScroll,h,"iScrollLoadGap","iLoadGap");r(g.oScroll,h,"bScrollAutoCss","bAutoCss");r(g,
-h,"asStripeClasses");r(g,h,"asStripClasses","asStripeClasses");r(g,h,"fnServerData");r(g,h,"fnFormatNumber");r(g,h,"sServerMethod");r(g,h,"aaSorting");r(g,h,"aaSortingFixed");r(g,h,"aLengthMenu");r(g,h,"sPaginationType");r(g,h,"sAjaxSource");r(g,h,"sAjaxDataProp");r(g,h,"iCookieDuration");r(g,h,"sCookiePrefix");r(g,h,"sDom");r(g,h,"bSortCellsTop");r(g,h,"iTabIndex");r(g,h,"oSearch","oPreviousSearch");r(g,h,"aoSearchCols","aoPreSearchCols");r(g,h,"iDisplayLength","_iDisplayLength");r(g,h,"bJQueryUI",
-"bJUI");r(g,h,"fnCookieCallback");r(g,h,"fnStateLoad");r(g,h,"fnStateSave");r(g.oLanguage,h,"fnInfoCallback");J(g,"aoDrawCallback",h.fnDrawCallback,"user");J(g,"aoServerParams",h.fnServerParams,"user");J(g,"aoStateSaveParams",h.fnStateSaveParams,"user");J(g,"aoStateLoadParams",h.fnStateLoadParams,"user");J(g,"aoStateLoaded",h.fnStateLoaded,"user");J(g,"aoRowCallback",h.fnRowCallback,"user");J(g,"aoRowCreatedCallback",h.fnCreatedRow,"user");J(g,"aoHeaderCallback",h.fnHeaderCallback,"user");J(g,"aoFooterCallback",
-h.fnFooterCallback,"user");J(g,"aoInitComplete",h.fnInitComplete,"user");J(g,"aoPreDrawCallback",h.fnPreDrawCallback,"user");if(g.oFeatures.bServerSide&&g.oFeatures.bSort&&g.oFeatures.bSortClasses)J(g,"aoDrawCallback",ba,"server_side_sort_classes");else g.oFeatures.bDeferRender&&J(g,"aoDrawCallback",ba,"defer_sort_classes");if(h.bJQueryUI){i.extend(g.oClasses,l.ext.oJUIClasses);if(h.sDom===l.defaults.sDom&&l.defaults.sDom==="lfrtip")g.sDom='<"H"lfr>t<"F"ip>'}else i.extend(g.oClasses,l.ext.oStdClasses);
-i(this).addClass(g.oClasses.sTable);if(g.oScroll.sX!==""||g.oScroll.sY!=="")g.oScroll.iBarWidth=eb();if(g.iInitDisplayStart===p){g.iInitDisplayStart=h.iDisplayStart;g._iDisplayStart=h.iDisplayStart}if(h.bStateSave){g.oFeatures.bStateSave=true;gb(g,h);J(g,"aoDrawCallback",Ha,"state_save")}if(h.iDeferLoading!==null){g.bDeferLoading=true;a=i.isArray(h.iDeferLoading);g._iRecordsDisplay=a?h.iDeferLoading[0]:h.iDeferLoading;g._iRecordsTotal=a?h.iDeferLoading[1]:h.iDeferLoading}if(h.aaData!==null)f=true;
-if(h.oLanguage.sUrl!==""){g.oLanguage.sUrl=h.oLanguage.sUrl;i.getJSON(g.oLanguage.sUrl,null,function(k){Fa(k);i.extend(true,g.oLanguage,h.oLanguage,k);ra(g)});e=true}else i.extend(true,g.oLanguage,h.oLanguage);if(h.asStripeClasses===null)g.asStripeClasses=[g.oClasses.sStripeOdd,g.oClasses.sStripeEven];b=g.asStripeClasses.length;g.asDestroyStripes=[];if(b){c=false;d=i(this).children("tbody").children("tr:lt("+b+")");for(a=0;a<b;a++)if(d.hasClass(g.asStripeClasses[a])){c=true;g.asDestroyStripes.push(g.asStripeClasses[a])}c&&
-d.removeClass(g.asStripeClasses.join(" "))}c=[];a=this.getElementsByTagName("thead");if(a.length!==0){ha(g.aoHeader,a[0]);c=Z(g)}if(h.aoColumns===null){d=[];a=0;for(b=c.length;a<b;a++)d.push(null)}else d=h.aoColumns;a=0;for(b=d.length;a<b;a++){if(h.saved_aoColumns!==p&&h.saved_aoColumns.length==b){if(d[a]===null)d[a]={};d[a].bVisible=h.saved_aoColumns[a].bVisible}n(g,c?c[a]:null)}ma(g,h.aoColumnDefs,d,function(k,m){q(g,k,m)});a=0;for(b=g.aaSorting.length;a<b;a++){if(g.aaSorting[a][0]>=g.aoColumns.length)g.aaSorting[a][0]=
-0;var j=g.aoColumns[g.aaSorting[a][0]];if(g.aaSorting[a][2]===p)g.aaSorting[a][2]=0;if(h.aaSorting===p&&g.saved_aaSorting===p)g.aaSorting[a][1]=j.asSorting[0];c=0;for(d=j.asSorting.length;c<d;c++)if(g.aaSorting[a][1]==j.asSorting[c]){g.aaSorting[a][2]=c;break}}ba(g);ib(g);a=i(this).children("caption").each(function(){this._captionSide=i(this).css("caption-side")});b=i(this).children("thead");if(b.length===0){b=[s.createElement("thead")];this.appendChild(b[0])}g.nTHead=b[0];b=i(this).children("tbody");
-if(b.length===0){b=[s.createElement("tbody")];this.appendChild(b[0])}g.nTBody=b[0];g.nTBody.setAttribute("role","alert");g.nTBody.setAttribute("aria-live","polite");g.nTBody.setAttribute("aria-relevant","all");b=i(this).children("tfoot");if(b.length===0&&a.length>0&&(g.oScroll.sX!==""||g.oScroll.sY!=="")){b=[s.createElement("tfoot")];this.appendChild(b[0])}if(b.length>0){g.nTFoot=b[0];ha(g.aoFooter,g.nTFoot)}if(f)for(a=0;a<h.aaData.length;a++)R(g,h.aaData[a]);else ea(g);g.aiDisplay=g.aiDisplayMaster.slice();
-g.bInitialised=true;e===false&&ra(g)}});sa=null;return this};l.fnVersionCheck=function(h){var n=function(A,G){for(;A.length<G;)A+="0";return A},q=l.ext.sVersion.split(".");h=h.split(".");for(var o="",v="",w=0,D=h.length;w<D;w++){o+=n(q[w],3);v+=n(h[w],3)}return parseInt(o,10)>=parseInt(v,10)};l.fnIsDataTable=function(h){for(var n=l.settings,q=0;q<n.length;q++)if(n[q].nTable===h||n[q].nScrollHead===h||n[q].nScrollFoot===h)return true;return false};l.fnTables=function(h){var n=[];jQuery.each(l.settings,
-function(q,o){if(!h||h===true&&i(o.nTable).is(":visible"))n.push(o.nTable)});return n};l.version="1.9.4";l.settings=[];l.models={};l.models.ext={afnFiltering:[],afnSortData:[],aoFeatures:[],aTypes:[],fnVersionCheck:l.fnVersionCheck,iApiIndex:0,ofnSearch:{},oApi:{},oStdClasses:{},oJUIClasses:{},oPagination:{},oSort:{},sVersion:l.version,sErrMode:"alert",_oExternConfig:{iNextUnique:0}};l.models.oSearch={bCaseInsensitive:true,sSearch:"",bRegex:false,bSmart:true};l.models.oRow={nTr:null,_aData:[],_aSortData:[],
-_anHidden:[],_sRowStripe:""};l.models.oColumn={aDataSort:null,asSorting:null,bSearchable:null,bSortable:null,bUseRendered:null,bVisible:null,_bAutoType:true,fnCreatedCell:null,fnGetData:null,fnRender:null,fnSetData:null,mData:null,mRender:null,nTh:null,nTf:null,sClass:null,sContentPadding:null,sDefaultContent:null,sName:null,sSortDataType:"std",sSortingClass:null,sSortingClassJUI:null,sTitle:null,sType:null,sWidth:null,sWidthOrig:null};l.defaults={aaData:null,aaSorting:[[0,"asc"]],aaSortingFixed:null,
-aLengthMenu:[10,25,50,100],aoColumns:null,aoColumnDefs:null,aoSearchCols:[],asStripeClasses:null,bAutoWidth:true,bDeferRender:false,bDestroy:false,bFilter:true,bInfo:true,bJQueryUI:false,bLengthChange:true,bPaginate:true,bProcessing:false,bRetrieve:false,bScrollAutoCss:true,bScrollCollapse:false,bScrollInfinite:false,bServerSide:false,bSort:true,bSortCellsTop:false,bSortClasses:true,bStateSave:false,fnCookieCallback:null,fnCreatedRow:null,fnDrawCallback:null,fnFooterCallback:null,fnFormatNumber:function(h){if(h<
-1E3)return h;var n=h+"";h=n.split("");var q="";n=n.length;for(var o=0;o<n;o++){if(o%3===0&&o!==0)q=this.oLanguage.sInfoThousands+q;q=h[n-o-1]+q}return q},fnHeaderCallback:null,fnInfoCallback:null,fnInitComplete:null,fnPreDrawCallback:null,fnRowCallback:null,fnServerData:function(h,n,q,o){o.jqXHR=i.ajax({url:h,data:n,success:function(v){v.sError&&o.oApi._fnLog(o,0,v.sError);i(o.oInstance).trigger("xhr",[o,v]);q(v)},dataType:"json",cache:false,type:o.sServerMethod,error:function(v,w){w=="parsererror"&&
-o.oApi._fnLog(o,0,"DataTables warning: JSON data from server could not be parsed. This is caused by a JSON formatting error.")}})},fnServerParams:null,fnStateLoad:function(h){h=this.oApi._fnReadCookie(h.sCookiePrefix+h.sInstance);var n;try{n=typeof i.parseJSON==="function"?i.parseJSON(h):eval("("+h+")")}catch(q){n=null}return n},fnStateLoadParams:null,fnStateLoaded:null,fnStateSave:function(h,n){this.oApi._fnCreateCookie(h.sCookiePrefix+h.sInstance,this.oApi._fnJsonString(n),h.iCookieDuration,h.sCookiePrefix,
-h.fnCookieCallback)},fnStateSaveParams:null,iCookieDuration:7200,iDeferLoading:null,iDisplayLength:10,iDisplayStart:0,iScrollLoadGap:100,iTabIndex:0,oLanguage:{oAria:{sSortAscending:": activate to sort column ascending",sSortDescending:": activate to sort column descending"},oPaginate:{sFirst:"First",sLast:"Last",sNext:"Next",sPrevious:"Previous"},sEmptyTable:"No data available in table",sInfo:"Showing _START_ to _END_ of _TOTAL_ entries",sInfoEmpty:"Showing 0 to 0 of 0 entries",sInfoFiltered:"(filtered from _MAX_ total entries)",
-sInfoPostFix:"",sInfoThousands:",",sLengthMenu:"Show _MENU_ entries",sLoadingRecords:"Loading...",sProcessing:"Processing...",sSearch:"Search:",sUrl:"",sZeroRecords:"No matching records found"},oSearch:i.extend({},l.models.oSearch),sAjaxDataProp:"aaData",sAjaxSource:null,sCookiePrefix:"SpryMedia_DataTables_",sDom:"lfrtip",sPaginationType:"two_button",sScrollX:"",sScrollXInner:"",sScrollY:"",sServerMethod:"GET"};l.defaults.columns={aDataSort:null,asSorting:["asc","desc"],bSearchable:true,bSortable:true,
-bUseRendered:true,bVisible:true,fnCreatedCell:null,fnRender:null,iDataSort:-1,mData:null,mRender:null,sCellType:"td",sClass:"",sContentPadding:"",sDefaultContent:null,sName:"",sSortDataType:"std",sTitle:null,sType:null,sWidth:null};l.models.oSettings={oFeatures:{bAutoWidth:null,bDeferRender:null,bFilter:null,bInfo:null,bLengthChange:null,bPaginate:null,bProcessing:null,bServerSide:null,bSort:null,bSortClasses:null,bStateSave:null},oScroll:{bAutoCss:null,bCollapse:null,bInfinite:null,iBarWidth:0,iLoadGap:null,
-sX:null,sXInner:null,sY:null},oLanguage:{fnInfoCallback:null},oBrowser:{bScrollOversize:false},aanFeatures:[],aoData:[],aiDisplay:[],aiDisplayMaster:[],aoColumns:[],aoHeader:[],aoFooter:[],asDataSearch:[],oPreviousSearch:{},aoPreSearchCols:[],aaSorting:null,aaSortingFixed:null,asStripeClasses:null,asDestroyStripes:[],sDestroyWidth:0,aoRowCallback:[],aoHeaderCallback:[],aoFooterCallback:[],aoDrawCallback:[],aoRowCreatedCallback:[],aoPreDrawCallback:[],aoInitComplete:[],aoStateSaveParams:[],aoStateLoadParams:[],
-aoStateLoaded:[],sTableId:"",nTable:null,nTHead:null,nTFoot:null,nTBody:null,nTableWrapper:null,bDeferLoading:false,bInitialised:false,aoOpenRows:[],sDom:null,sPaginationType:"two_button",iCookieDuration:0,sCookiePrefix:"",fnCookieCallback:null,aoStateSave:[],aoStateLoad:[],oLoadedState:null,sAjaxSource:null,sAjaxDataProp:null,bAjaxDataGet:true,jqXHR:null,fnServerData:null,aoServerParams:[],sServerMethod:null,fnFormatNumber:null,aLengthMenu:null,iDraw:0,bDrawing:false,iDrawError:-1,_iDisplayLength:10,
-_iDisplayStart:0,_iDisplayEnd:10,_iRecordsTotal:0,_iRecordsDisplay:0,bJUI:null,oClasses:{},bFiltered:false,bSorted:false,bSortCellsTop:null,oInit:null,aoDestroyCallback:[],fnRecordsTotal:function(){return this.oFeatures.bServerSide?parseInt(this._iRecordsTotal,10):this.aiDisplayMaster.length},fnRecordsDisplay:function(){return this.oFeatures.bServerSide?parseInt(this._iRecordsDisplay,10):this.aiDisplay.length},fnDisplayEnd:function(){return this.oFeatures.bServerSide?this.oFeatures.bPaginate===false||
-this._iDisplayLength==-1?this._iDisplayStart+this.aiDisplay.length:Math.min(this._iDisplayStart+this._iDisplayLength,this._iRecordsDisplay):this._iDisplayEnd},oInstance:null,sInstance:null,iTabIndex:0,nScrollHead:null,nScrollFoot:null};l.ext=i.extend(true,{},l.models.ext);i.extend(l.ext.oStdClasses,{sTable:"dataTable",sPagePrevEnabled:"paginate_enabled_previous",sPagePrevDisabled:"paginate_disabled_previous",sPageNextEnabled:"paginate_enabled_next",sPageNextDisabled:"paginate_disabled_next",sPageJUINext:"",
-sPageJUIPrev:"",sPageButton:"paginate_button",sPageButtonActive:"paginate_active",sPageButtonStaticDisabled:"paginate_button paginate_button_disabled",sPageFirst:"first",sPagePrevious:"previous",sPageNext:"next",sPageLast:"last",sStripeOdd:"odd",sStripeEven:"even",sRowEmpty:"dataTables_empty",sWrapper:"dataTables_wrapper",sFilter:"dataTables_filter",sInfo:"dataTables_info",sPaging:"dataTables_paginate paging_",sLength:"dataTables_length",sProcessing:"dataTables_processing",sSortAsc:"sorting_asc",
-sSortDesc:"sorting_desc",sSortable:"sorting",sSortableAsc:"sorting_asc_disabled",sSortableDesc:"sorting_desc_disabled",sSortableNone:"sorting_disabled",sSortColumn:"sorting_",sSortJUIAsc:"",sSortJUIDesc:"",sSortJUI:"",sSortJUIAscAllowed:"",sSortJUIDescAllowed:"",sSortJUIWrapper:"",sSortIcon:"",sScrollWrapper:"dataTables_scroll",sScrollHead:"dataTables_scrollHead",sScrollHeadInner:"dataTables_scrollHeadInner",sScrollBody:"dataTables_scrollBody",sScrollFoot:"dataTables_scrollFoot",sScrollFootInner:"dataTables_scrollFootInner",
-sFooterTH:"",sJUIHeader:"",sJUIFooter:""});i.extend(l.ext.oJUIClasses,l.ext.oStdClasses,{sPagePrevEnabled:"fg-button ui-button ui-state-default ui-corner-left",sPagePrevDisabled:"fg-button ui-button ui-state-default ui-corner-left ui-state-disabled",sPageNextEnabled:"fg-button ui-button ui-state-default ui-corner-right",sPageNextDisabled:"fg-button ui-button ui-state-default ui-corner-right ui-state-disabled",sPageJUINext:"ui-icon ui-icon-circle-arrow-e",sPageJUIPrev:"ui-icon ui-icon-circle-arrow-w",
-sPageButton:"fg-button ui-button ui-state-default",sPageButtonActive:"fg-button ui-button ui-state-default ui-state-disabled",sPageButtonStaticDisabled:"fg-button ui-button ui-state-default ui-state-disabled",sPageFirst:"first ui-corner-tl ui-corner-bl",sPageLast:"last ui-corner-tr ui-corner-br",sPaging:"dataTables_paginate fg-buttonset ui-buttonset fg-buttonset-multi ui-buttonset-multi paging_",sSortAsc:"ui-state-default",sSortDesc:"ui-state-default",sSortable:"ui-state-default",sSortableAsc:"ui-state-default",
-sSortableDesc:"ui-state-default",sSortableNone:"ui-state-default",sSortJUIAsc:"css_right ui-icon ui-icon-triangle-1-n",sSortJUIDesc:"css_right ui-icon ui-icon-triangle-1-s",sSortJUI:"css_right ui-icon ui-icon-carat-2-n-s",sSortJUIAscAllowed:"css_right ui-icon ui-icon-carat-1-n",sSortJUIDescAllowed:"css_right ui-icon ui-icon-carat-1-s",sSortJUIWrapper:"DataTables_sort_wrapper",sSortIcon:"DataTables_sort_icon",sScrollHead:"dataTables_scrollHead ui-state-default",sScrollFoot:"dataTables_scrollFoot ui-state-default",
-sFooterTH:"ui-state-default",sJUIHeader:"fg-toolbar ui-toolbar ui-widget-header ui-corner-tl ui-corner-tr ui-helper-clearfix",sJUIFooter:"fg-toolbar ui-toolbar ui-widget-header ui-corner-bl ui-corner-br ui-helper-clearfix"});i.extend(l.ext.oPagination,{two_button:{fnInit:function(h,n,q){var o=h.oLanguage.oPaginate,v=function(D){h.oApi._fnPageChange(h,D.data.action)&&q(h)};o=!h.bJUI?'<a class="'+h.oClasses.sPagePrevDisabled+'" tabindex="'+h.iTabIndex+'" role="button">'+o.sPrevious+'</a><a class="'+
-h.oClasses.sPageNextDisabled+'" tabindex="'+h.iTabIndex+'" role="button">'+o.sNext+"</a>":'<a class="'+h.oClasses.sPagePrevDisabled+'" tabindex="'+h.iTabIndex+'" role="button"><span class="'+h.oClasses.sPageJUIPrev+'"></span></a><a class="'+h.oClasses.sPageNextDisabled+'" tabindex="'+h.iTabIndex+'" role="button"><span class="'+h.oClasses.sPageJUINext+'"></span></a>';i(n).append(o);var w=i("a",n);o=w[0];w=w[1];h.oApi._fnBindAction(o,{action:"previous"},v);h.oApi._fnBindAction(w,{action:"next"},v);
-if(!h.aanFeatures.p){n.id=h.sTableId+"_paginate";o.id=h.sTableId+"_previous";w.id=h.sTableId+"_next";o.setAttribute("aria-controls",h.sTableId);w.setAttribute("aria-controls",h.sTableId)}},fnUpdate:function(h){if(h.aanFeatures.p)for(var n=h.oClasses,q=h.aanFeatures.p,o,v=0,w=q.length;v<w;v++)if(o=q[v].firstChild){o.className=h._iDisplayStart===0?n.sPagePrevDisabled:n.sPagePrevEnabled;o=o.nextSibling;o.className=h.fnDisplayEnd()==h.fnRecordsDisplay()?n.sPageNextDisabled:n.sPageNextEnabled}}},iFullNumbersShowPages:5,
-full_numbers:{fnInit:function(h,n,q){var o=h.oLanguage.oPaginate,v=h.oClasses,w=function(G){h.oApi._fnPageChange(h,G.data.action)&&q(h)};i(n).append('<a  tabindex="'+h.iTabIndex+'" class="'+v.sPageButton+" "+v.sPageFirst+'">'+o.sFirst+'</a><a  tabindex="'+h.iTabIndex+'" class="'+v.sPageButton+" "+v.sPagePrevious+'">'+o.sPrevious+'</a><span></span><a tabindex="'+h.iTabIndex+'" class="'+v.sPageButton+" "+v.sPageNext+'">'+o.sNext+'</a><a tabindex="'+h.iTabIndex+'" class="'+v.sPageButton+" "+v.sPageLast+
-'">'+o.sLast+"</a>");var D=i("a",n);o=D[0];v=D[1];var A=D[2];D=D[3];h.oApi._fnBindAction(o,{action:"first"},w);h.oApi._fnBindAction(v,{action:"previous"},w);h.oApi._fnBindAction(A,{action:"next"},w);h.oApi._fnBindAction(D,{action:"last"},w);if(!h.aanFeatures.p){n.id=h.sTableId+"_paginate";o.id=h.sTableId+"_first";v.id=h.sTableId+"_previous";A.id=h.sTableId+"_next";D.id=h.sTableId+"_last"}},fnUpdate:function(h,n){if(h.aanFeatures.p){var q=l.ext.oPagination.iFullNumbersShowPages,o=Math.floor(q/2),v=
-Math.ceil(h.fnRecordsDisplay()/h._iDisplayLength),w=Math.ceil(h._iDisplayStart/h._iDisplayLength)+1,D="",A,G=h.oClasses,E,Y=h.aanFeatures.p,ma=function(R){h.oApi._fnBindAction(this,{page:R+A-1},function(ea){h.oApi._fnPageChange(h,ea.data.page);n(h);ea.preventDefault()})};if(h._iDisplayLength===-1)w=o=A=1;else if(v<q){A=1;o=v}else if(w<=o){A=1;o=q}else if(w>=v-o){A=v-q+1;o=v}else{A=w-Math.ceil(q/2)+1;o=A+q-1}for(q=A;q<=o;q++)D+=w!==q?'<a tabindex="'+h.iTabIndex+'" class="'+G.sPageButton+'">'+h.fnFormatNumber(q)+
-"</a>":'<a tabindex="'+h.iTabIndex+'" class="'+G.sPageButtonActive+'">'+h.fnFormatNumber(q)+"</a>";q=0;for(o=Y.length;q<o;q++){E=Y[q];if(E.hasChildNodes()){i("span:eq(0)",E).html(D).children("a").each(ma);E=E.getElementsByTagName("a");E=[E[0],E[1],E[E.length-2],E[E.length-1]];i(E).removeClass(G.sPageButton+" "+G.sPageButtonActive+" "+G.sPageButtonStaticDisabled);i([E[0],E[1]]).addClass(w==1?G.sPageButtonStaticDisabled:G.sPageButton);i([E[2],E[3]]).addClass(v===0||w===v||h._iDisplayLength===-1?G.sPageButtonStaticDisabled:
-G.sPageButton)}}}}}});i.extend(l.ext.oSort,{"string-pre":function(h){if(typeof h!="string")h=h!==null&&h.toString?h.toString():"";return h.toLowerCase()},"string-asc":function(h,n){return h<n?-1:h>n?1:0},"string-desc":function(h,n){return h<n?1:h>n?-1:0},"html-pre":function(h){return h.replace(/<.*?>/g,"").toLowerCase()},"html-asc":function(h,n){return h<n?-1:h>n?1:0},"html-desc":function(h,n){return h<n?1:h>n?-1:0},"date-pre":function(h){h=Date.parse(h);if(isNaN(h)||h==="")h=Date.parse("01/01/1970 00:00:00");
-return h},"date-asc":function(h,n){return h-n},"date-desc":function(h,n){return n-h},"numeric-pre":function(h){return h=="-"||h===""?0:h*1},"numeric-asc":function(h,n){return h-n},"numeric-desc":function(h,n){return n-h}});i.extend(l.ext.aTypes,[function(h){if(typeof h==="number")return"numeric";else if(typeof h!=="string")return null;var n,q=false;n=h.charAt(0);if("0123456789-".indexOf(n)==-1)return null;for(var o=1;o<h.length;o++){n=h.charAt(o);if("0123456789.".indexOf(n)==-1)return null;if(n==
-"."){if(q)return null;q=true}}return"numeric"},function(h){var n=Date.parse(h);if(n!==null&&!isNaN(n)||typeof h==="string"&&h.length===0)return"date";return null},function(h){if(typeof h==="string"&&h.indexOf("<")!=-1&&h.indexOf(">")!=-1)return"html";return null}]);i.fn.DataTable=l;i.fn.dataTable=l;i.fn.dataTableSettings=l.settings;i.fn.dataTableExt=l.ext})})(window,document);


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


[04/50] [abbrv] hadoop git commit: Make 3.1.1 awared by other branches

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49c68760/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_3.1.1.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_3.1.1.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_3.1.1.xml
new file mode 100644
index 0000000..e3dbe6a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_3.1.1.xml
@@ -0,0 +1,2920 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Thu Aug 02 05:12:52 UTC 2018 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop YARN Client 3.1.1"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/hadoop-annotations.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/jdiff.jar -verbose -classpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/classes:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-3.1.1.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/commons-codec/commons-codec/1.11/commons-codec-1.11.jar:/maven/commons-io/commons-io/2.5/commons-io-2.5.jar:/maven/commons-net/commons-net/3.6/commons-net-3.6.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/maven/javax/servlet/javax.servlet-api/3.1.0/javax.
 servlet-api-3.1.0.jar:/maven/org/eclipse/jetty/jetty-server/9.3.19.v20170502/jetty-server-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-http/9.3.19.v20170502/jetty-http-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-io/9.3.19.v20170502/jetty-io-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-util/9.3.19.v20170502/jetty-util-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-servlet/9.3.19.v20170502/jetty-servlet-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-security/9.3.19.v20170502/jetty-security-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-webapp/9.3.19.v20170502/jetty-webapp-9.3.19.v20170502.jar:/maven/org/eclipse/jetty/jetty-xml/9.3.19.v20170502/jetty-xml-9.3.19.v20170502.jar:/maven/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/com/sun/jersey/jersey-core/1.19/jersey-core-1.19.jar:/maven/javax/ws/rs/jsr311-api/1.1.1/jsr311-api-1.1.1.jar:/maven/com/sun/jersey/jersey-servlet/1.19/jersey-servlet-1.19.jar:/maven/com/sun/jersey/jersey-json/1.19/jers
 ey-json-1.19.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/maven/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/com/sun/jersey/jersey-server/1.19/jersey-server-1.19.jar:/maven/commons-beanutils/commons-beanutils/1.9.3/commons-beanutils-1.9.3.jar:/maven/org/apache/commons/commons-configuration2/2.1.1/commons-configuration2-2.1.1.jar:/maven/org/apache/commons/commons-lang3/3.4/commons-lang3-3.4.jar:/maven/org/slf4j/slf4j-api/1.7.25/slf4j-api-1.7.25.jar:/maven/org/slf4j/slf4j-log4j12/1.7.25/slf4j-log4j12-1.7.25.jar:/maven/org/apache/avro/avro/1.7.7/avro-1.7.7.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy-java/1.0.5/snappy-java-1.0.5.jar:/maven/com/google
 /re2j/re2j/1.1/re2j-1.1.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-3.1.1.jar:/maven/com/nimbusds/nimbus-jose-jwt/4.41.1/nimbus-jose-jwt-4.41.1.jar:/maven/com/github/stephenc/jcip/jcip-annotations/1.0-1/jcip-annotations-1.0-1.jar:/maven/net/minidev/json-smart/2.3/json-smart-2.3.jar:/maven/net/minidev/accessors-smart/1.2/accessors-smart-1.2.jar:/maven/org/ow2/asm/asm/5.0.4/asm-5.0.4.jar:/maven/org/apache/curator/curator-framework/2.12.0/curator-framework-2.12.0.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.12.0/curator-client-2.12.0.jar:/maven/org/apache/curator/curator-recipes/2.12.0/curator-recipes-2.12.0.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/htrace/htrace-core4/4.1.0-incubating/htrace-core4-4.1.0-incubating.jar:/maven/org/apache/zookeeper/z
 ookeeper/3.4.9/zookeeper-3.4.9.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/maven/org/apache/kerby/kerb-simplekdc/1.0.1/kerb-simplekdc-1.0.1.jar:/maven/org/apache/kerby/kerb-client/1.0.1/kerb-client-1.0.1.jar:/maven/org/apache/kerby/kerby-config/1.0.1/kerby-config-1.0.1.jar:/maven/org/apache/kerby/kerb-core/1.0.1/kerb-core-1.0.1.jar:/maven/org/apache/kerby/kerby-pkix/1.0.1/kerby-pkix-1.0.1.jar:/maven/org/apache/kerby/kerby-asn1/1.0.1/kerby-asn1-1.0.1.jar:/maven/org/apache/kerby/kerby-util/1.0.1/kerby-util-1.0.1.jar:/maven/org/apache/kerby/kerb-common/1.0.1/kerb-common-1.0.1.jar:/maven/org/apache/kerby/kerb-crypto/1.0.1/kerb-crypto-1.0.1.jar:/maven/org/apache/kerby/kerb-util/1.0.1/kerb-util-1.0.1.jar:/maven/org/apache/kerby/token-provider/1.0.1/token-provider-1.0.1.jar:/maven/org/apache/kerby/kerb-admin/1.0.1/kerb-admin-1.0.1.jar:/maven/org/apache/kerby/kerb-server/1.0.1/kerb-server-1.0.1.jar:/maven/org/apache/k
 erby/kerb-identity/1.0.1/kerb-identity-1.0.1.jar:/maven/org/apache/kerby/kerby-xdr/1.0.1/kerby-xdr-1.0.1.jar:/maven/com/fasterxml/jackson/core/jackson-databind/2.7.8/jackson-databind-2.7.8.jar:/maven/org/codehaus/woodstox/stax2-api/3.1.4/stax2-api-3.1.4.jar:/maven/com/fasterxml/woodstox/woodstox-core/5.0.3/woodstox-core-5.0.3.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-3.1.1.jar:/usr/lib/jvm/java-8-openjdk-amd64/lib/tools.jar:/maven/io/netty/netty/3.10.5.Final/netty-3.10.5.Final.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-3.1.1.jar:/maven/javax/xml/bind/jaxb-api/2.2.11/jaxb-api-2.2.11.jar:/maven/com/fasterxml/jackson/core/
 jackson-annotations/2.7.8/jackson-annotations-2.7.8.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-3.1.1.jar:/maven/com/sun/jersey/jersey-client/1.19/jersey-client-1.19.jar:/maven/com/google/inject/extensions/guice-servlet/4.0/guice-servlet-4.0.jar:/maven/com/google/inject/guice/4.0/guice-4.0.jar:/maven/javax/inject/javax.inject/1/javax.inject-1.jar:/maven/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/maven/com/sun/jersey/contribs/jersey-guice/1.19/jersey-guice-1.19.jar:/maven/com/fasterxml/jackson/core/jackson-core/2.7.8/jackson-core-2.7.8.jar:/maven/com/fasterxml/jackson/module/jackson-module-jaxb-annotations/2.7.8/jackson-module-jaxb-annotations-2.7.8.jar:/maven/com/fasterxml/jackson/jaxrs/jackson-jaxrs-json-provider/2.7.8/jackson-jaxrs-json-provider-2.7.8.jar:/maven/com/fasterxml/jackson/jaxrs/jackson-jaxrs-base/2.7.8/jackson-jaxrs-base-2.7.8.jar:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/maven/xerces/xercesImpl
 /2.11.0/xercesImpl-2.11.0.jar:/maven/xml-apis/xml-apis/1.4.01/xml-apis-1.4.01.jar -sourcepath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/hadoop-annotations.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/jdiff.jar -apidir /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/site/jdiff/xml -apiname Apache Hadoop YARN Client 3.1.1 -->
+<package name="org.apache.hadoop.yarn.client">
+</package>
+<package name="org.apache.hadoop.yarn.client.api">
+  <!-- start class org.apache.hadoop.yarn.client.api.AHSClient -->
+  <class name="AHSClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AHSClient" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAHSClient" return="org.apache.hadoop.yarn.client.api.AHSClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of AHSClient.]]>
+      </doc>
+    </method>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get a report of the given Application.
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+   <li>host - set to "N/A"</li>
+   <li>RPC port - set to -1</li>
+   <li>client token - set to "N/A"</li>
+   <li>diagnostics - set to "N/A"</li>
+   <li>tracking URL - set to "N/A"</li>
+   <li>original tracking URL - set to "N/A"</li>
+   <li>resource usage report - all values are -1</li>
+ </ul>
+ 
+ @param appId
+          {@link ApplicationId} of the application that needs a report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of all Applications in the cluster.
+ </p>
+ 
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+ 
+ @return a list of reports for all applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given ApplicationAttempt.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the application attempt that needs
+          a report
+ @return application attempt report
+ @throws YarnException
+ @throws ApplicationAttemptNotFoundException if application attempt
+         not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (ApplicationAttempts) of Application in the cluster.
+ </p>
+ 
+ @param applicationId
+ @return a list of reports for all application attempts for specified
+         application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Container.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ @param containerId
+          {@link ContainerId} of the container that needs a report
+ @return container report
+ @throws YarnException
+ @throws ContainerNotFoundException if container not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (Containers) of ApplicationAttempt in the cluster.
+ </p>
+ 
+ @param applicationAttemptId
+ @return a list of reports of all containers for specified application
+         attempt
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AHSClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.AMRMClient -->
+  <class name="AMRMClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAMRMClient" return="org.apache.hadoop.yarn.client.api.AMRMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of AMRMClient.
+ For usage:
+ <pre>
+ {@code
+ AMRMClient.<T>createAMRMClientContainerRequest()
+ }</pre>
+ @return the newly create AMRMClient instance.]]>
+      </doc>
+    </method>
+    <method name="addSchedulingRequests"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="schedulingRequests" type="java.util.Collection"/>
+      <doc>
+      <![CDATA[Add a Collection of SchedulingRequests. The AMRMClient will ensure that
+ all requests in the same batch are sent in the same allocate call.
+ @param schedulingRequests Collection of Scheduling Requests.]]>
+      </doc>
+    </method>
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appHostName" type="java.lang.String"/>
+      <param name="appHostPort" type="int"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Register the application master. This must be called before any 
+ other interaction
+ @param appHostName Name of the host on which master is running
+ @param appHostPort Port master is listening on
+ @param appTrackingUrl URL at which the master info can be seen
+ @return <code>RegisterApplicationMasterResponse</code>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appHostName" type="java.lang.String"/>
+      <param name="appHostPort" type="int"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <param name="placementConstraints" type="java.util.Map"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Register the application master. This must be called before any
+ other interaction
+ @param appHostName Name of the host on which master is running
+ @param appHostPort Port master is listening on
+ @param appTrackingUrl URL at which the master info can be seen
+ @param placementConstraints Placement Constraints mappings.
+ @return <code>RegisterApplicationMasterResponse</code>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="allocate" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="progressIndicator" type="float"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Request additional containers and receive new container allocations.
+ Requests made via <code>addContainerRequest</code> are sent to the
+ <code>ResourceManager</code>. New containers assigned to the master are
+ retrieved. Status of completed containers and node health updates are also
+ retrieved. This also doubles up as a heartbeat to the ResourceManager and
+ must be made periodically. The call may not always return any new
+ allocations of containers. App should not make concurrent allocate
+ requests. May cause request loss.
+ 
+ <p>
+ Note : If the user has not removed container requests that have already
+ been satisfied, then the re-register may end up sending the entire
+ container requests to the RM (including matched requests). Which would mean
+ the RM could end up giving it a lot of new allocated containers.
+ </p>
+ 
+ @param progressIndicator Indicates progress made by the master
+ @return the response of the allocate request
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="unregisterApplicationMaster"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="appMessage" type="java.lang.String"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Unregister the application master. This must be called in the end.
+ @param appStatus Success/Failure status of the master
+ @param appMessage Diagnostics message on failure
+ @param appTrackingUrl New URL to get master info
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="addContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Request containers for resources before calling <code>allocate</code>
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="removeContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Remove previous container request. The previous container request may have 
+ already been sent to the ResourceManager. So even after the remove request 
+ the app must be prepared to receive an allocation for the previous request 
+ even after the remove request
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="requestContainerResourceChange"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="use
+ {@link #requestContainerUpdate(Container, UpdateContainerRequest)}">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <doc>
+      <![CDATA[Request container resource change before calling <code>allocate</code>.
+ Any previous pending resource change request of the same container will be
+ removed.
+
+ Application that calls this method is expected to maintain the
+ <code>Container</code>s that are returned from previous successful
+ allocations or resource changes. By passing in the existing container and a
+ target resource capability to this method, the application requests the
+ ResourceManager to change the existing resource allocation to the target
+ resource allocation.
+
+ @deprecated use
+ {@link #requestContainerUpdate(Container, UpdateContainerRequest)}
+
+ @param container The container returned from the last successful resource
+                  allocation or resource change
+ @param capability  The target resource capability of the container]]>
+      </doc>
+    </method>
+    <method name="requestContainerUpdate"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <param name="updateContainerRequest" type="org.apache.hadoop.yarn.api.records.UpdateContainerRequest"/>
+      <doc>
+      <![CDATA[Request a container update before calling <code>allocate</code>.
+ Any previous pending update request of the same container will be
+ removed.
+
+ @param container The container returned from the last successful resource
+                  allocation or update
+ @param updateContainerRequest The <code>UpdateContainerRequest</code>.]]>
+      </doc>
+    </method>
+    <method name="releaseAssignedContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <doc>
+      <![CDATA[Release containers assigned by the Resource Manager. If the app cannot use
+ the container or wants to give up the container then it can release them.
+ The app needs to make new requests for the released resource capability if
+ it still needs it. eg. it released non-local resources
+ @param containerId]]>
+      </doc>
+    </method>
+    <method name="getAvailableResources" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the currently available resources in the cluster.
+ A valid value is available after a call to allocate has been made
+ @return Currently available resources]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeCount" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the current number of nodes in the cluster.
+ A valid values is available after a call to allocate has been made
+ @return Current number of nodes in the cluster]]>
+      </doc>
+    </method>
+    <method name="getMatchingRequests" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="resourceName" type="java.lang.String"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <doc>
+      <![CDATA[Get outstanding <code>ContainerRequest</code>s matching the given 
+ parameters. These ContainerRequests should have been added via
+ <code>addContainerRequest</code> earlier in the lifecycle. For performance,
+ the AMRMClient may return its internal collection directly without creating 
+ a copy. Users should not perform mutable operations on the return value.
+ Each collection in the list contains requests with identical 
+ <code>Resource</code> size that fit in the given capability. In a 
+ collection, requests will be returned in the same order as they were added.
+
+ NOTE: This API only matches Container requests that were created by the
+ client WITHOUT the allocationRequestId being set.
+
+ @return Collection of request matching the parameters]]>
+      </doc>
+    </method>
+    <method name="getMatchingRequests" return="java.util.List"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="resourceName" type="java.lang.String"/>
+      <param name="executionType" type="org.apache.hadoop.yarn.api.records.ExecutionType"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <doc>
+      <![CDATA[Get outstanding <code>ContainerRequest</code>s matching the given
+ parameters. These ContainerRequests should have been added via
+ <code>addContainerRequest</code> earlier in the lifecycle. For performance,
+ the AMRMClient may return its internal collection directly without creating
+ a copy. Users should not perform mutable operations on the return value.
+ Each collection in the list contains requests with identical
+ <code>Resource</code> size that fit in the given capability. In a
+ collection, requests will be returned in the same order as they were added.
+ specify an <code>ExecutionType</code>.
+
+ NOTE: This API only matches Container requests that were created by the
+ client WITHOUT the allocationRequestId being set.
+
+ @param priority Priority
+ @param resourceName Location
+ @param executionType ExecutionType
+ @param capability Capability
+ @return Collection of request matching the parameters]]>
+      </doc>
+    </method>
+    <method name="getMatchingRequests" return="java.util.List"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="resourceName" type="java.lang.String"/>
+      <param name="executionType" type="org.apache.hadoop.yarn.api.records.ExecutionType"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="profile" type="java.lang.String"/>
+    </method>
+    <method name="getMatchingRequests" return="java.util.Collection"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="allocationRequestId" type="long"/>
+      <doc>
+      <![CDATA[Get outstanding <code>ContainerRequest</code>s matching the given
+ allocationRequestId. These ContainerRequests should have been added via
+ <code>addContainerRequest</code> earlier in the lifecycle. For performance,
+ the AMRMClient may return its internal collection directly without creating
+ a copy. Users should not perform mutable operations on the return value.
+
+ NOTE: This API only matches Container requests that were created by the
+ client WITH the allocationRequestId being set to a non-default value.
+
+ @param allocationRequestId Allocation Request Id
+ @return Collection of request matching the parameters]]>
+      </doc>
+    </method>
+    <method name="updateBlacklist"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="blacklistAdditions" type="java.util.List"/>
+      <param name="blacklistRemovals" type="java.util.List"/>
+      <doc>
+      <![CDATA[Update application's blacklist with addition or removal resources.
+ 
+ @param blacklistAdditions list of resources which should be added to the 
+        application blacklist
+ @param blacklistRemovals list of resources which should be removed from the 
+        application blacklist]]>
+      </doc>
+    </method>
+    <method name="setNMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nmTokenCache" type="org.apache.hadoop.yarn.client.api.NMTokenCache"/>
+      <doc>
+      <![CDATA[Set the NM token cache for the <code>AMRMClient</code>. This cache must
+ be shared with the {@link NMClient} used to manage containers for the
+ <code>AMRMClient</code>
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @param nmTokenCache the NM token cache to use.]]>
+      </doc>
+    </method>
+    <method name="getNMTokenCache" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the NM token cache of the <code>AMRMClient</code>. This cache must be
+ shared with the {@link NMClient} used to manage containers for the
+ <code>AMRMClient</code>.
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @return the NM token cache.]]>
+      </doc>
+    </method>
+    <method name="registerTimelineV2Client"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="client" type="org.apache.hadoop.yarn.client.api.TimelineV2Client"/>
+      <doc>
+      <![CDATA[Register TimelineV2Client to AMRMClient. Writer's address for the timeline
+ V2 client will be updated dynamically if registered.
+
+ @param client the timeline v2 client to register]]>
+      </doc>
+    </method>
+    <method name="getRegisteredTimelineV2Client" return="org.apache.hadoop.yarn.client.api.TimelineV2Client"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get registered timeline v2 client.
+ @return the registered timeline v2 client]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="java.util.function.Supplier"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each 1000 ms.
+ See also {@link #waitFor(java.util.function.Supplier, int)}
+ and {@link #waitFor(java.util.function.Supplier, int, int)}
+ @param check the condition for which it should wait]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="java.util.function.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms.
+ See also {@link #waitFor(java.util.function.Supplier, int, int)}
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="java.util.function.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <param name="logInterval" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms. In the main loop, this method will log
+ the message "waiting in main loop" for each <code>logInterval</code> times
+ iteration to confirm the thread is alive.
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>
+ @param logInterval interval to log for each]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AMRMClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.NMClient -->
+  <class name="NMClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createNMClient" return="org.apache.hadoop.yarn.client.api.NMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of NMClient.]]>
+      </doc>
+    </method>
+    <method name="createNMClient" return="org.apache.hadoop.yarn.client.api.NMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Create a new instance of NMClient.]]>
+      </doc>
+    </method>
+    <method name="startContainer" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <param name="containerLaunchContext" type="org.apache.hadoop.yarn.api.records.ContainerLaunchContext"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Start an allocated container.</p>
+
+ <p>The <code>ApplicationMaster</code> or other applications that use the
+ client must provide the details of the allocated container, including the
+ Id, the assigned node's Id and the token via {@link Container}. In
+ addition, the AM needs to provide the {@link ContainerLaunchContext} as
+ well.</p>
+
+ @param container the allocated container
+ @param containerLaunchContext the context information needed by the
+                               <code>NodeManager</code> to launch the
+                               container
+ @return a map between the auxiliary service names and their outputs
+ @throws YarnException YarnException.
+ @throws IOException IOException.]]>
+      </doc>
+    </method>
+    <method name="increaseContainerResource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Increase the resource of a container.</p>
+
+ <p>The <code>ApplicationMaster</code> or other applications that use the
+ client must provide the details of the container, including the Id and
+ the target resource encapsulated in the updated container token via
+ {@link Container}.
+ </p>
+
+ @param container the container with updated token.
+
+ @throws YarnException YarnException.
+ @throws IOException IOException.]]>
+      </doc>
+    </method>
+    <method name="updateContainerResource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Update the resources of a container.</p>
+
+ <p>The <code>ApplicationMaster</code> or other applications that use the
+ client must provide the details of the container, including the Id and
+ the target resource encapsulated in the updated container token via
+ {@link Container}.
+ </p>
+
+ @param container the container with updated token.
+
+ @throws YarnException YarnException.
+ @throws IOException IOException.]]>
+      </doc>
+    </method>
+    <method name="stopContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Stop an started container.</p>
+
+ @param containerId the Id of the started container
+ @param nodeId the Id of the <code>NodeManager</code>
+
+ @throws YarnException YarnException.
+ @throws IOException IOException.]]>
+      </doc>
+    </method>
+    <method name="getContainerStatus" return="org.apache.hadoop.yarn.api.records.ContainerStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Query the status of a container.</p>
+
+ @param containerId the Id of the started container
+ @param nodeId the Id of the <code>NodeManager</code>
+ 
+ @return the status of a container.
+
+ @throws YarnException YarnException.
+ @throws IOException IOException.]]>
+      </doc>
+    </method>
+    <method name="reInitializeContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="containerLaunchContex" type="org.apache.hadoop.yarn.api.records.ContainerLaunchContext"/>
+      <param name="autoCommit" type="boolean"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Re-Initialize the Container.</p>
+
+ @param containerId the Id of the container to Re-Initialize.
+ @param containerLaunchContex the updated ContainerLaunchContext.
+ @param autoCommit commit re-initialization automatically ?
+
+ @throws YarnException YarnException.
+ @throws IOException IOException.]]>
+      </doc>
+    </method>
+    <method name="restartContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Restart the specified container.</p>
+
+ @param containerId the Id of the container to restart.
+
+ @throws YarnException YarnException.
+ @throws IOException IOException.]]>
+      </doc>
+    </method>
+    <method name="rollbackLastReInitialization"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Rollback last reInitialization of the specified container.</p>
+
+ @param containerId the Id of the container to restart.
+
+ @throws YarnException YarnException.
+ @throws IOException IOException.]]>
+      </doc>
+    </method>
+    <method name="commitLastReInitialization"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Commit last reInitialization of the specified container.</p>
+
+ @param containerId the Id of the container to commit reInitialize.
+
+ @throws YarnException YarnException.
+ @throws IOException IOException.]]>
+      </doc>
+    </method>
+    <method name="cleanupRunningContainersOnStop"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="enabled" type="boolean"/>
+      <doc>
+      <![CDATA[<p>Set whether the containers that are started by this client, and are
+ still running should be stopped when the client stops. By default, the
+ feature should be enabled.</p> However, containers will be stopped only  
+ when service is stopped. i.e. after {@link NMClient#stop()}. 
+
+ @param enabled whether the feature is enabled or not]]>
+      </doc>
+    </method>
+    <method name="setNMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nmTokenCache" type="org.apache.hadoop.yarn.client.api.NMTokenCache"/>
+      <doc>
+      <![CDATA[Set the NM Token cache of the <code>NMClient</code>. This cache must be
+ shared with the {@link AMRMClient} that requested the containers managed
+ by this <code>NMClient</code>
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @param nmTokenCache the NM token cache to use.]]>
+      </doc>
+    </method>
+    <method name="getNMTokenCache" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the NM token cache of the <code>NMClient</code>. This cache must be
+ shared with the {@link AMRMClient} that requested the containers managed
+ by this <code>NMClient</code>
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @return the NM token cache]]>
+      </doc>
+    </method>
+    <method name="getNodeIdOfStartedContainer" return="org.apache.hadoop.yarn.api.records.NodeId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <doc>
+      <![CDATA[Get the NodeId of the node on which container is running. It returns
+ null if the container if container is not found or if it is not running.
+
+ @param containerId Container Id of the container.
+ @return NodeId of the container on which it is running.]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.NMClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.NMTokenCache -->
+  <class name="NMTokenCache" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMTokenCache"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Creates a NM token cache instance.]]>
+      </doc>
+    </constructor>
+    <method name="getSingleton" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Returns the singleton NM token cache.
+
+ @return the singleton NM token cache.]]>
+      </doc>
+    </method>
+    <method name="getNMToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Returns NMToken, null if absent. Only the singleton obtained from
+ {@link #getSingleton()} is looked at for the tokens. If you are using your
+ own NMTokenCache that is different from the singleton, use
+ {@link #getToken(String) }
+ 
+ @param nodeAddr
+ @return {@link Token} NMToken required for communicating with node manager]]>
+      </doc>
+    </method>
+    <method name="setNMToken"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <param name="token" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <doc>
+      <![CDATA[Sets the NMToken for node address only in the singleton obtained from
+ {@link #getSingleton()}. If you are using your own NMTokenCache that is
+ different from the singleton, use {@link #setToken(String, Token) }
+ 
+ @param nodeAddr
+          node address (host:port)
+ @param token
+          NMToken]]>
+      </doc>
+    </method>
+    <method name="getToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Returns NMToken, null if absent
+ @param nodeAddr
+ @return {@link Token} NMToken required for communicating with node
+         manager]]>
+      </doc>
+    </method>
+    <method name="setToken"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <param name="token" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <doc>
+      <![CDATA[Sets the NMToken for node address
+ @param nodeAddr node address (host:port)
+ @param token NMToken]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[NMTokenCache manages NMTokens required for an Application Master
+ communicating with individual NodeManagers.
+ <p>
+ By default YARN client libraries {@link AMRMClient} and {@link NMClient} use
+ {@link #getSingleton()} instance of the cache.
+ <ul>
+   <li>
+     Using the singleton instance of the cache is appropriate when running a
+     single ApplicationMaster in the same JVM.
+   </li>
+   <li>
+     When using the singleton, users don't need to do anything special,
+     {@link AMRMClient} and {@link NMClient} are already set up to use the
+     default singleton {@link NMTokenCache}
+     </li>
+ </ul>
+ If running multiple Application Masters in the same JVM, a different cache
+ instance should be used for each Application Master.
+ <ul>
+   <li>
+     If using the {@link AMRMClient} and the {@link NMClient}, setting up
+     and using an instance cache is as follows:
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   AMRMClient rmClient = AMRMClient.createAMRMClient();
+   NMClient nmClient = NMClient.createNMClient();
+   nmClient.setNMTokenCache(nmTokenCache);
+   ...
+ </pre>
+   </li>
+   <li>
+     If using the {@link AMRMClientAsync} and the {@link NMClientAsync},
+     setting up and using an instance cache is as follows:
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   AMRMClient rmClient = AMRMClient.createAMRMClient();
+   NMClient nmClient = NMClient.createNMClient();
+   nmClient.setNMTokenCache(nmTokenCache);
+   AMRMClientAsync rmClientAsync = new AMRMClientAsync(rmClient, 1000, [AMRM_CALLBACK]);
+   NMClientAsync nmClientAsync = new NMClientAsync("nmClient", nmClient, [NM_CALLBACK]);
+   ...
+ </pre>
+   </li>
+   <li>
+     If using {@link ApplicationMasterProtocol} and
+     {@link ContainerManagementProtocol} directly, setting up and using an
+     instance cache is as follows:
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   ...
+   ApplicationMasterProtocol amPro = ClientRMProxy.createRMProxy(conf, ApplicationMasterProtocol.class);
+   ...
+   AllocateRequest allocateRequest = ...
+   ...
+   AllocateResponse allocateResponse = rmClient.allocate(allocateRequest);
+   for (NMToken token : allocateResponse.getNMTokens()) {
+     nmTokenCache.setToken(token.getNodeId().toString(), token.getToken());
+   }
+   ...
+   ContainerManagementProtocolProxy nmPro = ContainerManagementProtocolProxy(conf, nmTokenCache);
+   ...
+   nmPro.startContainer(container, containerContext);
+   ...
+ </pre>
+   </li>
+ </ul>
+ It is also possible to mix the usage of a client ({@code AMRMClient} or
+ {@code NMClient}, or the async versions of them) with a protocol proxy
+ ({@code ContainerManagementProtocolProxy} or
+ {@code ApplicationMasterProtocol}).]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.NMTokenCache -->
+  <!-- start class org.apache.hadoop.yarn.client.api.SharedCacheClient -->
+  <class name="SharedCacheClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="SharedCacheClient" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createSharedCacheClient" return="org.apache.hadoop.yarn.client.api.SharedCacheClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="use" return="org.apache.hadoop.yarn.api.records.URL"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="resourceKey" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ The method to claim a resource with the <code>SharedCacheManager.</code>
+ The client uses a checksum to identify the resource and an
+ {@link ApplicationId} to identify which application will be using the
+ resource.
+ </p>
+
+ <p>
+ The <code>SharedCacheManager</code> responds with whether or not the
+ resource exists in the cache. If the resource exists, a <code>URL</code> to
+ the resource in the shared cache is returned. If the resource does not
+ exist, null is returned instead.
+ </p>
+
+ <p>
+ Once a URL has been returned for a resource, that URL is safe to use for
+ the lifetime of the application that corresponds to the provided
+ ApplicationId.
+ </p>
+
+ @param applicationId ApplicationId of the application using the resource
+ @param resourceKey the key (i.e. checksum) that identifies the resource
+ @return URL to the resource, or null if it does not exist]]>
+      </doc>
+    </method>
+    <method name="release"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="resourceKey" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ The method to release a resource with the <code>SharedCacheManager.</code>
+ This method is called once an application is no longer using a claimed
+ resource in the shared cache. The client uses a checksum to identify the
+ resource and an {@link ApplicationId} to identify which application is
+ releasing the resource.
+ </p>
+ 
+ <p>
+ Note: This method is an optimization and the client is not required to call
+ it for correctness.
+ </p>
+ 
+ @param applicationId ApplicationId of the application releasing the
+          resource
+ @param resourceKey the key (i.e. checksum) that identifies the resource]]>
+      </doc>
+    </method>
+    <method name="getFileChecksum" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="sourceFile" type="org.apache.hadoop.fs.Path"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[A convenience method to calculate the checksum of a specified file.
+ 
+ @param sourceFile A path to the input file
+ @return A hex string containing the checksum digest
+ @throws IOException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[This is the client for YARN's shared cache.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.SharedCacheClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.YarnClient -->
+  <class name="YarnClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="YarnClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createYarnClient" return="org.apache.hadoop.yarn.client.api.YarnClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of YarnClient.]]>
+      </doc>
+    </method>
+    <method name="createApplication" return="org.apache.hadoop.yarn.client.api.YarnClientApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Obtain a {@link YarnClientApplication} for a new application,
+ which in turn contains the {@link ApplicationSubmissionContext} and
+ {@link org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse}
+ objects.
+ </p>
+
+ @return {@link YarnClientApplication} built for a new application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="submitApplication" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appContext" type="org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Submit a new application to <code>YARN.</code> It is a blocking call - it
+ will not return {@link ApplicationId} until the submitted application is
+ submitted successfully and accepted by the ResourceManager.
+ </p>
+ 
+ <p>
+ Users should provide an {@link ApplicationId} as part of the parameter
+ {@link ApplicationSubmissionContext} when submitting a new application,
+ otherwise it will throw the {@link ApplicationIdNotProvidedException}.
+ </p>
+
+ <p>This internally calls {@link ApplicationClientProtocol#submitApplication
+ (SubmitApplicationRequest)}, and after that, it internally invokes
+ {@link ApplicationClientProtocol#getApplicationReport
+ (GetApplicationReportRequest)} and waits till it can make sure that the
+ application gets properly submitted. If RM fails over or RM restart
+ happens before ResourceManager saves the application's state,
+ {@link ApplicationClientProtocol
+ #getApplicationReport(GetApplicationReportRequest)} will throw
+ the {@link ApplicationNotFoundException}. This API automatically resubmits
+ the application with the same {@link ApplicationSubmissionContext} when it
+ catches the {@link ApplicationNotFoundException}</p>
+
+ @param appContext
+          {@link ApplicationSubmissionContext} containing all the details
+          needed to submit a new application
+ @return {@link ApplicationId} of the accepted application
+ @throws YarnException
+ @throws IOException
+ @see #createApplication()]]>
+      </doc>
+    </method>
+    <method name="failApplicationAttempt"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Fail an application attempt identified by given ID.
+ </p>
+
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the attempt to fail.
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException
+ @see #getQueueAclsInfo()]]>
+      </doc>
+    </method>
+    <method name="killApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Kill an application identified by given ID.
+ </p>
+ 
+ @param applicationId
+          {@link ApplicationId} of the application that needs to be killed
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException
+ @see #getQueueAclsInfo()]]>
+      </doc>
+    </method>
+    <method name="killApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="diagnostics" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Kill an application identified by given ID.
+ </p>
+ @param applicationId {@link ApplicationId} of the application that needs to
+          be killed
+ @param diagnostics for killing an application.
+ @throws YarnException in case of errors or if YARN rejects the request due
+           to access-control restrictions.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Application.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+ <li>host - set to "N/A"</li>
+ <li>RPC port - set to -1</li>
+ <li>client token - set to "N/A"</li>
+ <li>diagnostics - set to "N/A"</li>
+ <li>tracking URL - set to "N/A"</li>
+ <li>original tracking URL - set to "N/A"</li>
+ <li>resource usage report - all values are -1</li>
+ </ul>
+ 
+ @param appId
+          {@link ApplicationId} of the application that needs a report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAMRMToken" return="org.apache.hadoop.security.token.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get the AMRM token of the application.
+ <p>
+ The AMRM token is required for AM to RM scheduling operations. For 
+ managed Application Masters YARN takes care of injecting it. For unmanaged
+ Applications Masters, the token must be obtained via this method and set
+ in the {@link org.apache.hadoop.security.UserGroupInformation} of the
+ current user.
+ <p>
+ The AMRM token will be returned only if all the following conditions are
+ met:
+ <ul>
+   <li>the requester is the owner of the ApplicationMaster</li>
+   <li>the application master is an unmanaged ApplicationMaster</li>
+   <li>the application master is in ACCEPTED state</li>
+ </ul>
+ Else this method returns NULL.
+
+ @param appId {@link ApplicationId} of the application to get the AMRM token
+ @return the AMRM token if available
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of all Applications in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @return a list of reports of all running applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationTypes" type="java.util.Set"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications
+ matching the given application types in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationTypes set of application types you are interested in
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given
+ application states in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationStates set of application states you are interested in
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationTypes" type="java.util.Set"/>
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given
+ application types and application states in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationTypes set of application types you are interested in
+ @param applicationStates set of application states you are interested in
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationTypes" type="java.util.Set"/>
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <param name="applicationTags" type="java.util.Set"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given
+ application types, application states and application tags in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationTypes set of application types you are interested in
+ @param applicationStates set of application states you are interested in
+ @param applicationTags set of application tags you are interested in
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="queues" type="java.util.Set"/>
+      <param name="users" type="java.util.Set"/>
+      <param name="applicationTypes" type="java.util.Set"/>
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given users,
+ queues, application types and application states in the cluster. If any of
+ the params is set to null, it is not used when filtering.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param queues set of queues you are interested in
+ @param users set of users you are interested in
+ @param applicationTypes set of application types you are interested in
+ @param applicationStates set of application states you are interested in
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a list of ApplicationReports that match the given
+ {@link GetApplicationsRequest}.
+</p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param request the request object to get the list of applications.
+ @return The list of ApplicationReports that match the request
+ @throws YarnException Exception specific to YARN.
+ @throws IOException Exception mostly related to connection errors.]]>
+      </doc>
+    </method>
+    <method name="getYarnClusterMetrics" return="org.apache.hadoop.yarn.api.records.YarnClusterMetrics"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get metrics ({@link YarnClusterMetrics}) about the cluster.
+ </p>
+ 
+ @return cluster metrics
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getNodeReports" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="states" type="org.apache.hadoop.yarn.api.records.NodeState[]"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of nodes ({@link NodeReport}) in the cluster.
+ </p>
+ 
+ @param states The {@link NodeState}s to filter on. If no filter states are
+          given, nodes in all states will be returned.
+ @return A list of node reports
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRMDelegationToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="renewer" type="org.apache.hadoop.io.Text"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a delegation token so as to be able to talk to YARN using those tokens.
+ 
+ @param renewer
+          Address of the renewer who can renew these tokens when needed by
+          securely talking to YARN.
+ @return a delegation token ({@link Token}) that can be used to
+         talk to YARN
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueInfo" return="org.apache.hadoop.yarn.api.records.QueueInfo"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="queueName" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about a given <em>queue</em>.
+ </p>
+ 
+ @param queueName
+          Name of the queue whose information is needed
+ @return queue information
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAllQueues" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about all queues, recursively if there
+ is a hierarchy
+ </p>
+ 
+ @return a list of queue-information for all queues
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRootQueueInfos" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about top level queues.
+ </p>
+ 
+ @return a list of queue-information for all the top-level queues
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getChildQueueInfos" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="parent" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about all the immediate children queues
+ of the given queue
+ </p>
+ 
+ @param parent
+          Name of the queue whose child-queues' information is needed
+ @return a list of queue-information for all queues who are direct children
+         of the given parent queue.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueAclsInfo" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information about <em>acls</em> for <em>current user</em> on all the
+ existing queues.
+ </p>
+ 
+ @return a list of queue acls ({@link QueueUserACLInfo}) for
+         <em>current user</em>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given ApplicationAttempt.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the application attempt that needs
+          a report
+ @return application attempt report
+ @throws YarnException
+ @throws ApplicationAttemptNotFoundException if application attempt
+         not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (ApplicationAttempts) of Application in the cluster.
+ </p>
+ 
+ @param applicationId application id of the app
+ @return a list of reports for all application attempts for specified
+         application.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Container.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ @param containerId
+          {@link ContainerId} of the container that needs a report
+ @return container report
+ @throws YarnException
+ @throws ContainerNotFoundException if container not found.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (Containers) of ApplicationAttempt in the cluster.
+ </p>
+ 
+ @param applicationAttemptId application attempt id
+ @return a list of reports of all containers for specified application
+         attempts
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="moveApplicationAcrossQueues"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="queue" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Attempts to move the given application to the given queue.
+ </p>
+ 
+ @param appId
+    Application to move.
+ @param queue
+    Queue to place it in to.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="createReservation" return="org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Obtain a {@link GetNewReservationResponse} for a new reservation,
+ which contains the {@link ReservationId} object.
+ </p>
+
+ @return The {@link GetNewReservationResponse} containing a new
+         {@link ReservationId} object.
+ @throws YarnException if reservation cannot be created.
+ @throws IOException if reservation cannot be created.]]>
+      </doc>
+    </method>
+    <method name="submitReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to submit a new reservation to the
+ {@code ResourceManager}.
+ </p>
+ 
+ <p>
+ The client packages all details of its request in a
+ {@link ReservationSubmissionRequest} object. This contains information
+ about the amount of capacity, temporal constraints, and gang needs.
+ Furthermore, the reservation might be composed of multiple stages, with
+ ordering dependencies among them.
+ </p>
+ 
+ <p>
+ In order to respond, a new admission control component in the
+ {@code ResourceManager} performs an analysis of the resources that have
+ been committed over the period of time the user is requesting, verify that
+ the user requests can be fulfilled, and that it respect a sharing policy
+ (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined
+ that the ReservationRequest is satisfiable the {@code ResourceManager}
+ answers with a {@link ReservationSubmissionResponse} that includes a
+ {@link ReservationId}. Upon failure to find a valid allocation the response
+ is an exception with the message detailing the reason of failure.
+ </p>
+ 
+ <p>
+ The semantics guarantees that the {@link ReservationId} returned,
+ corresponds to a valid reservation existing in the time-range request by
+ the user. The amount of capacity dedicated to such reservation can vary
+ overtime, depending of the allocation that has been determined. But it is
+ guaranteed to satisfy all the constraint expressed by the user in the
+ {@link ReservationDefinition}
+ </p>
+ 
+ @param request request to submit a new Reservation
+ @return response contains the {@link ReservationId} on accepting the
+         submission
+ @throws YarnException if the reservation cannot be created successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="updateReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to update an existing Reservation. This is
+ referred to as a re-negotiation process, in which a user that has
+ previously submitted a Reservation.
+ </p>
+ 
+ <p>
+ The allocation is attempted by virtually substituting all previous
+ allocations related to this Reservation with new ones, that satisfy the new
+ {@link ReservationDefinition}. Upon success the previous allocation is
+ atomically substituted by the new one, and on failure (i.e., if the system
+ cannot find a valid allocation for the updated request), the previous
+ allocation remains valid.
+ </p>
+ 
+ @param request to update an existing Reservation (the
+          {@link ReservationUpdateRequest} should refer to an existing valid
+          {@link ReservationId})
+ @return response empty on successfully updating the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           updated successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="deleteReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse"
+      abstract="true" native="false" sy

<TRUNCATED>

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


[42/50] [abbrv] hadoop git commit: HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java
new file mode 100644
index 0000000..dd59892
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java
@@ -0,0 +1,507 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import javax.crypto.Mac;
+import javax.crypto.spec.SecretKeySpec;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.URLDecoder;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TimeZone;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.Charsets;
+/**
+ * Represents the shared key credentials used to access an Azure Storage
+ * account.
+ */
+public class SharedKeyCredentials {
+  private static final int EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH = 300;
+  private static final Pattern CRLF = Pattern.compile("\r\n", Pattern.LITERAL);
+  private static final String HMAC_SHA256 = "HmacSHA256";
+  private static final Base64 BASE_64 = new Base64();
+
+  /**
+   * Stores a reference to the RFC1123 date/time pattern.
+   */
+  private static final String RFC1123_PATTERN = "EEE, dd MMM yyyy HH:mm:ss z";
+
+
+  private String accountName;
+  private byte[] accountKey;
+  private Mac hmacSha256;
+
+  public SharedKeyCredentials(final String accountName,
+                              final String accountKey) {
+    if (accountName == null || accountName.isEmpty()) {
+      throw new IllegalArgumentException("Invalid account name.");
+    }
+    if (accountKey == null || accountKey.isEmpty()) {
+      throw new IllegalArgumentException("Invalid account key.");
+    }
+    this.accountName = accountName;
+    this.accountKey = BASE_64.decode(accountKey);
+    initializeMac();
+  }
+
+  public void signRequest(HttpURLConnection connection, final long contentLength) throws UnsupportedEncodingException {
+
+    connection.setRequestProperty(HttpHeaderConfigurations.X_MS_DATE, getGMTTime());
+
+    final String stringToSign = canonicalize(connection, accountName, contentLength);
+
+    final String computedBase64Signature = computeHmac256(stringToSign);
+
+    connection.setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
+        String.format("%s %s:%s", "SharedKey", accountName, computedBase64Signature));
+  }
+
+  private String computeHmac256(final String stringToSign) {
+    byte[] utf8Bytes = null;
+    try {
+      utf8Bytes = stringToSign.getBytes(AbfsHttpConstants.UTF_8);
+    } catch (final UnsupportedEncodingException e) {
+      throw new IllegalArgumentException(e);
+    }
+    byte[] hmac;
+    synchronized (this) {
+      hmac = hmacSha256.doFinal(utf8Bytes);
+    }
+    return new String(BASE_64.encode(hmac), Charsets.UTF_8);
+  }
+
+  /**
+   * Add x-ms- prefixed headers in a fixed order.
+   *
+   * @param conn                the HttpURLConnection for the operation
+   * @param canonicalizedString the canonicalized string to add the canonicalized headerst to.
+   */
+  private static void addCanonicalizedHeaders(final HttpURLConnection conn, final StringBuilder canonicalizedString) {
+    // Look for header names that start with
+    // HeaderNames.PrefixForStorageHeader
+    // Then sort them in case-insensitive manner.
+
+    final Map<String, List<String>> headers = conn.getRequestProperties();
+    final ArrayList<String> httpStorageHeaderNameArray = new ArrayList<String>();
+
+    for (final String key : headers.keySet()) {
+      if (key.toLowerCase(Locale.ROOT).startsWith(AbfsHttpConstants.HTTP_HEADER_PREFIX)) {
+        httpStorageHeaderNameArray.add(key.toLowerCase(Locale.ROOT));
+      }
+    }
+
+    Collections.sort(httpStorageHeaderNameArray);
+
+    // Now go through each header's values in the sorted order and append
+    // them to the canonicalized string.
+    for (final String key : httpStorageHeaderNameArray) {
+      final StringBuilder canonicalizedElement = new StringBuilder(key);
+      String delimiter = ":";
+      final ArrayList<String> values = getHeaderValues(headers, key);
+
+      boolean appendCanonicalizedElement = false;
+      // Go through values, unfold them, and then append them to the
+      // canonicalized element string.
+      for (final String value : values) {
+        if (value != null) {
+          appendCanonicalizedElement = true;
+        }
+
+        // Unfolding is simply removal of CRLF.
+        final String unfoldedValue = CRLF.matcher(value)
+            .replaceAll(Matcher.quoteReplacement(""));
+
+        // Append it to the canonicalized element string.
+        canonicalizedElement.append(delimiter);
+        canonicalizedElement.append(unfoldedValue);
+        delimiter = ",";
+      }
+
+      // Now, add this canonicalized element to the canonicalized header
+      // string.
+      if (appendCanonicalizedElement) {
+        appendCanonicalizedElement(canonicalizedString, canonicalizedElement.toString());
+      }
+    }
+  }
+
+  /**
+   * Initialie the HmacSha256 associated with the account key.
+   */
+  private void initializeMac() {
+    // Initializes the HMAC-SHA256 Mac and SecretKey.
+    try {
+      hmacSha256 = Mac.getInstance(HMAC_SHA256);
+      hmacSha256.init(new SecretKeySpec(accountKey, HMAC_SHA256));
+    } catch (final Exception e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * Append a string to a string builder with a newline constant
+   *
+   * @param builder the StringBuilder object
+   * @param element the string to append.
+   */
+  private static void appendCanonicalizedElement(final StringBuilder builder, final String element) {
+    builder.append("\n");
+    builder.append(element);
+  }
+
+  /**
+   * Constructs a canonicalized string from the request's headers that will be used to construct the signature string
+   * for signing a Blob or Queue service request under the Shared Key Full authentication scheme.
+   *
+   * @param address       the request URI
+   * @param accountName   the account name associated with the request
+   * @param method        the verb to be used for the HTTP request.
+   * @param contentType   the content type of the HTTP request.
+   * @param contentLength the length of the content written to the outputstream in bytes, -1 if unknown
+   * @param date          the date/time specification for the HTTP request
+   * @param conn          the HttpURLConnection for the operation.
+   * @return A canonicalized string.
+   */
+  private static String canonicalizeHttpRequest(final java.net.URL address, final String accountName,
+                                                final String method, final String contentType, final long contentLength, final String date,
+                                                final HttpURLConnection conn) throws UnsupportedEncodingException {
+
+    // The first element should be the Method of the request.
+    // I.e. GET, POST, PUT, or HEAD.
+    final StringBuilder canonicalizedString = new StringBuilder(EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH);
+    canonicalizedString.append(conn.getRequestMethod());
+
+    // The next elements are
+    // If any element is missing it may be empty.
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_ENCODING, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_LANGUAGE, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        contentLength <= 0 ? "" : String.valueOf(contentLength));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_MD5, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString, contentType != null ? contentType : AbfsHttpConstants.EMPTY_STRING);
+
+    final String dateString = getHeaderValue(conn, HttpHeaderConfigurations.X_MS_DATE, AbfsHttpConstants.EMPTY_STRING);
+    // If x-ms-date header exists, Date should be empty string
+    appendCanonicalizedElement(canonicalizedString, dateString.equals(AbfsHttpConstants.EMPTY_STRING) ? date
+        : "");
+
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.IF_MODIFIED_SINCE, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.IF_MATCH, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.IF_UNMODIFIED_SINCE, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.RANGE, AbfsHttpConstants.EMPTY_STRING));
+
+    addCanonicalizedHeaders(conn, canonicalizedString);
+
+    appendCanonicalizedElement(canonicalizedString, getCanonicalizedResource(address, accountName));
+
+    return canonicalizedString.toString();
+  }
+
+  /**
+   * Gets the canonicalized resource string for a Blob or Queue service request under the Shared Key Lite
+   * authentication scheme.
+   *
+   * @param address     the resource URI.
+   * @param accountName the account name for the request.
+   * @return the canonicalized resource string.
+   */
+  private static String getCanonicalizedResource(final java.net.URL address, final String accountName) throws UnsupportedEncodingException {
+    // Resource path
+    final StringBuilder resourcepath = new StringBuilder(AbfsHttpConstants.FORWARD_SLASH);
+    resourcepath.append(accountName);
+
+    // Note that AbsolutePath starts with a '/'.
+    resourcepath.append(address.getPath());
+    final StringBuilder canonicalizedResource = new StringBuilder(resourcepath.toString());
+
+    // query parameters
+    if (address.getQuery() == null || !address.getQuery().contains(AbfsHttpConstants.EQUAL)) {
+      //no query params.
+      return canonicalizedResource.toString();
+    }
+
+    final Map<String, String[]> queryVariables = parseQueryString(address.getQuery());
+
+    final Map<String, String> lowercasedKeyNameValue = new HashMap<String, String>();
+
+    for (final Entry<String, String[]> entry : queryVariables.entrySet()) {
+      // sort the value and organize it as comma separated values
+      final List<String> sortedValues = Arrays.asList(entry.getValue());
+      Collections.sort(sortedValues);
+
+      final StringBuilder stringValue = new StringBuilder();
+
+      for (final String value : sortedValues) {
+        if (stringValue.length() > 0) {
+          stringValue.append(AbfsHttpConstants.COMMA);
+        }
+
+        stringValue.append(value);
+      }
+
+      // key turns out to be null for ?a&b&c&d
+      lowercasedKeyNameValue.put((entry.getKey()) == null ? null
+          : entry.getKey().toLowerCase(Locale.ROOT), stringValue.toString());
+    }
+
+    final ArrayList<String> sortedKeys = new ArrayList<String>(lowercasedKeyNameValue.keySet());
+
+    Collections.sort(sortedKeys);
+
+    for (final String key : sortedKeys) {
+      final StringBuilder queryParamString = new StringBuilder();
+
+      queryParamString.append(key);
+      queryParamString.append(":");
+      queryParamString.append(lowercasedKeyNameValue.get(key));
+
+      appendCanonicalizedElement(canonicalizedResource, queryParamString.toString());
+    }
+
+    return canonicalizedResource.toString();
+  }
+
+  /**
+   * Gets all the values for the given header in the one to many map, performs a trimStart() on each return value
+   *
+   * @param headers    a one to many map of key / values representing the header values for the connection.
+   * @param headerName the name of the header to lookup
+   * @return an ArrayList<String> of all trimmed values corresponding to the requested headerName. This may be empty
+   * if the header is not found.
+   */
+  private static ArrayList<String> getHeaderValues(final Map<String, List<String>> headers, final String headerName) {
+
+    final ArrayList<String> arrayOfValues = new ArrayList<String>();
+    List<String> values = null;
+
+    for (final Entry<String, List<String>> entry : headers.entrySet()) {
+      if (entry.getKey().toLowerCase(Locale.ROOT).equals(headerName)) {
+        values = entry.getValue();
+        break;
+      }
+    }
+    if (values != null) {
+      for (final String value : values) {
+        // canonicalization formula requires the string to be left
+        // trimmed.
+        arrayOfValues.add(trimStart(value));
+      }
+    }
+    return arrayOfValues;
+  }
+
+  /**
+   * Parses a query string into a one to many hashmap.
+   *
+   * @param parseString the string to parse
+   * @return a HashMap<String, String[]> of the key values.
+   */
+  private static HashMap<String, String[]> parseQueryString(String parseString) throws UnsupportedEncodingException {
+    final HashMap<String, String[]> retVals = new HashMap<String, String[]>();
+    if (parseString == null || parseString.isEmpty()) {
+      return retVals;
+    }
+
+    // 1. Remove ? if present
+    final int queryDex = parseString.indexOf(AbfsHttpConstants.QUESTION_MARK);
+    if (queryDex >= 0 && parseString.length() > 0) {
+      parseString = parseString.substring(queryDex + 1);
+    }
+
+    // 2. split name value pairs by splitting on the 'c&' character
+    final String[] valuePairs = parseString.contains(AbfsHttpConstants.AND_MARK)
+            ? parseString.split(AbfsHttpConstants.AND_MARK)
+            : parseString.split(AbfsHttpConstants.SEMICOLON);
+
+    // 3. for each field value pair parse into appropriate map entries
+    for (int m = 0; m < valuePairs.length; m++) {
+      final int equalDex = valuePairs[m].indexOf(AbfsHttpConstants.EQUAL);
+
+      if (equalDex < 0 || equalDex == valuePairs[m].length() - 1) {
+        continue;
+      }
+
+      String key = valuePairs[m].substring(0, equalDex);
+      String value = valuePairs[m].substring(equalDex + 1);
+
+      key = safeDecode(key);
+      value = safeDecode(value);
+
+      // 3.1 add to map
+      String[] values = retVals.get(key);
+
+      if (values == null) {
+        values = new String[]{value};
+        if (!value.equals("")) {
+          retVals.put(key, values);
+        }
+      }
+    }
+
+    return retVals;
+  }
+
+  /**
+   * Performs safe decoding of the specified string, taking care to preserve each <code>+</code> character, rather
+   * than replacing it with a space character.
+   *
+   * @param stringToDecode A <code>String</code> that represents the string to decode.
+   * @return A <code>String</code> that represents the decoded string.
+   * <p>
+   * If a storage service error occurred.
+   */
+  private static String safeDecode(final String stringToDecode) throws UnsupportedEncodingException {
+    if (stringToDecode == null) {
+      return null;
+    }
+
+    if (stringToDecode.length() == 0) {
+      return "";
+    }
+
+    if (stringToDecode.contains(AbfsHttpConstants.PLUS)) {
+      final StringBuilder outBuilder = new StringBuilder();
+
+      int startDex = 0;
+      for (int m = 0; m < stringToDecode.length(); m++) {
+        if (stringToDecode.charAt(m) == '+') {
+          if (m > startDex) {
+            outBuilder.append(URLDecoder.decode(stringToDecode.substring(startDex, m),
+                    AbfsHttpConstants.UTF_8));
+          }
+
+          outBuilder.append(AbfsHttpConstants.PLUS);
+          startDex = m + 1;
+        }
+      }
+
+      if (startDex != stringToDecode.length()) {
+        outBuilder.append(URLDecoder.decode(stringToDecode.substring(startDex, stringToDecode.length()),
+                AbfsHttpConstants.UTF_8));
+      }
+
+      return outBuilder.toString();
+    } else {
+      return URLDecoder.decode(stringToDecode, AbfsHttpConstants.UTF_8);
+    }
+  }
+
+  private static String trimStart(final String value) {
+    int spaceDex = 0;
+    while (spaceDex < value.length() && value.charAt(spaceDex) == ' ') {
+      spaceDex++;
+    }
+
+    return value.substring(spaceDex);
+  }
+
+  private static String getHeaderValue(final HttpURLConnection conn, final String headerName, final String defaultValue) {
+    final String headerValue = conn.getRequestProperty(headerName);
+    return headerValue == null ? defaultValue : headerValue;
+  }
+
+
+  /**
+   * Constructs a canonicalized string for signing a request.
+   *
+   * @param conn          the HttpURLConnection to canonicalize
+   * @param accountName   the account name associated with the request
+   * @param contentLength the length of the content written to the outputstream in bytes,
+   *                      -1 if unknown
+   * @return a canonicalized string.
+   */
+  private String canonicalize(final HttpURLConnection conn,
+                              final String accountName,
+                              final Long contentLength) throws UnsupportedEncodingException {
+
+    if (contentLength < -1) {
+      throw new IllegalArgumentException(
+          "The Content-Length header must be greater than or equal to -1.");
+    }
+
+    String contentType = getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_TYPE, "");
+
+    return canonicalizeHttpRequest(conn.getURL(), accountName,
+        conn.getRequestMethod(), contentType, contentLength, null, conn);
+  }
+
+  /**
+   * Thread local for storing GMT date format.
+   */
+  private static ThreadLocal<DateFormat> rfc1123GmtDateTimeFormatter
+      = new ThreadLocal<DateFormat>() {
+    @Override
+    protected DateFormat initialValue() {
+      final DateFormat formatter = new SimpleDateFormat(RFC1123_PATTERN, Locale.ROOT);
+      formatter.setTimeZone(GMT_ZONE);
+      return formatter;
+    }
+  };
+
+  public static final TimeZone GMT_ZONE = TimeZone.getTimeZone(AbfsHttpConstants.GMT_TIMEZONE);
+
+
+  /**
+   * Returns the current GMT date/time String using the RFC1123 pattern.
+   *
+   * @return A <code>String</code> that represents the current GMT date/time using the RFC1123 pattern.
+   */
+  static String getGMTTime() {
+    return getGMTTime(new Date());
+  }
+
+  /**
+   * Returns the GTM date/time String for the specified value using the RFC1123 pattern.
+   *
+   * @param date
+   *            A <code>Date</code> object that represents the date to convert to GMT date/time in the RFC1123
+   *            pattern.
+   *
+   * @return A <code>String</code> that represents the GMT date/time for the specified value using the RFC1123
+   *         pattern.
+   */
+  static String getGMTTime(final Date date) {
+    return rfc1123GmtDateTimeFormatter.get().format(date);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java
new file mode 100644
index 0000000..57b6463
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.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.azurebfs.services;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Objects;
+
+import com.google.common.base.Preconditions;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
+import org.apache.htrace.core.HTraceConfiguration;
+import org.apache.htrace.core.Sampler;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Singleton
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+final class TracingServiceImpl implements TracingService {
+  private static final Logger LOG = LoggerFactory.getLogger(TracingService.class);
+
+  private final Tracer tracer;
+  private final ThreadLocal<SpanId> currentScopeId;
+
+  @Inject
+  TracingServiceImpl(
+      final Configuration configuration) {
+    Preconditions.checkNotNull(configuration, "configuration");
+
+    this.currentScopeId = new ThreadLocal<>();
+
+    this.tracer = new Tracer.Builder(TracingService.class.getSimpleName()).
+        conf(new HTraceConfiguration() {
+          @Override
+          public String get(String key) {
+            if (Objects.equals(key, Tracer.SPAN_RECEIVER_CLASSES_KEY)) {
+              return LoggerSpanReceiver.class.getName();
+            }
+            return null;
+          }
+
+          @Override
+          public String get(String key, String defaultValue) {
+            String value = get(key);
+            if (value != null) {
+              return value;
+            }
+            return defaultValue;
+          }
+        }).
+        build();
+
+    this.tracer.addSampler(Sampler.ALWAYS);
+  }
+
+  @Override
+  public TraceScope traceBegin(String description) {
+    if (this.LOG.isTraceEnabled()) {
+      TraceScope traceScope = this.tracer.newScope(description);
+      this.currentScopeId.set(traceScope.getSpanId());
+      return traceScope;
+    }
+
+    return null;
+  }
+
+  @Override
+  public TraceScope traceBegin(String description, SpanId parentSpanId) {
+    if (this.LOG.isTraceEnabled()) {
+      TraceScope traceScope = this.tracer.newScope(description, parentSpanId);
+      this.currentScopeId.set(traceScope.getSpanId());
+      return traceScope;
+    }
+
+    return null;
+  }
+
+  @Override
+  public void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException) {
+    if (this.LOG.isTraceEnabled()) {
+      Preconditions.checkNotNull(traceScope, "traceScope");
+      Preconditions.checkNotNull(azureBlobFileSystemException, "azureBlobFileSystemException");
+
+      StringWriter stringWriter = new StringWriter();
+      PrintWriter printWriter = new PrintWriter(stringWriter);
+      azureBlobFileSystemException.printStackTrace(printWriter);
+      printWriter.flush();
+
+      traceScope.addKVAnnotation("Exception", stringWriter.toString());
+    }
+  }
+
+  @Override
+  public SpanId getCurrentTraceScopeSpanId() {
+    return this.currentScopeId.get();
+  }
+
+  @Override
+  public void traceEnd(TraceScope traceScope) {
+    if (this.LOG.isTraceEnabled()) {
+      Preconditions.checkNotNull(traceScope, "traceScope");
+
+      SpanId[] parents = traceScope.getSpan().getParents();
+      this.currentScopeId.set(parents != null && parents.length > 0 ? parents[parents.length - 1] : null);
+      traceScope.close();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java
new file mode 100644
index 0000000..97c1d71
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.services;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java
new file mode 100644
index 0000000..7652adf
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java
@@ -0,0 +1,77 @@
+/**
+ * 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.azurebfs.utils;
+
+import java.util.regex.Pattern;
+
+/**
+ * Utility class to help with Abfs url transformation to blob urls.
+ */
+public final class UriUtils {
+  private static final String ABFS_URI_REGEX = "[^.]+\\.dfs\\.(preprod\\.){0,1}core\\.windows\\.net";
+  private static final Pattern ABFS_URI_PATTERN = Pattern.compile(ABFS_URI_REGEX);
+
+  /**
+   * Checks whether a string includes abfs url.
+   * @param string the string to check.
+   * @return true if string has abfs url.
+   */
+  public static boolean containsAbfsUrl(final String string) {
+    if (string == null || string.isEmpty()) {
+      return false;
+    }
+
+    return ABFS_URI_PATTERN.matcher(string).matches();
+  }
+
+  /**
+   * Extracts the raw account name from account name.
+   * @param accountName to extract the raw account name.
+   * @return extracted raw account name.
+   */
+  public static String extractRawAccountFromAccountName(final String accountName) {
+    if (accountName == null || accountName.isEmpty()) {
+      return null;
+    }
+
+    if (!containsAbfsUrl(accountName)) {
+      return null;
+    }
+
+    String[] splitByDot = accountName.split("\\.");
+    if (splitByDot.length == 0) {
+      return null;
+    }
+
+    return splitByDot[0];
+  }
+
+  /**
+   * Generate unique test path for multiple user tests.
+   *
+   * @return root test path
+   */
+  public static String generateUniqueTestPath() {
+    String testUniqueForkId = System.getProperty("test.unique.fork.id");
+    return testUniqueForkId == null ? "/test" : "/" + testUniqueForkId + "/test";
+  }
+
+  private UriUtils() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java
new file mode 100644
index 0000000..d8cc940
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.utils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
new file mode 100644
index 0000000..5ec1e2e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
@@ -0,0 +1,225 @@
+/**
+ * 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.azurebfs;
+
+import java.net.URI;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.mockito.internal.util.MockUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
+import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+import org.apache.hadoop.fs.azurebfs.services.MockAbfsHttpClientFactoryImpl;
+import org.apache.hadoop.fs.azurebfs.services.MockAbfsServiceInjectorImpl;
+import org.apache.hadoop.fs.azurebfs.services.MockServiceProviderImpl;
+
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assume.assumeNotNull;
+
+/**
+ * Provide dependencies for AzureBlobFileSystem tests.
+ */
+public abstract class DependencyInjectedTest {
+  private final MockAbfsServiceInjectorImpl mockServiceInjector;
+  private final boolean isEmulator;
+  private NativeAzureFileSystem wasb;
+  private String abfsScheme;
+
+  private Configuration configuration;
+  private String fileSystemName;
+  private String accountName;
+  private String testUrl;
+
+  public DependencyInjectedTest(final boolean secure) {
+    this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME);
+  }
+
+  public MockAbfsServiceInjectorImpl getMockServiceInjector() {
+    return this.mockServiceInjector;
+  }
+
+  protected DependencyInjectedTest() {
+    this(FileSystemUriSchemes.ABFS_SCHEME);
+  }
+
+  private DependencyInjectedTest(final String scheme) {
+    abfsScheme = scheme;
+    fileSystemName = UUID.randomUUID().toString();
+    configuration = new Configuration();
+    configuration.addResource("azure-bfs-test.xml");
+
+    assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME));
+    assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + configuration.get(TestConfigurationKeys
+        .FS_AZURE_TEST_ACCOUNT_NAME)));
+
+    final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
+    URI defaultUri = null;
+
+    try {
+      defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
+    } catch (Exception ex) {
+      Assert.fail(ex.getMessage());
+    }
+
+    this.testUrl = defaultUri.toString();
+    configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
+    configuration.setBoolean(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    this.mockServiceInjector = new MockAbfsServiceInjectorImpl(configuration);
+    this.isEmulator = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
+    this.accountName = this.configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
+  }
+
+  @Before
+  public void initialize() throws Exception {
+    if (this.isEmulator) {
+      this.mockServiceInjector.replaceProvider(AbfsHttpClientFactory.class, MockAbfsHttpClientFactoryImpl.class);
+    }
+
+    MockServiceProviderImpl.create(this.mockServiceInjector);
+
+    if (!this.isEmulator) {
+      final URI wasbUri = new URI(abfsUrlToWasbUrl(this.getTestUrl()));
+      final AzureNativeFileSystemStore azureNativeFileSystemStore = new AzureNativeFileSystemStore();
+      azureNativeFileSystemStore.initialize(
+          wasbUri,
+          this.getConfiguration(),
+          new AzureFileSystemInstrumentation(this.getConfiguration()));
+
+      this.wasb = new NativeAzureFileSystem(azureNativeFileSystemStore);
+      this.wasb.initialize(wasbUri, configuration);
+    }
+  }
+
+  @After
+  public void testCleanup() throws Exception {
+    if (this.wasb != null) {
+      this.wasb.close();
+    }
+
+    FileSystem.closeAll();
+
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final AbfsHttpService abfsHttpService = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class);
+    abfsHttpService.deleteFilesystem(fs);
+
+    if (!(new MockUtil().isMock(abfsHttpService))) {
+      AbfsRestOperationException ex = intercept(
+          AbfsRestOperationException.class,
+          new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+              abfsHttpService.getFilesystemProperties(fs);
+              return null;
+            }
+          });
+
+      assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode());
+    }
+  }
+
+  public AzureBlobFileSystem getFileSystem() throws Exception {
+    final Configuration configuration = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getConfiguration();
+    final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration);
+    return fs;
+  }
+
+  protected NativeAzureFileSystem getWasbFileSystem() {
+    return this.wasb;
+  }
+
+  protected String getHostName() {
+    return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME);
+  }
+
+  protected void updateTestUrl(String testUrl) {
+    this.testUrl = testUrl;
+  }
+  protected String getTestUrl() {
+    return testUrl;
+  }
+
+  protected void updateFileSystemName(String fileSystemName) {
+    this.fileSystemName = fileSystemName;
+  }
+  protected String getFileSystemName() {
+    return fileSystemName;
+  }
+
+  protected String getAccountName() {
+    return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
+  }
+
+  protected String getAccountKey() {
+    return configuration.get(
+        TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
+            + getAccountName());
+  }
+
+  protected Configuration getConfiguration() {
+    return this.configuration;
+  }
+
+  protected boolean isEmulator() {
+    return isEmulator;
+  }
+
+  protected static String wasbUrlToAbfsUrl(final String wasbUrl) {
+    return convertTestUrls(
+        wasbUrl, FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX,
+        FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX);
+  }
+
+  protected static String abfsUrlToWasbUrl(final String abfsUrl) {
+    return convertTestUrls(
+        abfsUrl, FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX,
+        FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX);
+  }
+
+  private static String convertTestUrls(
+      final String url, final String fromNonSecureScheme, final String fromSecureScheme, final String fromDnsPrefix,
+      final String toNonSecureScheme, final String toSecureScheme, final String toDnsPrefix) {
+    String data = null;
+    if (url.startsWith(fromNonSecureScheme + "://")) {
+      data = url.replace(fromNonSecureScheme + "://", toNonSecureScheme + "://");
+    } else if (url.startsWith(fromSecureScheme + "://")) {
+      data = url.replace(fromSecureScheme + "://", toSecureScheme + "://");
+    }
+
+    data = data.replace("." + fromDnsPrefix + ".", "." + toDnsPrefix + ".");
+    return data;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
new file mode 100644
index 0000000..10d42d1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
@@ -0,0 +1,78 @@
+/**
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test append operations.
+ */
+public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest {
+  private static final Path TEST_FILE_PATH = new Path("testfile");
+  private static final Path TEST_FOLDER_PATH = new Path("testFolder");
+  public ITestAzureBlobFileSystemAppend() {
+    super();
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testAppendDirShouldFail() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Path filePath = TEST_FILE_PATH;
+    fs.mkdirs(filePath);
+    fs.append(filePath, 0);
+  }
+
+  @Test
+  public void testAppendWithLength0() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
+    final byte[] b = new byte[1024];
+    new Random().nextBytes(b);
+    stream.write(b, 1000, 0);
+
+    assertEquals(0, stream.getPos());
+  }
+
+
+  @Test(expected = FileNotFoundException.class)
+  public void testAppendFileAfterDelete() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Path filePath = TEST_FILE_PATH;
+    fs.create(filePath);
+    fs.delete(filePath, false);
+
+    fs.append(filePath);
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testAppendDirectory() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Path folderPath = TEST_FOLDER_PATH;
+    fs.mkdirs(folderPath);
+    fs.append(folderPath);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
new file mode 100644
index 0000000..d107c9d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
@@ -0,0 +1,83 @@
+/**
+ * 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.azurebfs;
+
+import com.microsoft.azure.storage.CloudStorageAccount;
+import com.microsoft.azure.storage.blob.CloudBlobClient;
+import com.microsoft.azure.storage.blob.CloudBlobContainer;
+import com.microsoft.azure.storage.blob.CloudBlockBlob;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test AzureBlobFileSystem back compatibility with WASB.
+ */
+public class ITestAzureBlobFileSystemBackCompat extends DependencyInjectedTest {
+  public ITestAzureBlobFileSystemBackCompat() {
+    super();
+  }
+
+  @Test
+  public void testBlobBackCompat() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    String storageConnectionString = getBlobConnectionString();
+    CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString);
+    CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
+    CloudBlobContainer container = blobClient.getContainerReference(this.getFileSystemName());
+    container.createIfNotExists();
+
+    CloudBlockBlob blockBlob = container.getBlockBlobReference("test/10/10/10");
+    blockBlob.uploadText("");
+
+    blockBlob = container.getBlockBlobReference("test/10/123/3/2/1/3");
+    blockBlob.uploadText("");
+
+    FileStatus[] fileStatuses = fs.listStatus(new Path("/test/10/"));
+    assertEquals(fileStatuses.length, 2);
+    assertEquals(fileStatuses[0].getPath().getName(), "10");
+    assertTrue(fileStatuses[0].isDirectory());
+    assertEquals(fileStatuses[0].getLen(), 0);
+    assertEquals(fileStatuses[1].getPath().getName(), "123");
+    assertTrue(fileStatuses[1].isDirectory());
+    assertEquals(fileStatuses[1].getLen(), 0);
+  }
+
+  private String getBlobConnectionString() {
+    String connectionString;
+    if (isEmulator()) {
+      connectionString = "DefaultEndpointsProtocol=http;BlobEndpoint=http://"
+              + this.getHostName() + ":8880/" + this.getAccountName().split("\\.") [0]
+              + ";AccountName=" + this.getAccountName().split("\\.")[0]
+              + ";AccountKey=" + this.getAccountKey();
+    }
+    else {
+      connectionString = "DefaultEndpointsProtocol=http;BlobEndpoint=http://"
+              + this.getAccountName().replaceFirst("\\.dfs\\.", ".blob.")
+              + ";AccountName=" + this.getAccountName().split("\\.")[0]
+              + ";AccountKey=" + this.getAccountKey();
+    }
+
+    return connectionString;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java
new file mode 100644
index 0000000..c158e03
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java
@@ -0,0 +1,100 @@
+/**
+ * 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.azurebfs;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test copy operation.
+ */
+public class ITestAzureBlobFileSystemCopy extends DependencyInjectedTest {
+  public ITestAzureBlobFileSystemCopy() {
+    super();
+  }
+
+  @Test
+  public void testCopyFromLocalFileSystem() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    Path localFilePath = new Path(System.getProperty("test.build.data",
+        "azure_test"));
+    FileSystem localFs = FileSystem.get(new Configuration());
+    localFs.delete(localFilePath, true);
+    try {
+      writeString(localFs, localFilePath, "Testing");
+      Path dstPath = new Path("copiedFromLocal");
+      assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false,
+          fs.getConf()));
+      assertTrue(fs.exists(dstPath));
+      assertEquals("Testing", readString(fs, dstPath));
+      fs.delete(dstPath, true);
+    } finally {
+      localFs.delete(localFilePath, true);
+    }
+  }
+
+  private String readString(FileSystem fs, Path testFile) throws IOException {
+    FSDataInputStream inputStream = fs.open(testFile);
+    String ret = readString(inputStream);
+    inputStream.close();
+    return ret;
+  }
+
+  private String readString(FSDataInputStream inputStream) throws IOException {
+    BufferedReader reader = new BufferedReader(new InputStreamReader(
+        inputStream));
+    final int bufferSize = 1024;
+    char[] buffer = new char[bufferSize];
+    int count = reader.read(buffer, 0, bufferSize);
+    if (count > bufferSize) {
+      throw new IOException("Exceeded buffer size");
+    }
+    inputStream.close();
+    return new String(buffer, 0, count);
+  }
+
+  private void writeString(FileSystem fs, Path path, String value)
+      throws IOException {
+    FSDataOutputStream outputStream = fs.create(path, true);
+    writeString(outputStream, value);
+  }
+
+  private void writeString(FSDataOutputStream outputStream, String value)
+      throws IOException {
+    BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(
+        outputStream));
+    writer.write(value);
+    writer.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
new file mode 100644
index 0000000..c9b99e6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
@@ -0,0 +1,110 @@
+/**
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.util.EnumSet;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test create operation.
+ */
+public class ITestAzureBlobFileSystemCreate extends DependencyInjectedTest {
+  private static final Path TEST_FILE_PATH = new Path("testfile");
+  private static final Path TEST_FOLDER_PATH = new Path("testFolder");
+  private static final String TEST_CHILD_FILE = "childFile";
+  public ITestAzureBlobFileSystemCreate() {
+    super();
+  }
+
+  @Test(expected = FileAlreadyExistsException.class)
+  public void testCreateFileWithExistingDir() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(TEST_FOLDER_PATH);
+    fs.create(TEST_FOLDER_PATH);
+  }
+
+  @Test
+  public void testEnsureFileCreated() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(TEST_FILE_PATH);
+
+    FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
+    assertNotNull(fileStatus);
+  }
+
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testCreateNonRecursive() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
+    try {
+      fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null);
+      assertTrue("Should've thrown", false);
+    } catch (FileNotFoundException e) {
+    }
+    fs.mkdirs(TEST_FOLDER_PATH);
+    fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
+        .close();
+    assertTrue(fs.exists(testFile));
+  }
+
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testCreateNonRecursive1() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
+    try {
+      fs.createNonRecursive(testFile, FsPermission.getDefault(), EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1, 1024, null);
+      assertTrue("Should've thrown", false);
+    } catch (FileNotFoundException e) {
+    }
+    fs.mkdirs(TEST_FOLDER_PATH);
+    fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
+        .close();
+    assertTrue(fs.exists(testFile));
+  }
+
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testCreateNonRecursive2() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+
+    Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
+    try {
+      fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null);
+      assertTrue("Should've thrown", false);
+    } catch (FileNotFoundException e) {
+    }
+    fs.mkdirs(TEST_FOLDER_PATH);
+    fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
+        .close();
+    assertTrue(fs.exists(testFile));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
new file mode 100644
index 0000000..372a087
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
@@ -0,0 +1,119 @@
+/**
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test delete operation.
+ */
+public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest {
+  public ITestAzureBlobFileSystemDelete() {
+    super();
+  }
+
+  @Test
+  public void testDeleteRoot() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+
+    fs.mkdirs(new Path("/testFolder0"));
+    fs.mkdirs(new Path("/testFolder1"));
+    fs.mkdirs(new Path("/testFolder2"));
+    fs.create(new Path("/testFolder1/testfile"));
+    fs.create(new Path("/testFolder1/testfile2"));
+    fs.create(new Path("/testFolder1/testfile3"));
+
+    FileStatus[] ls = fs.listStatus(new Path("/"));
+    assertEquals(4, ls.length); // and user dir
+
+    fs.delete(new Path("/"), true);
+    ls = fs.listStatus(new Path("/"));
+    assertEquals(0, ls.length);
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testOpenFileAfterDelete() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(new Path("/testFile"));
+    fs.delete(new Path("/testFile"), false);
+
+    fs.open(new Path("/testFile"));
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testEnsureFileIsDeleted() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(new Path("testfile"));
+    fs.delete(new Path("testfile"), false);
+
+    fs.getFileStatus(new Path("testfile"));
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testDeleteDirectory() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("testfile"));
+    fs.mkdirs(new Path("testfile/test1"));
+    fs.mkdirs(new Path("testfile/test1/test2"));
+
+    fs.delete(new Path("testfile"), true);
+    fs.getFileStatus(new Path("testfile"));
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testDeleteFirstLevelDirectory() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final List<Future> tasks = new ArrayList<>();
+
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < 1000; i++) {
+      final Path fileName = new Path("/test/" + i);
+      Callable<Void> callable = new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          fs.create(fileName);
+          return null;
+        }
+      };
+
+      tasks.add(es.submit(callable));
+    }
+
+    for (Future<Void> task : tasks) {
+      task.get();
+    }
+
+    es.shutdownNow();
+    fs.delete(new Path("/test"), true);
+    fs.getFileStatus(new Path("/test"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
new file mode 100644
index 0000000..4985f58
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
@@ -0,0 +1,150 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertArrayEquals;
+
+/**
+ * Test end to end between ABFS client and ABFS server.
+ */
+public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
+  private static final Path TEST_FILE = new Path("testfile");
+  private static final int TEST_BYTE = 100;
+  private static final int TEST_OFFSET = 100;
+  private static final int TEST_DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
+  private static final int TEST_DEFAULT_READ_BUFFER_SIZE = 1023900;
+
+  public ITestAzureBlobFileSystemE2E() {
+    super();
+    Configuration configuration = this.getConfiguration();
+    configuration.set(ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, "0");
+    this.getMockServiceInjector().replaceInstance(Configuration.class, configuration);
+
+  }
+
+  @Test
+  public void testWriteOneByteToFile() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    FSDataOutputStream stream = fs.create(TEST_FILE);
+
+    stream.write(TEST_BYTE);
+    stream.close();
+
+    FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
+    assertEquals(1, fileStatus.getLen());
+  }
+
+  @Test
+  public void testReadWriteBytesToFile() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    testWriteOneByteToFile();
+    FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);
+    int i = inputStream.read();
+    inputStream.close();
+
+    assertEquals(TEST_BYTE, i);
+  }
+
+  @Test (expected = IOException.class)
+  public void testOOBWrites() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    int readBufferSize = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getReadBufferSize();
+
+    fs.create(TEST_FILE);
+    FSDataOutputStream writeStream = fs.create(TEST_FILE);
+
+    byte[] bytesToRead = new byte[readBufferSize];
+    final byte[] b = new byte[2 * readBufferSize];
+    new Random().nextBytes(b);
+
+    writeStream.write(b);
+    writeStream.flush();
+    writeStream.close();
+
+    FSDataInputStream readStream = fs.open(TEST_FILE);
+    readStream.read(bytesToRead, 0, readBufferSize);
+
+    writeStream = fs.create(TEST_FILE);
+    writeStream.write(b);
+    writeStream.flush();
+    writeStream.close();
+
+    readStream.read(bytesToRead, 0, readBufferSize);
+    readStream.close();
+  }
+
+  @Test
+  public void testWriteWithBufferOffset() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final FSDataOutputStream stream = fs.create(TEST_FILE);
+
+    final byte[] b = new byte[1024 * 1000];
+    new Random().nextBytes(b);
+    stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET);
+    stream.close();
+
+    final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE];
+    FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);
+    int result = inputStream.read(r);
+
+    assertNotEquals(-1, result);
+    assertArrayEquals(r, Arrays.copyOfRange(b, TEST_OFFSET, b.length));
+
+    inputStream.close();
+  }
+
+  @Test
+  public void testReadWriteHeavyBytesToFileWithSmallerChunks() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final FSDataOutputStream stream = fs.create(TEST_FILE);
+
+    final byte[] writeBuffer = new byte[5 * 1000 * 1024];
+    new Random().nextBytes(writeBuffer);
+    stream.write(writeBuffer);
+    stream.close();
+
+    final byte[] readBuffer = new byte[5 * 1000 * 1024];
+    FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);
+    int offset = 0;
+    while (inputStream.read(readBuffer, offset, TEST_OFFSET) > 0) {
+      offset += TEST_OFFSET;
+    }
+
+    assertArrayEquals(readBuffer, writeBuffer);
+    inputStream.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
new file mode 100644
index 0000000..616253b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java
@@ -0,0 +1,133 @@
+/**
+ * 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.azurebfs;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Test end to end between ABFS client and ABFS server with heavy traffic.
+ */
+public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest {
+  private static final int TEN = 10;
+  private static final int ONE_THOUSAND = 1000;
+  private static final int BASE_SIZE = 1024;
+  private static final int ONE_MB = 1024 * 1024;
+  private static final int DEFAULT_WRITE_TIMES = 100;
+  private static final Path TEST_FILE = new Path("testfile");
+
+  public ITestAzureBlobFileSystemE2EScale() {
+    super();
+  }
+
+  @Test
+  public void testWriteHeavyBytesToFile() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final FSDataOutputStream stream = fs.create(TEST_FILE);
+    ExecutorService es = Executors.newFixedThreadPool(TEN);
+
+    int testWriteBufferSize = 2 * TEN * ONE_THOUSAND * BASE_SIZE;
+    final byte[] b = new byte[testWriteBufferSize];
+    new Random().nextBytes(b);
+    List<Future<Void>> tasks = new ArrayList<>();
+
+    for (int i = 0; i < DEFAULT_WRITE_TIMES; i++) {
+      Callable<Void> callable = new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          stream.write(b);
+          return null;
+        }
+      };
+
+      tasks.add(es.submit(callable));
+    }
+
+    for (Future<Void> task : tasks) {
+      task.get();
+    }
+
+    tasks.clear();
+    stream.close();
+
+    es.shutdownNow();
+    FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
+    assertEquals(testWriteBufferSize * DEFAULT_WRITE_TIMES, fileStatus.getLen());
+  }
+
+  @Test
+  public void testReadWriteHeavyBytesToFile() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final FSDataOutputStream stream = fs.create(TEST_FILE);
+
+    int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE;
+    final byte[] b = new byte[testBufferSize];
+    new Random().nextBytes(b);
+    stream.write(b);
+    stream.close();
+
+    final byte[] r = new byte[testBufferSize];
+    FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB);
+    int result = inputStream.read(r);
+    inputStream.close();
+
+    assertNotEquals(-1, result);
+    assertArrayEquals(r, b);
+  }
+
+  @Test
+  public void testReadWriteHeavyBytesToFileWithStatistics() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final FSDataOutputStream stream = fs.create(TEST_FILE);
+    final FileSystem.Statistics abfsStatistics = fs.getFsStatistics();
+    abfsStatistics.reset();
+
+    int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE;
+    final byte[] b = new byte[testBufferSize];
+    new Random().nextBytes(b);
+    stream.write(b);
+    stream.close();
+
+    final byte[] r = new byte[testBufferSize];
+    FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB);
+    inputStream.read(r);
+    inputStream.close();
+
+    Assert.assertEquals(r.length, abfsStatistics.getBytesRead());
+    Assert.assertEquals(b.length, abfsStatistics.getBytesWritten());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
new file mode 100644
index 0000000..bfa662d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test FileStatus.
+ */
+public class ITestAzureBlobFileSystemFileStatus extends DependencyInjectedTest {
+  private static final Path TEST_FILE = new Path("testFile");
+  private static final Path TEST_FOLDER = new Path("testDir");
+  public ITestAzureBlobFileSystemFileStatus() {
+    super();
+  }
+
+  @Test
+  public void testEnsureStatusWorksForRoot() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+
+    fs.getFileStatus(new Path("/"));
+    fs.listStatus(new Path("/"));
+  }
+
+  @Test
+  public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(TEST_FILE);
+    fs.mkdirs(TEST_FOLDER);
+
+    FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
+    assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission());
+    assertEquals(fs.getOwnerUser(), fileStatus.getGroup());
+    assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner());
+
+    fileStatus = fs.getFileStatus(TEST_FOLDER);
+    assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission());
+    assertEquals(fs.getOwnerUser(), fileStatus.getGroup());
+    assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
new file mode 100644
index 0000000..8c2e8ce
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
@@ -0,0 +1,204 @@
+/**
+ * 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.azurebfs;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Test flush operation.
+ */
+public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest {
+  private static final int BASE_SIZE = 1024;
+  private static final int ONE_THOUSAND = 1000;
+  private static final int TEST_BUFFER_SIZE = 5 * ONE_THOUSAND * BASE_SIZE;
+  private static final int ONE_MB = 1024 * 1024;
+  private static final int FLUSH_TIMES = 200;
+  private static final int THREAD_SLEEP_TIME = 6000;
+
+  private static final Path TEST_FILE_PATH = new Path("/testfile");
+
+  public ITestAzureBlobFileSystemFlush() {
+    super();
+  }
+
+  @Test
+  public void testAbfsOutputStreamAsyncFlushWithRetainUncommitedData() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
+
+    final byte[] b = new byte[TEST_BUFFER_SIZE];
+    new Random().nextBytes(b);
+
+    for (int i = 0; i < 2; i++) {
+      stream.write(b);
+
+      for (int j = 0; j < FLUSH_TIMES; j++) {
+        stream.flush();
+        Thread.sleep(10);
+      }
+    }
+
+    stream.close();
+
+    final byte[] r = new byte[TEST_BUFFER_SIZE];
+    FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB);
+
+    while (inputStream.available() != 0) {
+      int result = inputStream.read(r);
+
+      assertNotEquals(-1, result);
+      assertArrayEquals(r, b);
+    }
+
+    inputStream.close();
+  }
+
+  @Test
+  public void testAbfsOutputStreamSyncFlush() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
+
+    final byte[] b = new byte[TEST_BUFFER_SIZE];
+    new Random().nextBytes(b);
+    stream.write(b);
+
+    for (int i = 0; i < FLUSH_TIMES; i++) {
+      stream.hsync();
+      stream.hflush();
+      Thread.sleep(10);
+    }
+    stream.close();
+
+    final byte[] r = new byte[TEST_BUFFER_SIZE];
+    FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB);
+    int result = inputStream.read(r);
+
+    assertNotEquals(-1, result);
+    assertArrayEquals(r, b);
+
+    inputStream.close();
+  }
+
+
+  @Test
+  public void testWriteHeavyBytesToFileSyncFlush() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
+    final FileSystem.Statistics abfsStatistics = fs.getFsStatistics();
+    abfsStatistics.reset();
+
+    ExecutorService es = Executors.newFixedThreadPool(10);
+
+    final byte[] b = new byte[TEST_BUFFER_SIZE];
+    new Random().nextBytes(b);
+
+    List<Future<Void>> tasks = new ArrayList<>();
+    for (int i = 0; i < FLUSH_TIMES; i++) {
+      Callable<Void> callable = new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          stream.write(b);
+          return null;
+        }
+      };
+
+      tasks.add(es.submit(callable));
+    }
+
+    boolean shouldStop = false;
+    while (!shouldStop) {
+      shouldStop = true;
+      for (Future<Void> task : tasks) {
+        if (!task.isDone()) {
+          stream.hsync();
+          shouldStop = false;
+          Thread.sleep(THREAD_SLEEP_TIME);
+        }
+      }
+    }
+
+    tasks.clear();
+    stream.close();
+
+    es.shutdownNow();
+    FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
+    assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen());
+    assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, abfsStatistics.getBytesWritten());
+  }
+
+  @Test
+  public void testWriteHeavyBytesToFileAsyncFlush() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(TEST_FILE_PATH);
+    final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
+    ExecutorService es = Executors.newFixedThreadPool(10);
+
+    final byte[] b = new byte[TEST_BUFFER_SIZE];
+    new Random().nextBytes(b);
+
+    List<Future<Void>> tasks = new ArrayList<>();
+    for (int i = 0; i < FLUSH_TIMES; i++) {
+      Callable<Void> callable = new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          stream.write(b);
+          return null;
+        }
+      };
+
+      tasks.add(es.submit(callable));
+    }
+
+    boolean shouldStop = false;
+    while (!shouldStop) {
+      shouldStop = true;
+      for (Future<Void> task : tasks) {
+        if (!task.isDone()) {
+          stream.flush();
+          shouldStop = false;
+        }
+      }
+    }
+    Thread.sleep(THREAD_SLEEP_TIME);
+    tasks.clear();
+    stream.close();
+
+    es.shutdownNow();
+    FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
+    assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
new file mode 100644
index 0000000..d2ed400
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
@@ -0,0 +1,50 @@
+/**
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+
+/**
+ * Test filesystem initialization and creation.
+ */
+public class ITestAzureBlobFileSystemInitAndCreate extends DependencyInjectedTest {
+  public ITestAzureBlobFileSystemInitAndCreate() {
+    super();
+
+    this.getConfiguration().unset(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION);
+  }
+
+  @Override
+  public void initialize() {
+  }
+
+  @Override
+  public void testCleanup() {
+  }
+
+  @Test (expected = FileNotFoundException.class)
+  public void ensureFilesystemWillNotBeCreatedIfCreationConfigIsNotSet() throws Exception {
+    super.initialize();
+    this.getFileSystem();
+  }
+}
\ 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


[15/50] [abbrv] hadoop git commit: YARN-8633. Update DataTables version in yarn-common in line with JQuery 3 upgrade. Contributed by Akhil PB.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/demo_table.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/demo_table.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/demo_table.css
deleted file mode 100644
index 3bc0433..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/demo_table.css
+++ /dev/null
@@ -1,538 +0,0 @@
-/*
- *  File:         demo_table.css
- *  CVS:          $Id$
- *  Description:  CSS descriptions for DataTables demo pages
- *  Author:       Allan Jardine
- *  Created:      Tue May 12 06:47:22 BST 2009
- *  Modified:     $Date$ by $Author$
- *  Language:     CSS
- *  Project:      DataTables
- *
- *  Copyright 2009 Allan Jardine. All Rights Reserved.
- *
- * ***************************************************************************
- * DESCRIPTION
- *
- * The styles given here are suitable for the demos that are used with the standard DataTables
- * distribution (see www.datatables.net). You will most likely wish to modify these styles to
- * meet the layout requirements of your site.
- *
- * Common issues:
- *   'full_numbers' pagination - I use an extra selector on the body tag to ensure that there is
- *     no conflict between the two pagination types. If you want to use full_numbers pagination
- *     ensure that you either have "example_alt_pagination" as a body class name, or better yet,
- *     modify that selector.
- *   Note that the path used for Images is relative. All images are by default located in
- *     ../images/ - relative to this CSS file.
- */
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * DataTables features
- */
-
-.dataTables_wrapper {
-	position: relative;
-	min-height: 302px;
-	clear: both;
-	_height: 302px;
-	zoom: 1; /* Feeling sorry for IE */
-}
-
-.dataTables_processing {
-	position: absolute;
-	top: 50%;
-	left: 50%;
-	width: 250px;
-	height: 30px;
-	margin-left: -125px;
-	margin-top: -15px;
-	padding: 14px 0 2px 0;
-	border: 1px solid #ddd;
-	text-align: center;
-	color: #999;
-	font-size: 14px;
-	background-color: white;
-}
-
-.dataTables_length {
-	width: 40%;
-	float: left;
-}
-
-.dataTables_filter {
-	width: 50%;
-	float: right;
-	text-align: right;
-}
-
-.dataTables_info {
-	width: 60%;
-	float: left;
-}
-
-.dataTables_paginate {
-	width: 44px;
-	* width: 50px;
-	float: right;
-	text-align: right;
-}
-
-/* Pagination nested */
-.paginate_disabled_previous, .paginate_enabled_previous, .paginate_disabled_next, .paginate_enabled_next {
-	height: 19px;
-	width: 19px;
-	margin-left: 3px;
-	float: left;
-}
-
-.paginate_disabled_previous {
-	background-image: url('../images/back_disabled.jpg');
-}
-
-.paginate_enabled_previous {
-	background-image: url('../images/back_enabled.jpg');
-}
-
-.paginate_disabled_next {
-	background-image: url('../images/forward_disabled.jpg');
-}
-
-.paginate_enabled_next {
-	background-image: url('../images/forward_enabled.jpg');
-}
-
-
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * DataTables display
- */
-table.display {
-	margin: 0 auto;
-	clear: both;
-	width: 100%;
-	
-	/* Note Firefox 3.5 and before have a bug with border-collapse
-	 * ( https://bugzilla.mozilla.org/show%5Fbug.cgi?id=155955 ) 
-	 * border-spacing: 0; is one possible option. Conditional-css.com is
-	 * useful for this kind of thing
-	 *
-	 * Further note IE 6/7 has problems when calculating widths with border width.
-	 * It subtracts one px relative to the other browsers from the first column, and
-	 * adds one to the end...
-	 *
-	 * If you want that effect I'd suggest setting a border-top/left on th/td's and 
-	 * then filling in the gaps with other borders.
-	 */
-}
-
-table.display thead th {
-	padding: 3px 18px 3px 10px;
-	border-bottom: 1px solid black;
-	font-weight: bold;
-	cursor: pointer;
-	* cursor: hand;
-}
-
-table.display tfoot th {
-	padding: 3px 18px 3px 10px;
-	border-top: 1px solid black;
-	font-weight: bold;
-}
-
-table.display tr.heading2 td {
-	border-bottom: 1px solid #aaa;
-}
-
-table.display td {
-	padding: 3px 10px;
-}
-
-table.display td.center {
-	text-align: center;
-}
-
-
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * DataTables sorting
- */
-
-.sorting_asc {
-	background: url('../images/sort_asc.png') no-repeat center right;
-}
-
-.sorting_desc {
-	background: url('../images/sort_desc.png') no-repeat center right;
-}
-
-.sorting {
-	background: url('../images/sort_both.png') no-repeat center right;
-}
-
-.sorting_asc_disabled {
-	background: url('../images/sort_asc_disabled.png') no-repeat center right;
-}
-
-.sorting_desc_disabled {
-	background: url('../images/sort_desc_disabled.png') no-repeat center right;
-}
-
-
-
-
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * DataTables row classes
- */
-table.display tr.odd.gradeA {
-	background-color: #ddffdd;
-}
-
-table.display tr.even.gradeA {
-	background-color: #eeffee;
-}
-
-table.display tr.odd.gradeC {
-	background-color: #ddddff;
-}
-
-table.display tr.even.gradeC {
-	background-color: #eeeeff;
-}
-
-table.display tr.odd.gradeX {
-	background-color: #ffdddd;
-}
-
-table.display tr.even.gradeX {
-	background-color: #ffeeee;
-}
-
-table.display tr.odd.gradeU {
-	background-color: #ddd;
-}
-
-table.display tr.even.gradeU {
-	background-color: #eee;
-}
-
-
-tr.odd {
-	background-color: #E2E4FF;
-}
-
-tr.even {
-	background-color: white;
-}
-
-
-
-
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * Misc
- */
-.dataTables_scroll {
-	clear: both;
-}
-
-.dataTables_scrollBody {
-	*margin-top: -1px;
-}
-
-.top, .bottom {
-	padding: 15px;
-	background-color: #F5F5F5;
-	border: 1px solid #CCCCCC;
-}
-
-.top .dataTables_info {
-	float: none;
-}
-
-.clear {
-	clear: both;
-}
-
-.dataTables_empty {
-	text-align: center;
-}
-
-tfoot input {
-	margin: 0.5em 0;
-	width: 100%;
-	color: #444;
-}
-
-tfoot input.search_init {
-	color: #999;
-}
-
-td.group {
-	background-color: #d1cfd0;
-	border-bottom: 2px solid #A19B9E;
-	border-top: 2px solid #A19B9E;
-}
-
-td.details {
-	background-color: #d1cfd0;
-	border: 2px solid #A19B9E;
-}
-
-
-.example_alt_pagination div.dataTables_info {
-	width: 40%;
-}
-
-.paging_full_numbers {
-	width: 400px;
-	height: 22px;
-	line-height: 22px;
-}
-
-.paging_full_numbers span.paginate_button,
- 	.paging_full_numbers span.paginate_active {
-	border: 1px solid #aaa;
-	-webkit-border-radius: 5px;
-	-moz-border-radius: 5px;
-	padding: 2px 5px;
-	margin: 0 3px;
-	cursor: pointer;
-	*cursor: hand;
-}
-
-.paging_full_numbers span.paginate_button {
-	background-color: #ddd;
-}
-
-.paging_full_numbers span.paginate_button:hover {
-	background-color: #ccc;
-}
-
-.paging_full_numbers span.paginate_active {
-	background-color: #99B3FF;
-}
-
-table.display tr.even.row_selected td {
-	background-color: #B0BED9;
-}
-
-table.display tr.odd.row_selected td {
-	background-color: #9FAFD1;
-}
-
-
-/*
- * Sorting classes for columns
- */
-/* For the standard odd/even */
-tr.odd td.sorting_1 {
-	background-color: #D3D6FF;
-}
-
-tr.odd td.sorting_2 {
-	background-color: #DADCFF;
-}
-
-tr.odd td.sorting_3 {
-	background-color: #E0E2FF;
-}
-
-tr.even td.sorting_1 {
-	background-color: #EAEBFF;
-}
-
-tr.even td.sorting_2 {
-	background-color: #F2F3FF;
-}
-
-tr.even td.sorting_3 {
-	background-color: #F9F9FF;
-}
-
-
-/* For the Conditional-CSS grading rows */
-/*
- 	Colour calculations (based off the main row colours)
-  Level 1:
-		dd > c4
-		ee > d5
-	Level 2:
-	  dd > d1
-	  ee > e2
- */
-tr.odd.gradeA td.sorting_1 {
-	background-color: #c4ffc4;
-}
-
-tr.odd.gradeA td.sorting_2 {
-	background-color: #d1ffd1;
-}
-
-tr.odd.gradeA td.sorting_3 {
-	background-color: #d1ffd1;
-}
-
-tr.even.gradeA td.sorting_1 {
-	background-color: #d5ffd5;
-}
-
-tr.even.gradeA td.sorting_2 {
-	background-color: #e2ffe2;
-}
-
-tr.even.gradeA td.sorting_3 {
-	background-color: #e2ffe2;
-}
-
-tr.odd.gradeC td.sorting_1 {
-	background-color: #c4c4ff;
-}
-
-tr.odd.gradeC td.sorting_2 {
-	background-color: #d1d1ff;
-}
-
-tr.odd.gradeC td.sorting_3 {
-	background-color: #d1d1ff;
-}
-
-tr.even.gradeC td.sorting_1 {
-	background-color: #d5d5ff;
-}
-
-tr.even.gradeC td.sorting_2 {
-	background-color: #e2e2ff;
-}
-
-tr.even.gradeC td.sorting_3 {
-	background-color: #e2e2ff;
-}
-
-tr.odd.gradeX td.sorting_1 {
-	background-color: #ffc4c4;
-}
-
-tr.odd.gradeX td.sorting_2 {
-	background-color: #ffd1d1;
-}
-
-tr.odd.gradeX td.sorting_3 {
-	background-color: #ffd1d1;
-}
-
-tr.even.gradeX td.sorting_1 {
-	background-color: #ffd5d5;
-}
-
-tr.even.gradeX td.sorting_2 {
-	background-color: #ffe2e2;
-}
-
-tr.even.gradeX td.sorting_3 {
-	background-color: #ffe2e2;
-}
-
-tr.odd.gradeU td.sorting_1 {
-	background-color: #c4c4c4;
-}
-
-tr.odd.gradeU td.sorting_2 {
-	background-color: #d1d1d1;
-}
-
-tr.odd.gradeU td.sorting_3 {
-	background-color: #d1d1d1;
-}
-
-tr.even.gradeU td.sorting_1 {
-	background-color: #d5d5d5;
-}
-
-tr.even.gradeU td.sorting_2 {
-	background-color: #e2e2e2;
-}
-
-tr.even.gradeU td.sorting_3 {
-	background-color: #e2e2e2;
-}
-
-
-/*
- * Row highlighting example
- */
-.ex_highlight #example tbody tr.even:hover, #example tbody tr.even td.highlighted {
-	background-color: #ECFFB3;
-}
-
-.ex_highlight #example tbody tr.odd:hover, #example tbody tr.odd td.highlighted {
-	background-color: #E6FF99;
-}
-
-.ex_highlight_row #example tr.even:hover {
-	background-color: #ECFFB3;
-}
-
-.ex_highlight_row #example tr.even:hover td.sorting_1 {
-	background-color: #DDFF75;
-}
-
-.ex_highlight_row #example tr.even:hover td.sorting_2 {
-	background-color: #E7FF9E;
-}
-
-.ex_highlight_row #example tr.even:hover td.sorting_3 {
-	background-color: #E2FF89;
-}
-
-.ex_highlight_row #example tr.odd:hover {
-	background-color: #E6FF99;
-}
-
-.ex_highlight_row #example tr.odd:hover td.sorting_1 {
-	background-color: #D6FF5C;
-}
-
-.ex_highlight_row #example tr.odd:hover td.sorting_2 {
-	background-color: #E0FF84;
-}
-
-.ex_highlight_row #example tr.odd:hover td.sorting_3 {
-	background-color: #DBFF70;
-}
-
-
-/*
- * KeyTable
- */
-table.KeyTable td {
-	border: 3px solid transparent;
-}
-
-table.KeyTable td.focus {
-	border: 3px solid #3366FF;
-}
-
-table.display tr.gradeA {
-	background-color: #eeffee;
-}
-
-table.display tr.gradeC {
-	background-color: #ddddff;
-}
-
-table.display tr.gradeX {
-	background-color: #ffdddd;
-}
-
-table.display tr.gradeU {
-	background-color: #ddd;
-}
-
-div.box {
-	height: 100px;
-	padding: 10px;
-	overflow: auto;
-	border: 1px solid #8080FF;
-	background-color: #E5E5FF;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/jui-dt.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/jui-dt.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/jui-dt.css
deleted file mode 100644
index 6f6f414..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/css/jui-dt.css
+++ /dev/null
@@ -1,322 +0,0 @@
-/*
- *  File:         demo_table_jui.css
- *  CVS:          $Id$
- *  Description:  CSS descriptions for DataTables demo pages
- *  Author:       Allan Jardine
- *  Created:      Tue May 12 06:47:22 BST 2009
- *  Modified:     $Date$ by $Author$
- *  Language:     CSS
- *  Project:      DataTables
- *
- *  Copyright 2009 Allan Jardine. All Rights Reserved.
- *
- * ***************************************************************************
- * DESCRIPTION
- *
- * The styles given here are suitable for the demos that are used with the standard DataTables
- * distribution (see www.datatables.net). You will most likely wish to modify these styles to
- * meet the layout requirements of your site.
- *
- * Common issues:
- *   'full_numbers' pagination - I use an extra selector on the body tag to ensure that there is
- *     no conflict between the two pagination types. If you want to use full_numbers pagination
- *     ensure that you either have "example_alt_pagination" as a body class name, or better yet,
- *     modify that selector.
- *   Note that the path used for Images is relative. All images are by default located in
- *     ../images/ - relative to this CSS file.
- */
-
-
-/*
- * jQuery UI specific styling
- */
-
-.paging_two_button .ui-button {
-	float: left;
-	cursor: pointer;
-	* cursor: hand;
-}
-
-.paging_full_numbers .ui-button {
-	padding: 2px 6px;
-	margin: 0;
-	cursor: pointer;
-	* cursor: hand;
-}
-
-.ui-buttonset .ui-button {
-	margin-right: -0.1em !important;
-}
-
-.paging_full_numbers {
-	width: 350px !important;
-}
-
-.ui-toolbar {
-	padding: 5px;
-}
-
-.dataTables_paginate {
-	width: auto;
-}
-
-.dataTables_info {
-	padding-top: 3px;
-}
-
-table.display thead th {
-	padding: 3px 0px 3px 10px;
-	cursor: pointer;
-	* cursor: hand;
-}
-
-div.dataTables_wrapper .ui-widget-header {
-	font-weight: normal;
-}
-
-
-/*
- * Sort arrow icon positioning
- */
-table.display thead th div.DataTables_sort_wrapper {
-	position: relative;
-	padding-right: 20px;
-	padding-right: 20px;
-}
-
-table.display thead th div.DataTables_sort_wrapper span {
-	position: absolute;
-	top: 50%;
-	margin-top: -8px;
-	right: 0;
-}
-
-
-
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- *
- * Everything below this line is the same as demo_table.css. This file is
- * required for 'cleanliness' of the markup
- *
- * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
-
-
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * DataTables features
- */
-
-.dataTables_wrapper {
-	position: relative;
-	min-height: 35px;
-	_height: 35px;
-	clear: both;
-}
-
-.dataTables_processing {
-	position: absolute;
-	top: 0px;
-	left: 50%;
-	width: 250px;
-	margin-left: -125px;
-	border: 1px solid #ddd;
-	text-align: center;
-	color: #999;
-	font-size: 11px;
-	padding: 2px 0;
-}
-
-.dataTables_length {
-	width: 40%;
-	float: left;
-}
-
-.dataTables_filter {
-	width: 50%;
-	float: right;
-	text-align: right;
-}
-
-.dataTables_info {
-	width: 50%;
-	float: left;
-}
-
-.dataTables_paginate {
-	float: right;
-	text-align: right;
-}
-
-/* Pagination nested */
-.paginate_disabled_previous, .paginate_enabled_previous, .paginate_disabled_next, .paginate_enabled_next {
-	height: 19px;
-	width: 19px;
-	margin-left: 3px;
-	float: left;
-}
-
-.paginate_disabled_previous {
-	background-image: url('../images/back_disabled.jpg');
-}
-
-.paginate_enabled_previous {
-	background-image: url('../images/back_enabled.jpg');
-}
-
-.paginate_disabled_next {
-	background-image: url('../images/forward_disabled.jpg');
-}
-
-.paginate_enabled_next {
-	background-image: url('../images/forward_enabled.jpg');
-}
-
-
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * DataTables display
- */
-table.display {
-	margin: 0 auto;
-	width: 100%;
-	clear: both;
-	border-collapse: collapse;
-}
-
-table.display tfoot th {
-	padding: 3px 0px 3px 10px;
-	font-weight: bold;
-	font-weight: normal;
-}
-
-table.display tr.heading2 td {
-	border-bottom: 1px solid #aaa;
-}
-
-table.display td {
-	padding: 3px 10px;
-}
-
-table.display td.center {
-	text-align: center;
-}
-
-
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * DataTables sorting
- */
-
-.sorting_asc {
-	background: url('../images/sort_asc.jpg') no-repeat center right;
-}
-
-.sorting_desc {
-	background: url('../images/sort_desc.jpg') no-repeat center right;
-}
-
-.sorting {
-	background: url('../images/sort_both.jpg') no-repeat center right;
-}
-
-
-/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * *
- * Misc
- */
-.dataTables_scroll {
-	clear: both;
-}
-
-.top, .bottom {
-	padding: 15px;
-	background-color: #F5F5F5;
-	border: 1px solid #CCCCCC;
-}
-
-.top .dataTables_info {
-	float: none;
-}
-
-.clear {
-	clear: both;
-}
-
-.dataTables_empty {
-	text-align: center;
-}
-
-tfoot input {
-	margin: 0.5em 0;
-	width: 100%;
-	color: #444;
-}
-
-tfoot input.search_init {
-	color: #999;
-}
-
-td.group {
-	background-color: #d1cfd0;
-	border-bottom: 2px solid #A19B9E;
-	border-top: 2px solid #A19B9E;
-}
-
-td.details {
-	background-color: #d1cfd0;
-	border: 2px solid #A19B9E;
-}
-
-
-.example_alt_pagination div.dataTables_info {
-	width: 40%;
-}
-
-.paging_full_numbers span.paginate_button,
- 	.paging_full_numbers span.paginate_active {
-	border: 1px solid #aaa;
-	-webkit-border-radius: 5px;
-	-moz-border-radius: 5px;
-	padding: 2px 5px;
-	margin: 0 3px;
-	cursor: pointer;
-	*cursor: hand;
-}
-
-.paging_full_numbers span.paginate_button {
-	background-color: #ddd;
-}
-
-.paging_full_numbers span.paginate_button:hover {
-	background-color: #ccc;
-}
-
-.paging_full_numbers span.paginate_active {
-	background-color: #99B3FF;
-}
-
-table.display tr.even.row_selected td {
-	background-color: #B0BED9;
-}
-
-table.display tr.odd.row_selected td {
-	background-color: #9FAFD1;
-}
-
-/* Striping */
-tr.odd { background: rgba(255, 255, 255, 0.1); }
-tr.even { background: rgba(0, 0, 255, 0.05); }
-
-
-/*
- * Sorting classes for columns
- */
-tr.odd td.sorting_1 { background: rgba(0, 0, 0, 0.03); }
-tr.odd td.sorting_2 { background: rgba(0, 0, 0, 0.02); } 
-tr.odd td.sorting_3 { background: rgba(0, 0, 0, 0.02); }
-tr.even td.sorting_1 { background: rgba(0, 0, 0, 0.08); }
-tr.even td.sorting_2 { background: rgba(0, 0, 0, 0.06); }
-tr.even td.sorting_3 { background: rgba(0, 0, 0, 0.06); }
-
-.css_left { position: relative; float: left; }
-.css_right { position: relative; float: right; }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/Sorting icons.psd
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/Sorting icons.psd b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/Sorting icons.psd
deleted file mode 100644
index 53b2e06..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/Sorting icons.psd and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/back_disabled.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/back_disabled.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/back_disabled.jpg
deleted file mode 100644
index 1e73a54..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/back_disabled.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/back_enabled.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/back_enabled.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/back_enabled.jpg
deleted file mode 100644
index a6d764c..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/back_enabled.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/favicon.ico
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/favicon.ico b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/favicon.ico
deleted file mode 100644
index 6eeaa2a..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/favicon.ico and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/forward_disabled.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/forward_disabled.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/forward_disabled.jpg
deleted file mode 100644
index 28a9dc5..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/forward_disabled.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/forward_enabled.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/forward_enabled.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/forward_enabled.jpg
deleted file mode 100644
index 598c075..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/forward_enabled.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_asc.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_asc.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_asc.png
deleted file mode 100644
index a56d0e2..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_asc.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_asc_disabled.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_asc_disabled.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_asc_disabled.png
deleted file mode 100644
index b7e621e..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_asc_disabled.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_both.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_both.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_both.png
deleted file mode 100644
index 839ac4b..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_both.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_desc.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_desc.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_desc.png
deleted file mode 100644
index 90b2951..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_desc.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00013d6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_desc_disabled.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_desc_disabled.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_desc_disabled.png
deleted file mode 100644
index 2409653..0000000
Binary files a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/images/sort_desc_disabled.png and /dev/null differ


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


[40/50] [abbrv] hadoop git commit: HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
new file mode 100644
index 0000000..6d1c4ae
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
@@ -0,0 +1,63 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for rename operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractRename extends AbstractContractRenameTest {
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractRename(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
new file mode 100644
index 0000000..46072ad
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.Ignore;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Contract test for root directory operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractRootDirectory extends AbstractContractRootDirectoryTest {
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractRootDirectory(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(secure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+
+  @Override
+  @Ignore("ABFS always return false when non-recursively remove root dir")
+  public void testRmNonEmptyRootDirNonRecursive() throws Throwable {
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
new file mode 100644
index 0000000..1780f6f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
@@ -0,0 +1,44 @@
+/**
+ * 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.azurebfs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
+
+/**
+ * Contract test for secure distCP operation.
+ */
+public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDistCpTest {
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractSecureDistCp() throws Exception {
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(true);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected ITestAbfsFileSystemContract createContract(Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
new file mode 100644
index 0000000..aeeb042
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
@@ -0,0 +1,63 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for seek operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractSeek(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
new file mode 100644
index 0000000..ea9392d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
@@ -0,0 +1,62 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSetTimesTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for setTimes operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractSetTimes extends AbstractContractSetTimesTest {
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractSetTimes(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
new file mode 100644
index 0000000..e148a05
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azurebfs.contract;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Basic Contract test for Azure BlobFileSystem.
+ */
+public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAzureBlobFileSystemBasics() throws Exception {
+    // If contract tests are running in parallel, some root level tests in this file will fail
+    // due to the race condition. Hence for this contract test it should be tested in different container
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(false, false);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    this.dependencyInjectedContractTest.initialize();
+    fs = this.dependencyInjectedContractTest.getFileSystem();
+  }
+
+  @Test
+  public void testListOnFolderWithNoChildren() throws IOException {
+    assertTrue(fs.mkdirs(path("testListStatus/c/1")));
+
+    FileStatus[] paths;
+    paths = fs.listStatus(path("testListStatus"));
+    assertEquals(1, paths.length);
+
+    // ListStatus on folder with child
+    paths = fs.listStatus(path("testListStatus/c"));
+    assertEquals(1, paths.length);
+
+    // Remove the child and listStatus
+    fs.delete(path("testListStatus/c/1"), true);
+    paths = fs.listStatus(path("testListStatus/c"));
+    assertEquals(0, paths.length);
+    assertTrue(fs.delete(path("testListStatus"), true));
+  }
+
+  @Test
+  public void testListOnfileAndFolder() throws IOException {
+    Path folderPath = path("testListStatus/folder");
+    Path filePath = path("testListStatus/file");
+
+    assertTrue(fs.mkdirs(folderPath));
+    fs.create(filePath);
+
+    FileStatus[] listFolderStatus;
+    listFolderStatus = fs.listStatus(path("testListStatus"));
+    assertEquals(filePath, listFolderStatus[0].getPath());
+
+    //List on file should return absolute path
+    FileStatus[] listFileStatus = fs.listStatus(filePath);
+    assertEquals(filePath, listFileStatus[0].getPath());
+  }
+
+  @Override
+  @Ignore("Not implemented in ABFS yet")
+  public void testMkdirsWithUmask() throws Exception {
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/package-info.java
new file mode 100644
index 0000000..f3ff483
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contract;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
new file mode 100644
index 0000000..03320d6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
@@ -0,0 +1,124 @@
+/**
+ * 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.azurebfs.diagnostics;
+
+import org.apache.commons.codec.Charsets;
+import org.junit.Test;
+
+
+import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.commons.codec.binary.Base64;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test configuration validators.
+ */
+public class TestConfigurationValidators extends DependencyInjectedTest {
+
+  private static final String FAKE_KEY = "FakeKey";
+
+  public TestConfigurationValidators() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testIntegerConfigValidator() throws Exception {
+    IntegerConfigurationBasicValidator integerConfigurationValidator = new IntegerConfigurationBasicValidator(
+        MIN_BUFFER_SIZE, MAX_BUFFER_SIZE, DEFAULT_READ_BUFFER_SIZE, FAKE_KEY, false);
+
+    assertEquals(MIN_BUFFER_SIZE, (int) integerConfigurationValidator.validate("3072"));
+    assertEquals(DEFAULT_READ_BUFFER_SIZE, (int) integerConfigurationValidator.validate(null));
+    assertEquals(MAX_BUFFER_SIZE, (int) integerConfigurationValidator.validate("104857600"));
+  }
+
+  @Test(expected = InvalidConfigurationValueException.class)
+  public void testIntegerConfigValidatorThrowsIfMissingValidValue() throws Exception {
+    IntegerConfigurationBasicValidator integerConfigurationValidator = new IntegerConfigurationBasicValidator(
+        MIN_BUFFER_SIZE, MAX_BUFFER_SIZE, DEFAULT_READ_BUFFER_SIZE, FAKE_KEY, true);
+    integerConfigurationValidator.validate("3072");
+  }
+
+  @Test
+  public void testLongConfigValidator() throws Exception {
+    LongConfigurationBasicValidator longConfigurationValidator = new LongConfigurationBasicValidator(
+        MIN_BUFFER_SIZE, MAX_BUFFER_SIZE, DEFAULT_WRITE_BUFFER_SIZE, FAKE_KEY, false);
+
+    assertEquals(DEFAULT_WRITE_BUFFER_SIZE, (long) longConfigurationValidator.validate(null));
+    assertEquals(MIN_BUFFER_SIZE, (long) longConfigurationValidator.validate("3072"));
+    assertEquals(MAX_BUFFER_SIZE, (long) longConfigurationValidator.validate("104857600"));
+  }
+
+  @Test(expected = InvalidConfigurationValueException.class)
+  public void testLongConfigValidatorThrowsIfMissingValidValue() throws Exception {
+    LongConfigurationBasicValidator longConfigurationValidator = new LongConfigurationBasicValidator(
+        MIN_BUFFER_SIZE, MAX_BUFFER_SIZE, DEFAULT_READ_BUFFER_SIZE, FAKE_KEY, true);
+    longConfigurationValidator.validate(null);
+  }
+
+  @Test
+  public void testBooleanConfigValidator() throws Exception {
+    BooleanConfigurationBasicValidator booleanConfigurationValidator = new BooleanConfigurationBasicValidator(FAKE_KEY, false, false);
+
+    assertEquals(true, booleanConfigurationValidator.validate("true"));
+    assertEquals(false, booleanConfigurationValidator.validate("False"));
+    assertEquals(false, booleanConfigurationValidator.validate(null));
+  }
+
+  @Test(expected = InvalidConfigurationValueException.class)
+  public void testBooleanConfigValidatorThrowsIfMissingValidValue() throws Exception {
+    BooleanConfigurationBasicValidator booleanConfigurationValidator = new BooleanConfigurationBasicValidator(FAKE_KEY, false, true);
+    booleanConfigurationValidator.validate("almostTrue");
+  }
+
+  @Test
+  public void testStringConfigValidator() throws Exception {
+    StringConfigurationBasicValidator stringConfigurationValidator = new StringConfigurationBasicValidator(FAKE_KEY, "value", false);
+
+    assertEquals("value", stringConfigurationValidator.validate(null));
+    assertEquals("someValue", stringConfigurationValidator.validate("someValue"));
+  }
+
+  @Test(expected = InvalidConfigurationValueException.class)
+  public void testStringConfigValidatorThrowsIfMissingValidValue() throws Exception {
+    StringConfigurationBasicValidator stringConfigurationValidator = new StringConfigurationBasicValidator(FAKE_KEY, "value", true);
+    stringConfigurationValidator.validate(null);
+  }
+
+  @Test
+  public void testBase64StringConfigValidator() throws Exception {
+    String encodedVal = new String(new Base64().encode("someValue".getBytes()), Charsets.UTF_8);
+    Base64StringConfigurationBasicValidator base64StringConfigurationValidator = new Base64StringConfigurationBasicValidator(FAKE_KEY, "", false);
+
+    assertEquals("", base64StringConfigurationValidator.validate(null));
+    assertEquals(encodedVal, base64StringConfigurationValidator.validate(encodedVal));
+  }
+
+  @Test(expected = InvalidConfigurationValueException.class)
+  public void testBase64StringConfigValidatorThrowsIfMissingValidValue() throws Exception {
+    Base64StringConfigurationBasicValidator base64StringConfigurationValidator = new Base64StringConfigurationBasicValidator(FAKE_KEY, "value", true);
+    base64StringConfigurationValidator.validate("some&%Value");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java
new file mode 100644
index 0000000..c3434ac
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.diagnostics;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/package-info.java
new file mode 100644
index 0000000..811fdcb
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java
new file mode 100644
index 0000000..588df20
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java
@@ -0,0 +1,122 @@
+/**
+ * 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.azurebfs.services;
+
+import java.util.Hashtable;
+
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test AbfsHttpServiceImpl.
+ */
+public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest {
+  private static final int TEST_DATA = 100;
+  private static final Path TEST_PATH = new Path("/testfile");
+  public ITestAbfsHttpServiceImpl() {
+    super();
+  }
+
+  @Test
+  public void testReadWriteBytesToFileAndEnsureThreadPoolCleanup() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    testWriteOneByteToFileAndEnsureThreadPoolCleanup();
+
+    FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024);
+    int i = inputStream.read();
+
+    assertEquals(TEST_DATA, i);
+  }
+
+  @Test
+  public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    FSDataOutputStream stream = fs.create(TEST_PATH);
+
+    stream.write(TEST_DATA);
+    stream.close();
+
+    FileStatus fileStatus = fs.getFileStatus(TEST_PATH);
+    assertEquals(1, fileStatus.getLen());
+  }
+
+  @Test
+  @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant")
+  public void testBase64FileSystemProperties() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "{ value: value }");
+    AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties(
+        fs, properties);
+    Hashtable<String, String> fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs);
+
+    Assert.assertEquals(properties, fetchedProperties);
+  }
+
+  @Test
+  public void testBase64PathProperties() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "{ value: valueTest }");
+    fs.create(TEST_PATH);
+    AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties(
+        fs, TEST_PATH, properties);
+    Hashtable<String, String> fetchedProperties =
+        AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH);
+
+    Assert.assertEquals(properties, fetchedProperties);
+  }
+
+  @Test (expected = Exception.class)
+  public void testBase64InvalidFileSystemProperties() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "{ value: value歲 }");
+    AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties(
+        fs, properties);
+    Hashtable<String, String> fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs);
+
+    Assert.assertEquals(properties, fetchedProperties);
+  }
+
+  @Test (expected = Exception.class)
+  public void testBase64InvalidPathProperties() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "{ value: valueTest兩 }");
+    fs.create(TEST_PATH);
+    AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties(
+        fs, TEST_PATH, properties);
+    Hashtable<String, String> fetchedProperties =
+        AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH);
+
+    Assert.assertEquals(properties, fetchedProperties);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
new file mode 100644
index 0000000..1db93cb
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
@@ -0,0 +1,80 @@
+/**
+ * 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.azurebfs.services;
+
+import java.util.Random;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+
+import org.junit.Test;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Test read, write and seek.
+ */
+public class ITestReadWriteAndSeek extends DependencyInjectedTest {
+  private static final Path TEST_PATH = new Path("/testfile");
+  public ITestReadWriteAndSeek() {
+    super();
+  }
+
+  @Test
+  public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception {
+    testReadWriteAndSeek(MIN_BUFFER_SIZE);
+    testReadWriteAndSeek(DEFAULT_READ_BUFFER_SIZE);
+    testReadWriteAndSeek(MAX_BUFFER_SIZE);
+  }
+
+  private void testReadWriteAndSeek(int bufferSize) throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final ConfigurationServiceImpl configurationservice = (ConfigurationServiceImpl) AbfsServiceProviderImpl.instance().get(ConfigurationService.class);
+
+    fs.create(TEST_PATH);
+
+    configurationservice.setWriteBufferSize(bufferSize);
+    configurationservice.setReadBufferSize(bufferSize);
+
+    final FSDataOutputStream stream = fs.create(TEST_PATH);
+
+    final byte[] b = new byte[2 * bufferSize];
+    new Random().nextBytes(b);
+    stream.write(b);
+    stream.close();
+
+    final byte[] r = new byte[2 * bufferSize];
+    final FSDataInputStream inputStream = fs.open(TEST_PATH);
+    inputStream.seek(bufferSize);
+    int result = inputStream.read(r, bufferSize, bufferSize);
+    assertNotEquals(-1, result);
+
+    inputStream.seek(0);
+    result = inputStream.read(r, 0, bufferSize);
+    assertNotEquals(-1, result);
+    assertArrayEquals(r, b);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
new file mode 100644
index 0000000..e90eb22
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
@@ -0,0 +1,79 @@
+/**
+ * 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.azurebfs.services;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.log4j.AppenderSkeleton;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.log4j.spi.LoggingEvent;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
+import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
+import org.apache.htrace.core.MilliSpan;
+import org.apache.htrace.core.TraceScope;
+
+/**
+ * Test tracing service.
+ */
+public class ITestTracingServiceImpl extends DependencyInjectedTest {
+  private final List<String> messageStorage;
+
+  public ITestTracingServiceImpl() {
+    super();
+    this.messageStorage = new ArrayList<>();
+  }
+
+  @Test
+  public void traceSerializationTest() throws Exception {
+    Logger rootLogger = Logger.getRootLogger();
+    rootLogger.setLevel(Level.TRACE);
+    rootLogger.addAppender(new AppenderSkeleton() {
+      @Override
+      protected void append(LoggingEvent event) {
+        if (event.getLogger().getName().indexOf("AzureBlobFileSystem") != -1) {
+          messageStorage.add(event.getRenderedMessage());
+        }
+      }
+
+      @Override
+      public void close() {
+
+      }
+
+      @Override
+      public boolean requiresLayout() {
+        return false;
+      }
+    });
+
+    TracingService tracingService = new TracingServiceImpl(new Configuration());
+    TraceScope traceScope = tracingService.traceBegin("Test Scope");
+    traceScope.addTimelineAnnotation("Timeline Annotations");
+    traceScope.addKVAnnotation("key", "value");
+    traceScope.close();
+
+    // Should not throw exception.
+    MilliSpan.fromJson(messageStorage.get(0));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java
new file mode 100644
index 0000000..2f27b16
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.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.fs.azurebfs.services;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
+import org.apache.http.client.utils.URIBuilder;
+
+/**
+ * Mock AbfsHttpClientFactoryImpl.
+ */
+@Singleton
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MockAbfsHttpClientFactoryImpl extends AbfsHttpClientFactoryImpl {
+  private final ConfigurationService configurationService;
+
+  @Inject
+  MockAbfsHttpClientFactoryImpl(
+      final ConfigurationService configurationService) {
+    super(configurationService);
+
+    this.configurationService = configurationService;
+  }
+
+  @VisibleForTesting
+  URIBuilder getURIBuilder(final String hostName, final FileSystem fs) {
+    final URIBuilder uriBuilder = new URIBuilder();
+
+    final String testHost = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME);
+    final Integer testHostPort = this.configurationService.getConfiguration().getInt(TestConfigurationKeys.FS_AZURE_TEST_HOST_PORT, 80);
+    final String testAccount = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
+
+    String scheme = FileSystemUriSchemes.HTTP_SCHEME;
+
+    uriBuilder.setScheme(scheme);
+    uriBuilder.setHost(testHost);
+    uriBuilder.setPort(testHostPort);
+
+    uriBuilder.setPath("/" + UriUtils.extractRawAccountFromAccountName(testAccount) + "/");
+
+    return uriBuilder;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
new file mode 100644
index 0000000..ff2fb2a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
@@ -0,0 +1,50 @@
+/**
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Mock AbfsServiceInjectorImpl.
+ */
+public class MockAbfsServiceInjectorImpl extends AbfsServiceInjectorImpl {
+  public MockAbfsServiceInjectorImpl(Configuration configuration) {
+    super(configuration);
+  }
+
+  public <T> void replaceInstance(Class<T> tInterface, Object object) {
+    this.removeInstance(tInterface);
+    this.removeProvider(tInterface);
+    this.getInstances().put(tInterface, object);
+  }
+
+  public <T> void removeInstance(Class<T> tInterface) {
+    this.getInstances().remove(tInterface);
+  }
+
+  public <T> void replaceProvider(Class<T> tInterface, Class<? extends T> tClazz) {
+    this.removeInstance(tInterface);
+    this.removeProvider(tInterface);
+    this.getProviders().put(tInterface, tClazz);
+  }
+
+  public <T> void removeProvider(Class<T> tInterface) {
+    this.getProviders().remove(tInterface);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java
new file mode 100644
index 0000000..5992f75
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.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.azurebfs.services;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+
+/**
+ * Mock ABFS ServiceProviderImpl.
+ */
+public final class MockServiceProviderImpl {
+  public static void create(MockAbfsServiceInjectorImpl abfsServiceInjector) {
+    Injector injector = Guice.createInjector(abfsServiceInjector);
+    AbfsServiceProviderImpl.create(injector);
+  }
+
+  private MockServiceProviderImpl() {
+    // no-op
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java
new file mode 100644
index 0000000..688c522
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java
@@ -0,0 +1,149 @@
+/**
+ * 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.azurebfs.services;
+
+import java.lang.reflect.Field;
+
+import org.apache.commons.codec.Charsets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT;
+
+import org.apache.commons.codec.binary.Base64;
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+/**
+ * Test ConfigurationServiceFieldsValidation.
+ */
+public class TestConfigurationServiceFieldsValidation  {
+  private ConfigurationServiceImpl configService;
+
+  private static final String INT_KEY= "intKey";
+  private static final String LONG_KEY= "longKey";
+  private static final String STRING_KEY= "stringKey";
+  private static final String BASE64_KEY= "base64Key";
+  private static final String BOOLEAN_KEY= "booleanKey";
+  private static final int DEFAULT_INT = 4194304;
+  private static final int DEFAULT_LONG = 4194304;
+
+  private static final int TEST_INT = 1234565;
+  private static final int TEST_LONG = 4194304;
+
+  private final String encodedString;
+  private final String encodedAccountKey;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = INT_KEY,
+      MinValue = Integer.MIN_VALUE,
+      MaxValue = Integer.MAX_VALUE,
+      DefaultValue = DEFAULT_INT)
+  private int intField;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey = LONG_KEY,
+      MinValue = Long.MIN_VALUE,
+      MaxValue = Long.MAX_VALUE,
+      DefaultValue = DEFAULT_LONG)
+  private int longField;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = STRING_KEY,
+  DefaultValue = "default")
+  private String stringField;
+
+  @Base64StringConfigurationValidatorAnnotation(ConfigurationKey = BASE64_KEY,
+  DefaultValue = "base64")
+  private String base64Field;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = BOOLEAN_KEY,
+  DefaultValue = false)
+  private boolean boolField;
+
+  public TestConfigurationServiceFieldsValidation() throws Exception {
+    super();
+    Base64 base64 = new Base64();
+    this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
+    this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
+    Configuration configuration = new Configuration();
+    configuration.addResource("azure-bfs-test.xml");
+    configuration.set(INT_KEY, "1234565");
+    configuration.set(LONG_KEY, "4194304");
+    configuration.set(STRING_KEY, "stringValue");
+    configuration.set(BASE64_KEY, encodedString);
+    configuration.set(BOOLEAN_KEY, "true");
+    configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "testaccount1.blob.core.windows.net", this.encodedAccountKey);
+    configService = new ConfigurationServiceImpl(configuration);
+  }
+
+  @Test
+  public void testValidateFunctionsInConfigServiceImpl() throws Exception {
+    Field[] fields = this.getClass().getDeclaredFields();
+    for (Field field : fields) {
+      field.setAccessible(true);
+      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
+        assertEquals(TEST_INT, configService.validateInt(field));
+      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
+        assertEquals(DEFAULT_LONG, configService.validateLong(field));
+      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
+        assertEquals("stringValue", configService.validateString(field));
+      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
+        assertEquals(this.encodedString, configService.validateBase64String(field));
+      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
+        assertEquals(true, configService.validateBoolean(field));
+      }
+    }
+  }
+
+  @Test
+  public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception {
+    // test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor
+    assertEquals(DEFAULT_WRITE_BUFFER_SIZE, configService.getWriteBufferSize());
+    assertEquals(DEFAULT_READ_BUFFER_SIZE, configService.getReadBufferSize());
+    assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, configService.getMinBackoffIntervalMilliseconds());
+    assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, configService.getMaxBackoffIntervalMilliseconds());
+    assertEquals(DEFAULT_BACKOFF_INTERVAL, configService.getBackoffIntervalMilliseconds());
+    assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, configService.getMaxIoRetries());
+    assertEquals(MAX_AZURE_BLOCK_SIZE, configService.getAzureBlockSize());
+    assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, configService.getAzureBlockLocationHost());
+  }
+
+  @Test
+  public void testGetAccountKey() throws Exception {
+    String accountKey = configService.getStorageAccountKey("testaccount1.blob.core.windows.net");
+    assertEquals(this.encodedAccountKey, accountKey);
+  }
+
+  @Test (expected = ConfigurationPropertyNotFoundException.class)
+  public void testGetAccountKeyWithNonExistingAccountName() throws Exception {
+    configService.getStorageAccountKey("bogusAccountName");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/package-info.java
new file mode 100644
index 0000000..97c1d71
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.services;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java
new file mode 100644
index 0000000..e71e31d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.utils;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test ABFS UriUtils.
+ */
+public final class TestUriUtils {
+  @Test
+  public void testIfUriContainsAbfs() throws Exception {
+    Assert.assertTrue(UriUtils.containsAbfsUrl("abfs.dfs.core.windows.net"));
+    Assert.assertTrue(UriUtils.containsAbfsUrl("abfs.dfs.preprod.core.windows.net"));
+    Assert.assertFalse(UriUtils.containsAbfsUrl("abfs.dfs.cores.windows.net"));
+    Assert.assertFalse(UriUtils.containsAbfsUrl(""));
+    Assert.assertFalse(UriUtils.containsAbfsUrl(null));
+    Assert.assertFalse(UriUtils.containsAbfsUrl("abfs.dfs.cores.windows.net"));
+    Assert.assertFalse(UriUtils.containsAbfsUrl("xhdfs.blob.core.windows.net"));
+  }
+
+  @Test
+  public void testExtractRawAccountName() throws Exception {
+    Assert.assertEquals("abfs", UriUtils.extractRawAccountFromAccountName("abfs.dfs.core.windows.net"));
+    Assert.assertEquals("abfs", UriUtils.extractRawAccountFromAccountName("abfs.dfs.preprod.core.windows.net"));
+    Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName("abfs.dfs.cores.windows.net"));
+    Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName(""));
+    Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName(null));
+    Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName("abfs.dfs.cores.windows.net"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java
new file mode 100644
index 0000000..d8cc940
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.utils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
new file mode 100644
index 0000000..caf8677
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
@@ -0,0 +1,64 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~       http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+    <property>
+        <name>fs.contract.test.root-tests-enabled</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-append</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-seek</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-overwrites-dest</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-returns-false-if-source-missing</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-creates-dest-dirs</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-settimes</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-concat</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-getfilestatus</name>
+        <value>true</value>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
new file mode 100644
index 0000000..508e5f6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+
+  <!--
+  <property>
+    <name>fs.azure.test.account.name</name>
+    <value>{YOURACCOUNT}</value>
+  </property>
+
+  <property>
+    <name>fs.azure.account.key.{YOURACCOUNT}.dfs.core.windows.net</name>
+    <value>{ACCOUNTKEY}</value>
+  </property>
+
+  <property>
+    <name>fs.azure.test.account.key.{YOURACCOUNT}.dfs.core.windows.net</name>
+    <value>{ACCOUNTKEY}</value>
+  </property>
+
+  <property>
+    <name>fs.contract.test.fs.abfs</name>
+    <value>abfs://{CONTAINERNAME}@{ACCOUNTNAME}.dfs.core.windows.net/value>
+    <description>The name of the azure file system for testing.</description>
+  </property>
+
+  -->
+  <!-- Save above configuration properties in a separate file named -->
+  <!-- azure-bfs-auth-keys.xml in the same directory as this file. -->
+  <!-- DO NOT ADD azure-bfs-auth-keys.xml TO REVISION CONTROL.  The keys to your -->
+  <!-- Azure Storage account are a secret and must not be shared. -->
+
+  <include xmlns="http://www.w3.org/2001/XInclude" href="azure-bfs-auth-keys.xml">
+    <fallback />
+  </include>
+
+  <!--<property>-->
+  <!--<name>fs.azure.test.host.name</name>-->
+  <!--<value>{HOSTNAME}</value>-->
+  <!--</property>-->
+
+  <!--<property>-->
+  <!--<name>fs.azure.test.host.port</name>-->
+  <!--<value>{PORT}</value>-->
+  <!--</property>-->
+
+  <!--<property>-->
+  <!--<name>fs.azure.abfs.emulator.enabled</name>-->
+  <!--<value>false</value>-->
+  <!--</property>-->
+
+  <property>
+    <name>fs.AbstractFileSystem.abfs.impl</name>
+    <value>org.apache.hadoop.fs.azurebfs.Abfs</value>
+  </property>
+
+  <property>
+    <name>fs.AbstractFileSystem.abfss.impl</name>
+    <value>org.apache.hadoop.fs.azurebfs.Abfss</value>
+  </property>
+
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
index a5e0c4f..fd0cd9d 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
@@ -24,3 +24,5 @@ log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t]: %c{2} (%F:
 
 log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG
 log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG
+log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE
+log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG


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


[41/50] [abbrv] hadoop git commit: HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java
new file mode 100644
index 0000000..6059766
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java
@@ -0,0 +1,132 @@
+/**
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest {
+  private static final int TEST_FILES_NUMBER = 6000;
+  public ITestAzureBlobFileSystemListStatus() {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final List<Future> tasks = new ArrayList<>();
+
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      final Path fileName = new Path("/test" + i);
+      Callable<Void> callable = new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          fs.create(fileName);
+          return null;
+        }
+      };
+
+      tasks.add(es.submit(callable));
+    }
+
+    for (Future<Void> task : tasks) {
+      task.get();
+    }
+
+    es.shutdownNow();
+    FileStatus[] files = fs.listStatus(new Path("/"));
+    Assert.assertEquals(files.length, TEST_FILES_NUMBER + 1 /* user directory */);
+  }
+
+  @Test
+  public void testListFileVsListDir() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(new Path("/testFile"));
+
+    FileStatus[] testFiles = fs.listStatus(new Path("/testFile"));
+    Assert.assertEquals(testFiles.length, 1);
+    Assert.assertFalse(testFiles[0].isDirectory());
+  }
+
+  @Test
+  public void testListFileVsListDir2() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    fs.mkdirs(new Path("/testFolder/testFolder2"));
+    fs.mkdirs(new Path("/testFolder/testFolder2/testFolder3"));
+    fs.create(new Path("/testFolder/testFolder2/testFolder3/testFile"));
+
+    FileStatus[] testFiles = fs.listStatus(new Path("/testFolder/testFolder2/testFolder3/testFile"));
+    Assert.assertEquals(testFiles.length, 1);
+    Assert.assertEquals(testFiles[0].getPath(), new Path(this.getTestUrl(),
+        "/testFolder/testFolder2/testFolder3/testFile"));
+    Assert.assertFalse(testFiles[0].isDirectory());
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testListNonExistentDir() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.listStatus(new Path("/testFile/"));
+  }
+
+  @Test
+  public void testListFiles() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("/test"));
+
+    FileStatus[] fileStatuses = fs.listStatus(new Path("/"));
+    assertEquals(fileStatuses.length, 2);
+
+    fs.mkdirs(new Path("/test/sub"));
+    fileStatuses = fs.listStatus(new Path("/test"));
+    assertEquals(fileStatuses.length, 1);
+    assertEquals(fileStatuses[0].getPath().getName(), "sub");
+    assertTrue(fileStatuses[0].isDirectory());
+    assertEquals(fileStatuses[0].getLen(), 0);
+
+    fs.create(new Path("/test/f"));
+    fileStatuses = fs.listStatus(new Path("/test"));
+    assertEquals(fileStatuses.length, 2);
+    assertEquals(fileStatuses[0].getPath().getName(), "f");
+    assertFalse(fileStatuses[0].isDirectory());
+    assertEquals(fileStatuses[0].getLen(), 0);
+    assertEquals(fileStatuses[1].getPath().getName(), "sub");
+    assertTrue(fileStatuses[1].isDirectory());
+    assertEquals(fileStatuses[1].getLen(), 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java
new file mode 100644
index 0000000..b61908c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.util.concurrent.Callable;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test mkdir operation.
+ */
+public class ITestAzureBlobFileSystemMkDir extends DependencyInjectedTest {
+  public ITestAzureBlobFileSystemMkDir() {
+    super();
+  }
+
+  @Test
+  public void testCreateDirWithExistingDir() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    assertTrue(fs.mkdirs(new Path("testFolder")));
+    assertTrue(fs.mkdirs(new Path("testFolder")));
+  }
+
+  @Test(expected = FileAlreadyExistsException.class)
+  public void createDirectoryUnderFile() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(new Path("testFile"));
+    fs.mkdirs(new Path("testFile/TestDirectory"));
+  }
+
+  @Test
+  public void testCreateDirectoryOverExistingFiles() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(new Path("/testPath"));
+    FileAlreadyExistsException ex = intercept(
+        FileAlreadyExistsException.class,
+        new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            fs.mkdirs(new Path("/testPath"));
+            return null;
+          }
+        });
+
+    assertTrue(ex instanceof FileAlreadyExistsException);
+
+    fs.create(new Path("/testPath1/file1"));
+    ex = intercept(
+        FileAlreadyExistsException.class,
+        new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            fs.mkdirs(new Path("/testPath1/file1"));
+            return null;
+          }
+        });
+
+    assertTrue(ex instanceof FileAlreadyExistsException);
+  }
+
+  @Test
+  public void testCreateRoot() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    assertTrue(fs.mkdirs(new Path("/")));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java
new file mode 100644
index 0000000..fef7f47
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java
@@ -0,0 +1,41 @@
+/**
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Test open operation.
+ */
+public class ITestAzureBlobFileSystemOpen extends DependencyInjectedTest {
+  public ITestAzureBlobFileSystemOpen() throws Exception {
+    super();
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testOpenDirectory() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("testFolder"));
+    fs.open(new Path("testFolder"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
new file mode 100644
index 0000000..9477587
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
@@ -0,0 +1,582 @@
+/**
+ * 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.azurebfs;
+
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Test;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertArrayEquals;
+
+
+/**
+ * Test random read operation.
+ */
+public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
+  private static final int KILOBYTE = 1024;
+  private static final int MEGABYTE = KILOBYTE * KILOBYTE;
+  private static final long TEST_FILE_SIZE = 8 * MEGABYTE;
+  private static final int MAX_ELAPSEDTIMEMS = 20;
+  private static final int SEQUENTIAL_READ_BUFFER_SIZE = 16 * KILOBYTE;
+  private static final int CREATE_BUFFER_SIZE = 26 * KILOBYTE;
+
+  private static final int SEEK_POSITION_ONE = 2* KILOBYTE;
+  private static final int SEEK_POSITION_TWO = 5 * KILOBYTE;
+  private static final int SEEK_POSITION_THREE = 10 * KILOBYTE;
+  private static final int SEEK_POSITION_FOUR = 4100 * KILOBYTE;
+
+  private static final Path TEST_FILE_PATH = new Path(
+            "/TestRandomRead.txt");
+  private static final String WASB = "WASB";
+  private static final String ABFS = "ABFS";
+  private static long testFileLength = 0;
+
+  public ITestAzureBlobFileSystemRandomRead() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testBasicRead() throws Exception {
+    assumeHugeFileExists();
+
+    try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
+      byte[] buffer = new byte[3 * MEGABYTE];
+
+      // forward seek and read a kilobyte into first kilobyte of bufferV2
+      inputStream.seek(5 * MEGABYTE);
+      int numBytesRead = inputStream.read(buffer, 0, KILOBYTE);
+      assertEquals(KILOBYTE, numBytesRead);
+
+      int len = MEGABYTE;
+      int offset = buffer.length - len;
+
+      // reverse seek and read a megabyte into last megabyte of bufferV1
+      inputStream.seek(3 * MEGABYTE);
+      numBytesRead = inputStream.read(buffer, offset, len);
+      assertEquals(len, numBytesRead);
+    }
+  }
+
+  /**
+   * Validates the implementation of random read in ABFS
+   * @throws IOException
+   */
+  @Test
+  public void testRandomRead() throws Exception {
+    assumeHugeFileExists();
+    try (
+            FSDataInputStream inputStreamV1
+                    = this.getFileSystem().open(TEST_FILE_PATH);
+            FSDataInputStream inputStreamV2
+                    = this.getWasbFileSystem().open(TEST_FILE_PATH);
+    ) {
+      final int bufferSize = 4 * KILOBYTE;
+      byte[] bufferV1 = new byte[bufferSize];
+      byte[] bufferV2 = new byte[bufferV1.length];
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      inputStreamV1.seek(0);
+      inputStreamV2.seek(0);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      inputStreamV1.seek(SEEK_POSITION_ONE);
+      inputStreamV2.seek(SEEK_POSITION_ONE);
+
+      inputStreamV1.seek(0);
+      inputStreamV2.seek(0);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      inputStreamV1.seek(SEEK_POSITION_TWO);
+      inputStreamV2.seek(SEEK_POSITION_TWO);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      inputStreamV1.seek(SEEK_POSITION_THREE);
+      inputStreamV2.seek(SEEK_POSITION_THREE);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      inputStreamV1.seek(SEEK_POSITION_FOUR);
+      inputStreamV2.seek(SEEK_POSITION_FOUR);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+    }
+  }
+
+  /**
+   * Validates the implementation of Seekable.seekToNewSource
+   * @throws IOException
+   */
+  @Test
+  public void testSeekToNewSource() throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
+      assertFalse(inputStream.seekToNewSource(0));
+    }
+  }
+
+  /**
+   * Validates the implementation of InputStream.skip and ensures there is no
+   * network I/O for AbfsInputStream
+   * @throws Exception
+   */
+  @Test
+  public void testSkipBounds() throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
+      ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+
+      long skipped = inputStream.skip(-1);
+      assertEquals(0, skipped);
+
+      skipped = inputStream.skip(0);
+      assertEquals(0, skipped);
+
+      assertTrue(testFileLength > 0);
+
+      skipped = inputStream.skip(testFileLength);
+      assertEquals(testFileLength, skipped);
+
+      intercept(EOFException.class,
+              new Callable<Long>() {
+                @Override
+                public Long call() throws Exception {
+                  return inputStream.skip(1);
+                }
+              }
+      );
+      long elapsedTimeMs = timer.elapsedTimeMs();
+      assertTrue(
+              String.format(
+                      "There should not be any network I/O (elapsedTimeMs=%1$d).",
+                      elapsedTimeMs),
+              elapsedTimeMs < MAX_ELAPSEDTIMEMS);
+    }
+  }
+
+  /**
+   * Validates the implementation of Seekable.seek and ensures there is no
+   * network I/O for forward seek.
+   * @throws Exception
+   */
+  @Test
+  public void testValidateSeekBounds() throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
+      ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+
+      inputStream.seek(0);
+      assertEquals(0, inputStream.getPos());
+
+      intercept(EOFException.class,
+              FSExceptionMessages.NEGATIVE_SEEK,
+              new Callable<FSDataInputStream>() {
+                @Override
+                public FSDataInputStream call() throws Exception {
+                  inputStream.seek(-1);
+                  return inputStream;
+                }
+              }
+      );
+
+      assertTrue("Test file length only " + testFileLength, testFileLength > 0);
+      inputStream.seek(testFileLength);
+      assertEquals(testFileLength, inputStream.getPos());
+
+      intercept(EOFException.class,
+              FSExceptionMessages.CANNOT_SEEK_PAST_EOF,
+              new Callable<FSDataInputStream>() {
+                @Override
+                public FSDataInputStream call() throws Exception {
+                  inputStream.seek(testFileLength + 1);
+                  return inputStream;
+                }
+              }
+      );
+
+      long elapsedTimeMs = timer.elapsedTimeMs();
+      assertTrue(
+              String.format(
+                      "There should not be any network I/O (elapsedTimeMs=%1$d).",
+                      elapsedTimeMs),
+              elapsedTimeMs < MAX_ELAPSEDTIMEMS);
+    }
+  }
+
+  /**
+   * Validates the implementation of Seekable.seek, Seekable.getPos,
+   * and InputStream.available.
+   * @throws Exception
+   */
+  @Test
+  public void testSeekAndAvailableAndPosition() throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
+      byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
+      byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
+      byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
+      byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'};
+      byte[] buffer = new byte[3];
+
+      int bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected1, buffer);
+      assertEquals(buffer.length, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected2, buffer);
+      assertEquals(2 * buffer.length, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+
+      // reverse seek
+      int seekPos = 0;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected1, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+
+      // reverse seek
+      seekPos = 1;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected3, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+
+      // forward seek
+      seekPos = 6;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected4, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+    }
+  }
+
+  /**
+   * Validates the implementation of InputStream.skip, Seekable.getPos,
+   * and InputStream.available.
+   * @throws IOException
+   */
+  @Test
+  public void testSkipAndAvailableAndPosition() throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
+      byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
+      byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
+      byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
+      byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'};
+
+      assertEquals(testFileLength, inputStream.available());
+      assertEquals(0, inputStream.getPos());
+
+      int n = 3;
+      long skipped = inputStream.skip(n);
+
+      assertEquals(skipped, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+      assertEquals(skipped, n);
+
+      byte[] buffer = new byte[3];
+      int bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected2, buffer);
+      assertEquals(buffer.length + skipped, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+
+      // does skip still work after seek?
+      int seekPos = 1;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected3, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+
+      long currentPosition = inputStream.getPos();
+      n = 2;
+      skipped = inputStream.skip(n);
+
+      assertEquals(currentPosition + skipped, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+      assertEquals(skipped, n);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected4, buffer);
+      assertEquals(buffer.length + skipped + currentPosition,
+              inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+              inputStream.available());
+    }
+  }
+
+  /**
+   * Ensures parity in the performance of sequential read after reverse seek for
+   * abfs of the AbfsInputStream.
+   * @throws IOException
+   */
+  @Test
+  public void testSequentialReadAfterReverseSeekPerformance()
+          throws Exception {
+    assumeHugeFileExists();
+    final int maxAttempts = 10;
+    final double maxAcceptableRatio = 1.01;
+    double beforeSeekElapsedMs = 0, afterSeekElapsedMs = 0;
+    double ratio = Double.MAX_VALUE;
+    for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
+      beforeSeekElapsedMs = sequentialRead(ABFS,
+              this.getFileSystem(), false);
+      afterSeekElapsedMs = sequentialRead(ABFS,
+              this.getFileSystem(), true);
+      ratio = afterSeekElapsedMs / beforeSeekElapsedMs;
+      System.out.println((String.format(
+              "beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d, ratio=%3$.2f",
+              (long) beforeSeekElapsedMs,
+              (long) afterSeekElapsedMs,
+              ratio)));
+    }
+    assertTrue(String.format(
+            "Performance of ABFS stream after reverse seek is not acceptable:"
+                    + " beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d,"
+                    + " ratio=%3$.2f",
+            (long) beforeSeekElapsedMs,
+            (long) afterSeekElapsedMs,
+            ratio),
+            ratio < maxAcceptableRatio);
+  }
+
+  @Test
+  public void testRandomReadPerformance() throws Exception {
+    createTestFile();
+    assumeHugeFileExists();
+
+    final AzureBlobFileSystem abFs = this.getFileSystem();
+    final NativeAzureFileSystem wasbFs = this.getWasbFileSystem();
+
+    final int maxAttempts = 10;
+    final double maxAcceptableRatio = 1.025;
+    double v1ElapsedMs = 0, v2ElapsedMs = 0;
+    double ratio = Double.MAX_VALUE;
+    for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
+      v1ElapsedMs = randomRead(1, wasbFs);
+      v2ElapsedMs = randomRead(2, abFs);
+
+      ratio = v2ElapsedMs / v1ElapsedMs;
+
+      System.out.println(String.format(
+              "v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f",
+              (long) v1ElapsedMs,
+              (long) v2ElapsedMs,
+              ratio));
+    }
+    assertTrue(String.format(
+            "Performance of version 2 is not acceptable: v1ElapsedMs=%1$d,"
+                    + " v2ElapsedMs=%2$d, ratio=%3$.2f",
+            (long) v1ElapsedMs,
+            (long) v2ElapsedMs,
+            ratio),
+            ratio < maxAcceptableRatio);
+  }
+
+
+  private long sequentialRead(String version,
+                              FileSystem fs,
+                              boolean afterReverseSeek) throws IOException {
+    byte[] buffer = new byte[SEQUENTIAL_READ_BUFFER_SIZE];
+    long totalBytesRead = 0;
+    long bytesRead = 0;
+
+    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      if (afterReverseSeek) {
+        while (bytesRead > 0 && totalBytesRead < 4 * MEGABYTE) {
+          bytesRead = inputStream.read(buffer);
+          totalBytesRead += bytesRead;
+        }
+        totalBytesRead = 0;
+        inputStream.seek(0);
+      }
+
+      ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+      while ((bytesRead = inputStream.read(buffer)) > 0) {
+        totalBytesRead += bytesRead;
+      }
+      long elapsedTimeMs = timer.elapsedTimeMs();
+
+      System.out.println(String.format(
+              "v%1$s: bytesRead=%2$d, elapsedMs=%3$d, Mbps=%4$.2f,"
+                      + " afterReverseSeek=%5$s",
+              version,
+              totalBytesRead,
+              elapsedTimeMs,
+              toMbps(totalBytesRead, elapsedTimeMs),
+              afterReverseSeek));
+
+      assertEquals(testFileLength, totalBytesRead);
+      inputStream.close();
+      return elapsedTimeMs;
+    }
+  }
+
+  private long randomRead(int version, FileSystem fs) throws Exception {
+    assumeHugeFileExists();
+    final long minBytesToRead = 2 * MEGABYTE;
+    Random random = new Random();
+    byte[] buffer = new byte[8 * KILOBYTE];
+    long totalBytesRead = 0;
+    long bytesRead = 0;
+    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+      do {
+        bytesRead = inputStream.read(buffer);
+        totalBytesRead += bytesRead;
+        inputStream.seek(random.nextInt(
+                (int) (TEST_FILE_SIZE - buffer.length)));
+        } while (bytesRead > 0 && totalBytesRead < minBytesToRead);
+      long elapsedTimeMs = timer.elapsedTimeMs();
+      inputStream.close();
+      System.out.println(String.format(
+              "v%1$d: totalBytesRead=%2$d, elapsedTimeMs=%3$d, Mbps=%4$.2f",
+              version,
+              totalBytesRead,
+              elapsedTimeMs,
+              toMbps(totalBytesRead, elapsedTimeMs)));
+      assertTrue(minBytesToRead <= totalBytesRead);
+      return elapsedTimeMs;
+    }
+  }
+
+  /**
+   * Calculate megabits per second from the specified values for bytes and
+   * milliseconds.
+   * @param bytes The number of bytes.
+   * @param milliseconds The number of milliseconds.
+   * @return The number of megabits per second.
+   */
+  private static double toMbps(long bytes, long milliseconds) {
+    return bytes / 1000.0 * 8 / milliseconds;
+  }
+
+  private void createTestFile() throws Exception {
+    FileSystem fs = this.getWasbFileSystem();
+
+    if (fs.exists(TEST_FILE_PATH)) {
+      FileStatus status = fs.getFileStatus(TEST_FILE_PATH);
+      if (status.getLen() >= TEST_FILE_SIZE) {
+        return;
+      }
+    }
+
+    byte[] buffer = new byte[CREATE_BUFFER_SIZE];
+    char character = 'a';
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) character;
+      character = (character == 'z') ? 'a' : (char) ((int) character + 1);
+    }
+
+    System.out.println(("Creating test file {} of size: {} " + TEST_FILE_PATH
+            + TEST_FILE_SIZE));
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+
+    try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
+      int bytesWritten = 0;
+      while (bytesWritten < TEST_FILE_SIZE) {
+        outputStream.write(buffer);
+        bytesWritten += buffer.length;
+      }
+      System.out.println("Closing stream {}" +  outputStream);
+      ContractTestUtils.NanoTimer closeTimer
+              = new ContractTestUtils.NanoTimer();
+      outputStream.close();
+      closeTimer.end("time to close() output stream");
+    }
+    timer.end("time to write %d KB", TEST_FILE_SIZE / 1024);
+    testFileLength = fs.getFileStatus(TEST_FILE_PATH).getLen();
+
+  }
+
+  private void assumeHugeFileExists() throws Exception{
+    createTestFile();
+    FileSystem fs = this.getFileSystem();
+    ContractTestUtils.assertPathExists(this.getFileSystem(), "huge file not created", TEST_FILE_PATH);
+    FileStatus status = fs.getFileStatus(TEST_FILE_PATH);
+    ContractTestUtils.assertIsFile(TEST_FILE_PATH, status);
+    assertTrue("File " + TEST_FILE_PATH + " is empty", status.getLen() > 0);
+  }
+
+  private void verifyConsistentReads(FSDataInputStream inputStreamV1,
+                                     FSDataInputStream inputStreamV2,
+                                     byte[] bufferV1,
+                                     byte[] bufferV2) throws IOException {
+    int size = bufferV1.length;
+    final int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, size);
+    assertEquals("Bytes read from wasb stream", size, numBytesReadV1);
+
+    final int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, size);
+    assertEquals("Bytes read from abfs stream", size, numBytesReadV2);
+
+    assertArrayEquals("Mismatch in read data", bufferV1, bufferV2);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
new file mode 100644
index 0000000..a0e648c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
@@ -0,0 +1,152 @@
+/**
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test rename operation.
+ */
+public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest {
+  public ITestAzureBlobFileSystemRename() {
+    super();
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testEnsureFileIsRenamed() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(new Path("testfile"));
+    fs.rename(new Path("testfile"), new Path("testfile2"));
+
+    FileStatus fileStatus = fs.getFileStatus(new Path("testfile2"));
+    assertNotNull(fileStatus);
+
+    fs.getFileStatus(new Path("testfile"));
+  }
+
+  @Test
+  public void testRenameFile() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("/testSrc"));
+    fs.create(new Path("/testSrc/file1"));
+
+    fs.rename(new Path("/testSrc"), new Path("/testDst"));
+    FileStatus[] fileStatus = fs.listStatus(new Path("/testDst"));
+    assertNotNull(fileStatus);
+  }
+
+  @Test
+  public void testRenameFileUsingUnicode() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    //known issue: ListStatus operation to folders/files whose name contains '?' will fail
+    //This is because Auto rest client didn't encode '?' in the uri query parameters
+    String[] folders1 = new String[]{"/%2c%26", "/ÖáΠ⇒", "/A +B", "/A~`!@#$%^&*()-_+={};:'>,<B"};
+    String[] folders2 = new String[]{"/abcÖ⇒123", "/abcÖáΠ⇒123", "/B+ C", "/B~`!@#$%^&*()-_+={};:'>,<C"};
+    String[] files = new String[]{"/%2c%27", "/中文", "/C +D", "/C~`!@#$%^&*()-_+={};:'>,<D"};
+
+    for (int i = 0; i < 4; i++) {
+      Path folderPath1 = new Path(folders1[i]);
+      assertTrue(fs.mkdirs(folderPath1));
+      assertTrue(fs.exists(folderPath1));
+
+      Path filePath = new Path(folders1[i] + files[i]);
+      fs.create(filePath);
+      assertTrue(fs.exists(filePath));
+
+      Path folderPath2 = new Path(folders2[i]);
+      fs.rename(folderPath1, folderPath2);
+      assertFalse(fs.exists(folderPath1));
+      assertTrue(fs.exists(folderPath2));
+
+      FileStatus[] fileStatus = fs.listStatus(folderPath2);
+      assertEquals("/" + fileStatus[0].getPath().getName(), files[i]);
+      assertNotNull(fileStatus);
+    }
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testRenameDirectory() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("testDir"));
+    fs.mkdirs(new Path("testDir/test1"));
+    fs.mkdirs(new Path("testDir/test1/test2"));
+    fs.mkdirs(new Path("testDir/test1/test2/test3"));
+
+    Assert.assertTrue(fs.rename(new Path("testDir/test1"), new Path("testDir/test10")));
+    fs.getFileStatus(new Path("testDir/test1"));
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testRenameFirstLevelDirectory() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    final List<Future> tasks = new ArrayList<>();
+
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < 1000; i++) {
+      final Path fileName = new Path("/test/" + i);
+      Callable<Void> callable = new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          fs.create(fileName);
+          return null;
+        }
+      };
+
+      tasks.add(es.submit(callable));
+    }
+
+    for (Future<Void> task : tasks) {
+      task.get();
+    }
+
+    es.shutdownNow();
+    fs.rename(new Path("/test"), new Path("/renamedDir"));
+
+    FileStatus[] files = fs.listStatus(new Path("/renamedDir"));
+    Assert.assertEquals(files.length, 1000);
+    fs.getFileStatus(new Path("/test"));
+  }
+
+  @Test
+  public void testRenameRoot() throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    boolean renamed = fs.rename(new Path("/"), new Path("/ddd"));
+    assertFalse(renamed);
+
+    renamed = fs.rename(new Path(fs.getUri().toString() + "/"), new Path(fs.getUri().toString() + "/s"));
+    assertFalse(renamed);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
new file mode 100644
index 0000000..aa30a85
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
@@ -0,0 +1,78 @@
+/**
+ * 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.azurebfs;
+
+import java.net.URI;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
+import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
+
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.doReturn;
+
+/**
+ * Test AzureBlobFileSystem initialization.
+ */
+public class ITestFileSystemInitialization extends DependencyInjectedTest {
+  public ITestFileSystemInitialization() {
+    super();
+
+    this.getMockServiceInjector().removeProvider(AbfsHttpService.class);
+    this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class));
+  }
+
+  @Test
+  public void ensureAzureBlobFileSystemIsInitialized() throws Exception {
+    doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
+        .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
+        .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
+
+    final FileSystem fs = FileSystem.get(this.getConfiguration());
+    final String accountName = this.getAccountName();
+    final String filesystem = this.getFileSystemName();
+
+    Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SCHEME, filesystem + "@" + accountName, null, null, null));
+    Assert.assertNotNull(fs.getWorkingDirectory());
+  }
+
+  @Test
+  public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception {
+    doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
+        .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
+        .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
+
+    final String accountName = this.getAccountName();
+    final String filesystem = this.getFileSystemName();
+    final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null);
+    this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
+
+    final FileSystem fs = FileSystem.get(this.getConfiguration());
+    Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null));
+    Assert.assertNotNull(fs.getWorkingDirectory());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
new file mode 100644
index 0000000..a55599b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
@@ -0,0 +1,82 @@
+/**
+ * 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.azurebfs;
+
+import java.net.URI;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.AbstractFileSystem;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
+
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.doReturn;
+
+/**
+ * Test AzureBlobFileSystem registration.
+ */
+public class ITestFileSystemRegistration extends DependencyInjectedTest {
+  public ITestFileSystemRegistration() throws Exception {
+    super();
+
+    this.getMockServiceInjector().removeProvider(AbfsHttpService.class);
+    this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class));
+  }
+
+  @Test
+  public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
+    doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
+        .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
+        .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
+
+    FileSystem fs = FileSystem.get(this.getConfiguration());
+    Assert.assertTrue(fs instanceof AzureBlobFileSystem);
+
+    AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem();
+    Assert.assertTrue(afs instanceof Abfs);
+  }
+
+  @Test
+  public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
+    doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
+        .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
+        .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
+
+    final String accountName = this.getAccountName();
+    final String filesystem = this.getFileSystemName();
+
+    final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null);
+    this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
+
+    FileSystem fs = FileSystem.get(this.getConfiguration());
+    Assert.assertTrue(fs instanceof SecureAzureBlobFileSystem);
+
+    AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem();
+    Assert.assertTrue(afs instanceof Abfss);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
new file mode 100644
index 0000000..7010e74
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -0,0 +1,202 @@
+/**
+ * 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.azurebfs;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertFalse;
+import static junit.framework.TestCase.assertTrue;
+
+/**
+ * Test compatibility between ABFS client and WASB client.
+ */
+public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
+  private static final String WASB_TEST_CONTEXT = "wasb test file";
+  private static final String ABFS_TEST_CONTEXT = "abfs test file";
+  private static final String TEST_CONTEXT = "THIS IS FOR TEST";
+
+  public ITestWasbAbfsCompatibility() throws Exception {
+    super();
+
+    Assume.assumeFalse(this.isEmulator());
+  }
+
+  @Test
+  public void testListFileStatus() throws Exception {
+    // crate file using abfs
+    AzureBlobFileSystem fs = this.getFileSystem();
+    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+
+    Path path1 = new Path("/testfiles/~12/!008/3/abFsTestfile");
+    FSDataOutputStream abfsStream = fs.create(path1, true);
+    abfsStream.write(ABFS_TEST_CONTEXT.getBytes());
+    abfsStream.flush();
+    abfsStream.hsync();
+    abfsStream.close();
+
+    // create file using wasb
+    Path path2 = new Path("/testfiles/~12/!008/3/nativeFsTestfile");
+    System.out.println(wasb.getUri());
+    FSDataOutputStream nativeFsStream = wasb.create(path2, true);
+    nativeFsStream.write(WASB_TEST_CONTEXT.getBytes());
+    nativeFsStream.flush();
+    nativeFsStream.hsync();
+    nativeFsStream.close();
+    // list file using abfs and wasb
+    FileStatus[] abfsFileStatus = fs.listStatus(new Path("/testfiles/~12/!008/3/"));
+    FileStatus[] nativeFsFileStatus = wasb.listStatus(new Path("/testfiles/~12/!008/3/"));
+
+    assertEquals(2, abfsFileStatus.length);
+    assertEquals(2, nativeFsFileStatus.length);
+  }
+
+  @Test
+  public void testReadFile() throws Exception {
+    boolean[] createFileWithAbfs = new boolean[]{false, true, false, true};
+    boolean[] readFileWithAbfs = new boolean[]{false, true, true, false};
+
+    AzureBlobFileSystem abfs = this.getFileSystem();
+    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+
+    FileSystem fs;
+    BufferedReader br = null;
+    for (int i = 0; i< 4; i++) {
+      try {
+        Path path = new Path("/testfiles/~12/!008/testfile" + i);
+        if (createFileWithAbfs[i]) {
+          fs = abfs;
+        } else {
+          fs = wasb;
+        }
+
+        // Write
+        FSDataOutputStream nativeFsStream = fs.create(path, true);
+        nativeFsStream.write(TEST_CONTEXT.getBytes());
+        nativeFsStream.flush();
+        nativeFsStream.hsync();
+        nativeFsStream.close();
+
+        // Check file status
+        assertEquals(true, fs.exists(path));
+        assertEquals(false, fs.getFileStatus(path).isDirectory());
+
+        // Read
+        if (readFileWithAbfs[i]) {
+          fs = abfs;
+        } else {
+          fs = wasb;
+        }
+        FSDataInputStream inputStream = fs.open(path);
+        br = new BufferedReader(new InputStreamReader(fs.open(path)));
+        String line = br.readLine();
+        assertEquals(TEST_CONTEXT, line);
+
+        // Remove file
+        fs.delete(path, true);
+        assertFalse(fs.exists(path));
+      } catch (Exception e) {
+        e.printStackTrace();
+      } finally {
+        if (br != null) {
+          br.close();
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testDir() throws Exception {
+    boolean[] createDirWithAbfs = new boolean[]{false, true, false, true};
+    boolean[] readDirWithAbfs = new boolean[]{false, true, true, false};
+
+    AzureBlobFileSystem abfs = this.getFileSystem();
+    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+
+    FileSystem fs;
+    for (int i = 0; i < 4; i++) {
+      Path path = new Path("/testDir/t" + i);
+      //create
+      if (createDirWithAbfs[i]) {
+        fs = abfs;
+      } else {
+        fs = wasb;
+      }
+      assertTrue(fs.mkdirs(path));
+      //check
+      assertTrue(fs.exists(path));
+      //read
+      if (readDirWithAbfs[i]) {
+        fs = abfs;
+      } else {
+        fs = wasb;
+      }
+      assertTrue(fs.exists(path));
+      FileStatus dirStatus = fs.getFileStatus(path);
+      assertTrue(dirStatus.isDirectory());
+      fs.delete(path, true);
+      assertFalse(fs.exists(path));
+    }
+  }
+
+
+  @Test
+  public void testUrlConversion(){
+    String abfsUrl = "abfs://abcde-1111-1111-1111-1111@xxxx.dfs.xxx.xxx.xxxx.xxxx";
+    String wabsUrl = "wasb://abcde-1111-1111-1111-1111@xxxx.blob.xxx.xxx.xxxx.xxxx";
+    Assert.assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl));
+    Assert.assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl));
+  }
+
+  @Test
+  public void testSetWorkingDirectory() throws Exception {
+    //create folders
+    AzureBlobFileSystem abfs = this.getFileSystem();
+    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+
+    assertTrue(abfs.mkdirs(new Path("/d1/d2/d3/d4")));
+
+    //set working directory to path1
+    Path path1 = new Path("/d1/d2");
+    wasb.setWorkingDirectory(path1);
+    abfs.setWorkingDirectory(path1);
+    assertEquals(path1, wasb.getWorkingDirectory());
+    assertEquals(path1, abfs.getWorkingDirectory());
+
+    //set working directory to path2
+    Path path2 = new Path("d3/d4");
+    wasb.setWorkingDirectory(path2);
+    abfs.setWorkingDirectory(path2);
+
+    Path path3 = new Path("/d1/d2/d3/d4");
+    assertEquals(path3, wasb.getWorkingDirectory());
+    assertEquals(path3, abfs.getWorkingDirectory());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
new file mode 100644
index 0000000..4b44765
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
@@ -0,0 +1,37 @@
+/**
+ * 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.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class TestConfigurationKeys {
+  public static final String FS_AZURE_TEST_ACCOUNT_NAME = "fs.azure.test.account.name";
+  public static final String FS_AZURE_TEST_ACCOUNT_KEY_PREFIX = "fs.azure.test.account.key.";
+  public static final String FS_AZURE_TEST_HOST_NAME = "fs.azure.test.host.name";
+  public static final String FS_AZURE_TEST_HOST_PORT = "fs.azure.test.host.port";
+  public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs";
+
+  private TestConfigurationKeys() {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java
new file mode 100644
index 0000000..109f887
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.constants;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java
new file mode 100644
index 0000000..5fc81ce
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java
@@ -0,0 +1,63 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
+
+/**
+ * Dependency inject for ABFS contract tests.
+ */
+public class DependencyInjectedContractTest extends DependencyInjectedTest {
+  private final URI testUri;
+
+  public DependencyInjectedContractTest(final boolean secure) throws Exception {
+    this(secure, true);
+  }
+
+  public DependencyInjectedContractTest(final boolean secure, final boolean useExistedFileSystem) throws Exception{
+    super(secure);
+    if (useExistedFileSystem) {
+      Configuration configuration = getConfiguration();
+      String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI);
+
+      if (secure) {
+        testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME);
+      }
+      updateTestUrl(testUrl);
+
+      this.testUri = new URI(testUrl);
+      //Get container for contract tests
+      configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString());
+      String[] splitAuthority = this.testUri.getAuthority().split("\\@");
+      updateFileSystemName(splitAuthority[0]);
+    } else {
+      this.testUri = new URI(super.getTestUrl());
+    }
+  }
+
+  public Configuration getConfiguration() {
+    return super.getConfiguration();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java
new file mode 100644
index 0000000..7f7a09a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java
@@ -0,0 +1,54 @@
+/**
+ * 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.azurebfs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+
+/**
+ * Azure BlobFileSystem Contract. Test paths are created using any maven fork
+ * identifier, if defined. This guarantees paths unique to tests
+ * running in parallel.
+ */
+public class ITestAbfsFileSystemContract extends AbstractBondedFSContract {
+
+  public static final String CONTRACT_XML = "abfs.xml";
+  private final boolean isSecure;
+
+  protected ITestAbfsFileSystemContract(final Configuration conf, boolean secure) {
+    super(conf);
+    //insert the base features
+    addConfResource(CONTRACT_XML);
+    this.isSecure = secure;
+  }
+
+  @Override
+  public String getScheme() {
+    return isSecure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME;
+  }
+
+  @Override
+  public Path getTestPath() {
+    Path path = new Path(UriUtils.generateUniqueTestPath());
+    return path;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
new file mode 100644
index 0000000..d4cca14
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
@@ -0,0 +1,70 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+
+/**
+ * Contract test for open operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTest {
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+
+  @Override
+  public void testRenameFileBeingAppended() throws Throwable {
+    skip("Skipping as renaming an opened file is not supported");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
new file mode 100644
index 0000000..4f724e2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
@@ -0,0 +1,62 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractConcatTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for concat operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTest{
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
new file mode 100644
index 0000000..16b959f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
@@ -0,0 +1,63 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for create operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractCreate extends AbstractContractCreateTest{
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractCreate(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
new file mode 100644
index 0000000..fabd3273
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
@@ -0,0 +1,63 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for delete operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractDelete extends AbstractContractDeleteTest {
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractDelete(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
new file mode 100644
index 0000000..a1360e4
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
@@ -0,0 +1,44 @@
+/**
+ * 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.azurebfs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
+
+/**
+ * Contract test for distCp operation.
+ */
+public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTest {
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractDistCp() throws Exception {
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(false);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected ITestAbfsFileSystemContract createContract(Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
new file mode 100644
index 0000000..5bb41ad
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
@@ -0,0 +1,62 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for getFileStatus operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractGetFileStatus extends AbstractContractGetFileStatusTest {
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractGetFileStatus(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
new file mode 100644
index 0000000..9d732d5
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
@@ -0,0 +1,63 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for mkdir operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractMkdir extends AbstractContractMkdirTest {
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractMkdir(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(secure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
new file mode 100644
index 0000000..a71149b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
@@ -0,0 +1,63 @@
+/**
+ * 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.azurebfs.contract;
+
+import java.util.Arrays;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for open operation.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsFileSystemContractOpen extends AbstractContractOpenTest {
+  @Parameterized.Parameters(name = "SecureMode={0}")
+  public static Iterable<Object[]> secure() {
+    return Arrays.asList(new Object[][] { {true}, {false} });
+  }
+
+  private final boolean isSecure;
+  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+
+  public ITestAbfsFileSystemContractOpen(final boolean secure) throws Exception {
+    this.isSecure = secure;
+    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    dependencyInjectedContractTest.initialize();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return this.dependencyInjectedContractTest.getConfiguration();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(final Configuration conf) {
+    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+  }
+}
\ 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


[33/50] [abbrv] hadoop git commit: Remove super user privilege requirement for NameNode.getServiceStatus. Contributed by Chao Sun.

Posted by tm...@apache.org.
Remove super user privilege requirement for NameNode.getServiceStatus. Contributed by Chao Sun.

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

Branch: refs/heads/HADOOP-15407
Commit: a2a8c486998b81d2c73804a07cc74f5269bfd904
Parents: e7951c6
Author: Chao Sun <su...@apache.org>
Authored: Fri Aug 10 15:59:39 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Aug 10 15:59:39 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java  | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2a8c486/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 8ad5767..fd856a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -1757,7 +1757,6 @@ public class NameNode extends ReconfigurableBase implements
 
   synchronized HAServiceStatus getServiceStatus()
       throws ServiceFailedException, AccessControlException {
-    namesystem.checkSuperuserPrivilege();
     if (!haEnabled) {
       throw new ServiceFailedException("HA for namenode is not enabled");
     }


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


[47/50] [abbrv] hadoop git commit: HADOOP-15560. ABFS: removed dependency injection and unnecessary dependencies. Contributed by Da Zhou.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
index e148a05..9f3b4a7 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -37,7 +38,7 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
   private final DependencyInjectedContractTest dependencyInjectedContractTest;
 
   public ITestAzureBlobFileSystemBasics() throws Exception {
-    // If contract tests are running in parallel, some root level tests in this file will fail
+    // If all contract tests are running in parallel, some root level tests in FileSystemContractBaseTest will fail
     // due to the race condition. Hence for this contract test it should be tested in different container
     dependencyInjectedContractTest = new DependencyInjectedContractTest(false, false);
   }
@@ -48,6 +49,14 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
     fs = this.dependencyInjectedContractTest.getFileSystem();
   }
 
+  @After
+  public void testCleanup() throws Exception {
+    // This contract test is not using existing container for test,
+    // instead it creates its own temp container for test, hence we need to destroy
+    // it after the test.
+    this.dependencyInjectedContractTest.testCleanup();
+  }
+
   @Test
   public void testListOnFolderWithNoChildren() throws IOException {
     assertTrue(fs.mkdirs(path("testListStatus/c/1")));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java
deleted file mode 100644
index 588df20..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java
+++ /dev/null
@@ -1,122 +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.fs.azurebfs.services;
-
-import java.util.Hashtable;
-
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
-import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test AbfsHttpServiceImpl.
- */
-public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest {
-  private static final int TEST_DATA = 100;
-  private static final Path TEST_PATH = new Path("/testfile");
-  public ITestAbfsHttpServiceImpl() {
-    super();
-  }
-
-  @Test
-  public void testReadWriteBytesToFileAndEnsureThreadPoolCleanup() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    testWriteOneByteToFileAndEnsureThreadPoolCleanup();
-
-    FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024);
-    int i = inputStream.read();
-
-    assertEquals(TEST_DATA, i);
-  }
-
-  @Test
-  public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    FSDataOutputStream stream = fs.create(TEST_PATH);
-
-    stream.write(TEST_DATA);
-    stream.close();
-
-    FileStatus fileStatus = fs.getFileStatus(TEST_PATH);
-    assertEquals(1, fileStatus.getLen());
-  }
-
-  @Test
-  @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant")
-  public void testBase64FileSystemProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final Hashtable<String, String> properties = new Hashtable<>();
-    properties.put("key", "{ value: value }");
-    AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties(
-        fs, properties);
-    Hashtable<String, String> fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs);
-
-    Assert.assertEquals(properties, fetchedProperties);
-  }
-
-  @Test
-  public void testBase64PathProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final Hashtable<String, String> properties = new Hashtable<>();
-    properties.put("key", "{ value: valueTest }");
-    fs.create(TEST_PATH);
-    AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties(
-        fs, TEST_PATH, properties);
-    Hashtable<String, String> fetchedProperties =
-        AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH);
-
-    Assert.assertEquals(properties, fetchedProperties);
-  }
-
-  @Test (expected = Exception.class)
-  public void testBase64InvalidFileSystemProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final Hashtable<String, String> properties = new Hashtable<>();
-    properties.put("key", "{ value: value歲 }");
-    AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties(
-        fs, properties);
-    Hashtable<String, String> fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs);
-
-    Assert.assertEquals(properties, fetchedProperties);
-  }
-
-  @Test (expected = Exception.class)
-  public void testBase64InvalidPathProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final Hashtable<String, String> properties = new Hashtable<>();
-    properties.put("key", "{ value: valueTest兩 }");
-    fs.create(TEST_PATH);
-    AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties(
-        fs, TEST_PATH, properties);
-    Hashtable<String, String> fetchedProperties =
-        AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH);
-
-    Assert.assertEquals(properties, fetchedProperties);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
index 1db93cb..cabe049 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
 
 import org.junit.Test;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
@@ -52,12 +51,11 @@ public class ITestReadWriteAndSeek extends DependencyInjectedTest {
 
   private void testReadWriteAndSeek(int bufferSize) throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
-    final ConfigurationServiceImpl configurationservice = (ConfigurationServiceImpl) AbfsServiceProviderImpl.instance().get(ConfigurationService.class);
+    final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(this.getConfiguration());
 
     fs.create(TEST_PATH);
-
-    configurationservice.setWriteBufferSize(bufferSize);
-    configurationservice.setReadBufferSize(bufferSize);
+    abfsConfiguration.setWriteBufferSize(bufferSize);
+    abfsConfiguration.setReadBufferSize(bufferSize);
 
     final FSDataOutputStream stream = fs.create(TEST_PATH);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
deleted file mode 100644
index e90eb22..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
+++ /dev/null
@@ -1,79 +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.fs.azurebfs.services;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.log4j.AppenderSkeleton;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.log4j.spi.LoggingEvent;
-import org.junit.Test;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
-import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
-import org.apache.htrace.core.MilliSpan;
-import org.apache.htrace.core.TraceScope;
-
-/**
- * Test tracing service.
- */
-public class ITestTracingServiceImpl extends DependencyInjectedTest {
-  private final List<String> messageStorage;
-
-  public ITestTracingServiceImpl() {
-    super();
-    this.messageStorage = new ArrayList<>();
-  }
-
-  @Test
-  public void traceSerializationTest() throws Exception {
-    Logger rootLogger = Logger.getRootLogger();
-    rootLogger.setLevel(Level.TRACE);
-    rootLogger.addAppender(new AppenderSkeleton() {
-      @Override
-      protected void append(LoggingEvent event) {
-        if (event.getLogger().getName().indexOf("AzureBlobFileSystem") != -1) {
-          messageStorage.add(event.getRenderedMessage());
-        }
-      }
-
-      @Override
-      public void close() {
-
-      }
-
-      @Override
-      public boolean requiresLayout() {
-        return false;
-      }
-    });
-
-    TracingService tracingService = new TracingServiceImpl(new Configuration());
-    TraceScope traceScope = tracingService.traceBegin("Test Scope");
-    traceScope.addTimelineAnnotation("Timeline Annotations");
-    traceScope.addKVAnnotation("key", "value");
-    traceScope.close();
-
-    // Should not throw exception.
-    MilliSpan.fromJson(messageStorage.get(0));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java
deleted file mode 100644
index 2f27b16..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java
+++ /dev/null
@@ -1,69 +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.fs.azurebfs.services;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
-import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
-import org.apache.http.client.utils.URIBuilder;
-
-/**
- * Mock AbfsHttpClientFactoryImpl.
- */
-@Singleton
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class MockAbfsHttpClientFactoryImpl extends AbfsHttpClientFactoryImpl {
-  private final ConfigurationService configurationService;
-
-  @Inject
-  MockAbfsHttpClientFactoryImpl(
-      final ConfigurationService configurationService) {
-    super(configurationService);
-
-    this.configurationService = configurationService;
-  }
-
-  @VisibleForTesting
-  URIBuilder getURIBuilder(final String hostName, final FileSystem fs) {
-    final URIBuilder uriBuilder = new URIBuilder();
-
-    final String testHost = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME);
-    final Integer testHostPort = this.configurationService.getConfiguration().getInt(TestConfigurationKeys.FS_AZURE_TEST_HOST_PORT, 80);
-    final String testAccount = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
-
-    String scheme = FileSystemUriSchemes.HTTP_SCHEME;
-
-    uriBuilder.setScheme(scheme);
-    uriBuilder.setHost(testHost);
-    uriBuilder.setPort(testHostPort);
-
-    uriBuilder.setPath("/" + UriUtils.extractRawAccountFromAccountName(testAccount) + "/");
-
-    return uriBuilder;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
deleted file mode 100644
index ff2fb2a..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
+++ /dev/null
@@ -1,50 +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.fs.azurebfs.services;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Mock AbfsServiceInjectorImpl.
- */
-public class MockAbfsServiceInjectorImpl extends AbfsServiceInjectorImpl {
-  public MockAbfsServiceInjectorImpl(Configuration configuration) {
-    super(configuration);
-  }
-
-  public <T> void replaceInstance(Class<T> tInterface, Object object) {
-    this.removeInstance(tInterface);
-    this.removeProvider(tInterface);
-    this.getInstances().put(tInterface, object);
-  }
-
-  public <T> void removeInstance(Class<T> tInterface) {
-    this.getInstances().remove(tInterface);
-  }
-
-  public <T> void replaceProvider(Class<T> tInterface, Class<? extends T> tClazz) {
-    this.removeInstance(tInterface);
-    this.removeProvider(tInterface);
-    this.getProviders().put(tInterface, tClazz);
-  }
-
-  public <T> void removeProvider(Class<T> tInterface) {
-    this.getProviders().remove(tInterface);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java
deleted file mode 100644
index 5992f75..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.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.fs.azurebfs.services;
-
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-
-/**
- * Mock ABFS ServiceProviderImpl.
- */
-public final class MockServiceProviderImpl {
-  public static void create(MockAbfsServiceInjectorImpl abfsServiceInjector) {
-    Injector injector = Guice.createInjector(abfsServiceInjector);
-    AbfsServiceProviderImpl.create(injector);
-  }
-
-  private MockServiceProviderImpl() {
-    // no-op
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
new file mode 100644
index 0000000..441a35a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
@@ -0,0 +1,149 @@
+/**
+ * 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.azurebfs.services;
+
+import java.lang.reflect.Field;
+
+import org.apache.commons.codec.Charsets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT;
+
+import org.apache.commons.codec.binary.Base64;
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+/**
+ * Test ConfigurationServiceFieldsValidation.
+ */
+public class TestAbfsConfigurationFieldsValidation {
+  private AbfsConfiguration abfsConfiguration;
+
+  private static final String INT_KEY= "intKey";
+  private static final String LONG_KEY= "longKey";
+  private static final String STRING_KEY= "stringKey";
+  private static final String BASE64_KEY= "base64Key";
+  private static final String BOOLEAN_KEY= "booleanKey";
+  private static final int DEFAULT_INT = 4194304;
+  private static final int DEFAULT_LONG = 4194304;
+
+  private static final int TEST_INT = 1234565;
+  private static final int TEST_LONG = 4194304;
+
+  private final String encodedString;
+  private final String encodedAccountKey;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = INT_KEY,
+      MinValue = Integer.MIN_VALUE,
+      MaxValue = Integer.MAX_VALUE,
+      DefaultValue = DEFAULT_INT)
+  private int intField;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey = LONG_KEY,
+      MinValue = Long.MIN_VALUE,
+      MaxValue = Long.MAX_VALUE,
+      DefaultValue = DEFAULT_LONG)
+  private int longField;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = STRING_KEY,
+  DefaultValue = "default")
+  private String stringField;
+
+  @Base64StringConfigurationValidatorAnnotation(ConfigurationKey = BASE64_KEY,
+  DefaultValue = "base64")
+  private String base64Field;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = BOOLEAN_KEY,
+  DefaultValue = false)
+  private boolean boolField;
+
+  public TestAbfsConfigurationFieldsValidation() throws Exception {
+    super();
+    Base64 base64 = new Base64();
+    this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
+    this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
+    Configuration configuration = new Configuration();
+    configuration.addResource("azure-bfs-test.xml");
+    configuration.set(INT_KEY, "1234565");
+    configuration.set(LONG_KEY, "4194304");
+    configuration.set(STRING_KEY, "stringValue");
+    configuration.set(BASE64_KEY, encodedString);
+    configuration.set(BOOLEAN_KEY, "true");
+    configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "testaccount1.blob.core.windows.net", this.encodedAccountKey);
+    abfsConfiguration = new AbfsConfiguration(configuration);
+  }
+
+  @Test
+  public void testValidateFunctionsInConfigServiceImpl() throws Exception {
+    Field[] fields = this.getClass().getDeclaredFields();
+    for (Field field : fields) {
+      field.setAccessible(true);
+      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
+        assertEquals(TEST_INT, abfsConfiguration.validateInt(field));
+      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
+        assertEquals(DEFAULT_LONG, abfsConfiguration.validateLong(field));
+      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
+        assertEquals("stringValue", abfsConfiguration.validateString(field));
+      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
+        assertEquals(this.encodedString, abfsConfiguration.validateBase64String(field));
+      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
+        assertEquals(true, abfsConfiguration.validateBoolean(field));
+      }
+    }
+  }
+
+  @Test
+  public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception {
+    // test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor
+    assertEquals(DEFAULT_WRITE_BUFFER_SIZE, abfsConfiguration.getWriteBufferSize());
+    assertEquals(DEFAULT_READ_BUFFER_SIZE, abfsConfiguration.getReadBufferSize());
+    assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, abfsConfiguration.getMinBackoffIntervalMilliseconds());
+    assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, abfsConfiguration.getMaxBackoffIntervalMilliseconds());
+    assertEquals(DEFAULT_BACKOFF_INTERVAL, abfsConfiguration.getBackoffIntervalMilliseconds());
+    assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, abfsConfiguration.getMaxIoRetries());
+    assertEquals(MAX_AZURE_BLOCK_SIZE, abfsConfiguration.getAzureBlockSize());
+    assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, abfsConfiguration.getAzureBlockLocationHost());
+  }
+
+  @Test
+  public void testGetAccountKey() throws Exception {
+    String accountKey = abfsConfiguration.getStorageAccountKey("testaccount1.blob.core.windows.net");
+    assertEquals(this.encodedAccountKey, accountKey);
+  }
+
+  @Test (expected = ConfigurationPropertyNotFoundException.class)
+  public void testGetAccountKeyWithNonExistingAccountName() throws Exception {
+    abfsConfiguration.getStorageAccountKey("bogusAccountName");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java
deleted file mode 100644
index 688c522..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java
+++ /dev/null
@@ -1,149 +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.fs.azurebfs.services;
-
-import java.lang.reflect.Field;
-
-import org.apache.commons.codec.Charsets;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
-
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT;
-
-import org.apache.commons.codec.binary.Base64;
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-/**
- * Test ConfigurationServiceFieldsValidation.
- */
-public class TestConfigurationServiceFieldsValidation  {
-  private ConfigurationServiceImpl configService;
-
-  private static final String INT_KEY= "intKey";
-  private static final String LONG_KEY= "longKey";
-  private static final String STRING_KEY= "stringKey";
-  private static final String BASE64_KEY= "base64Key";
-  private static final String BOOLEAN_KEY= "booleanKey";
-  private static final int DEFAULT_INT = 4194304;
-  private static final int DEFAULT_LONG = 4194304;
-
-  private static final int TEST_INT = 1234565;
-  private static final int TEST_LONG = 4194304;
-
-  private final String encodedString;
-  private final String encodedAccountKey;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = INT_KEY,
-      MinValue = Integer.MIN_VALUE,
-      MaxValue = Integer.MAX_VALUE,
-      DefaultValue = DEFAULT_INT)
-  private int intField;
-
-  @LongConfigurationValidatorAnnotation(ConfigurationKey = LONG_KEY,
-      MinValue = Long.MIN_VALUE,
-      MaxValue = Long.MAX_VALUE,
-      DefaultValue = DEFAULT_LONG)
-  private int longField;
-
-  @StringConfigurationValidatorAnnotation(ConfigurationKey = STRING_KEY,
-  DefaultValue = "default")
-  private String stringField;
-
-  @Base64StringConfigurationValidatorAnnotation(ConfigurationKey = BASE64_KEY,
-  DefaultValue = "base64")
-  private String base64Field;
-
-  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = BOOLEAN_KEY,
-  DefaultValue = false)
-  private boolean boolField;
-
-  public TestConfigurationServiceFieldsValidation() throws Exception {
-    super();
-    Base64 base64 = new Base64();
-    this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
-    this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
-    Configuration configuration = new Configuration();
-    configuration.addResource("azure-bfs-test.xml");
-    configuration.set(INT_KEY, "1234565");
-    configuration.set(LONG_KEY, "4194304");
-    configuration.set(STRING_KEY, "stringValue");
-    configuration.set(BASE64_KEY, encodedString);
-    configuration.set(BOOLEAN_KEY, "true");
-    configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "testaccount1.blob.core.windows.net", this.encodedAccountKey);
-    configService = new ConfigurationServiceImpl(configuration);
-  }
-
-  @Test
-  public void testValidateFunctionsInConfigServiceImpl() throws Exception {
-    Field[] fields = this.getClass().getDeclaredFields();
-    for (Field field : fields) {
-      field.setAccessible(true);
-      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
-        assertEquals(TEST_INT, configService.validateInt(field));
-      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
-        assertEquals(DEFAULT_LONG, configService.validateLong(field));
-      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
-        assertEquals("stringValue", configService.validateString(field));
-      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
-        assertEquals(this.encodedString, configService.validateBase64String(field));
-      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
-        assertEquals(true, configService.validateBoolean(field));
-      }
-    }
-  }
-
-  @Test
-  public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception {
-    // test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor
-    assertEquals(DEFAULT_WRITE_BUFFER_SIZE, configService.getWriteBufferSize());
-    assertEquals(DEFAULT_READ_BUFFER_SIZE, configService.getReadBufferSize());
-    assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, configService.getMinBackoffIntervalMilliseconds());
-    assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, configService.getMaxBackoffIntervalMilliseconds());
-    assertEquals(DEFAULT_BACKOFF_INTERVAL, configService.getBackoffIntervalMilliseconds());
-    assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, configService.getMaxIoRetries());
-    assertEquals(MAX_AZURE_BLOCK_SIZE, configService.getAzureBlockSize());
-    assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, configService.getAzureBlockLocationHost());
-  }
-
-  @Test
-  public void testGetAccountKey() throws Exception {
-    String accountKey = configService.getStorageAccountKey("testaccount1.blob.core.windows.net");
-    assertEquals(this.encodedAccountKey, accountKey);
-  }
-
-  @Test (expected = ConfigurationPropertyNotFoundException.class)
-  public void testGetAccountKeyWithNonExistingAccountName() throws Exception {
-    configService.getStorageAccountKey("bogusAccountName");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
new file mode 100644
index 0000000..2716bff
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
@@ -0,0 +1,68 @@
+/*
+ * 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.azurebfs.utils;
+
+import com.microsoft.azure.storage.CloudStorageAccount;
+import com.microsoft.azure.storage.blob.CloudBlobClient;
+import com.microsoft.azure.storage.blob.CloudBlobContainer;
+
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.azurebfs.DependencyInjectedTest.TEST_CONTAINER_PREFIX;
+
+/**
+ * If unit tests were interrupted and crushed accidentally, the test containers won't be deleted.
+ * In that case, dev can use this tool to list and delete all test containers.
+ * By default, all test container used in E2E tests sharing same prefix: "abfs-testcontainer-"
+ */
+public class CleanUpAbfsTestContainer {
+  @Test
+  public void testEnumContainers() throws Throwable {
+    int count = 0;
+    CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
+    CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
+    Iterable<CloudBlobContainer> containers
+            = blobClient.listContainers(TEST_CONTAINER_PREFIX);
+    for (CloudBlobContainer container : containers) {
+      count++;
+      System.out.println(String.format("Container %s URI %s",
+              container.getName(),
+              container.getUri()));
+    }
+    System.out.println(String.format("Found %d test containers", count));
+  }
+
+  @Test
+  public void testDeleteContainers() throws Throwable {
+    int count = 0;
+    CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
+    CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
+    Iterable<CloudBlobContainer> containers
+            = blobClient.listContainers(TEST_CONTAINER_PREFIX);
+    for (CloudBlobContainer container : containers) {
+      System.out.println(String.format("Container %s URI %s",
+              container.getName(),
+              container.getUri()));
+      if (container.deleteIfExists()) {
+        count++;
+      }
+    }
+    System.out.println(String.format("Deleted %s test containers", count));
+  }
+}


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


[09/50] [abbrv] hadoop git commit: Make 3.1.1 awared by other branches - adding missing files

Posted by tm...@apache.org.
Make 3.1.1 awared by other branches - adding missing files


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

Branch: refs/heads/HADOOP-15407
Commit: 3214cd75acd0474373951870e1ba2ec11833a3da
Parents: 49c6876
Author: Wangda Tan <wa...@apache.org>
Authored: Wed Aug 8 13:05:24 2018 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Aug 8 13:05:24 2018 -0700

----------------------------------------------------------------------
 .../jdiff/Apache_Hadoop_YARN_Common_3.1.1.xml   | 3327 ++++++++++++++++++
 1 file changed, 3327 insertions(+)
----------------------------------------------------------------------



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


[07/50] [abbrv] hadoop git commit: Make 3.1.1 awared by other branches

Posted by tm...@apache.org.
Make 3.1.1 awared by other branches


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

Branch: refs/heads/HADOOP-15407
Commit: 49c687608b65b772faeed614700ece8e526432e8
Parents: 9499df7
Author: Wangda Tan <wa...@apache.org>
Authored: Wed Aug 8 13:01:58 2018 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Aug 8 13:02:12 2018 -0700

----------------------------------------------------------------------
 .../markdown/release/3.1.1/CHANGES.3.1.1.md     |  498 +++
 .../release/3.1.1/RELEASENOTES.3.1.1.md         |  498 +++
 .../jdiff/Apache_Hadoop_HDFS_3.1.1.xml          |  676 ++++
 hadoop-project-dist/pom.xml                     |    2 +-
 .../jdiff/Apache_Hadoop_YARN_Client_3.1.1.xml   | 2920 ++++++++++++++++++
 5 files changed, 4593 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/49c68760/hadoop-common-project/hadoop-common/src/site/markdown/release/3.1.1/CHANGES.3.1.1.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/3.1.1/CHANGES.3.1.1.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.1.1/CHANGES.3.1.1.md
new file mode 100644
index 0000000..8e2c804
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.1.1/CHANGES.3.1.1.md
@@ -0,0 +1,498 @@
+
+<!---
+# 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.
+-->
+# Apache Hadoop Changelog
+
+## Release 3.1.1 - 2018-08-02
+
+
+
+### IMPORTANT ISSUES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HADOOP-14667](https://issues.apache.org/jira/browse/HADOOP-14667) | Flexible Visual Studio support |  Major | build | Allen Wittenauer | Allen Wittenauer |
+
+
+### NEW FEATURES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HDFS-13056](https://issues.apache.org/jira/browse/HDFS-13056) | Expose file-level composite CRCs in HDFS which are comparable across different instances/layouts |  Major | datanode, distcp, erasure-coding, federation, hdfs | Dennis Huo | Dennis Huo |
+| [HDFS-13283](https://issues.apache.org/jira/browse/HDFS-13283) | Percentage based Reserved Space Calculation for DataNode |  Major | datanode, hdfs | Lukas Majercak | Lukas Majercak |
+
+
+### IMPROVEMENTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-8028](https://issues.apache.org/jira/browse/YARN-8028) | Support authorizeUserAccessToQueue in RMWebServices |  Major | . | Wangda Tan | Wangda Tan |
+| [HADOOP-15332](https://issues.apache.org/jira/browse/HADOOP-15332) | Fix typos in hadoop-aws markdown docs |  Minor | . | Gabor Bota | Gabor Bota |
+| [HADOOP-15330](https://issues.apache.org/jira/browse/HADOOP-15330) | Remove jdk1.7 profile from hadoop-annotations module |  Minor | . | Akira Ajisaka | fang zhenyi |
+| [HADOOP-15342](https://issues.apache.org/jira/browse/HADOOP-15342) | Update ADLS connector to use the current SDK version (2.2.7) |  Major | fs/adl | Atul Sikaria | Atul Sikaria |
+| [YARN-1151](https://issues.apache.org/jira/browse/YARN-1151) | Ability to configure auxiliary services from HDFS-based JAR files |  Major | nodemanager | john lilley | Xuan Gong |
+| [HDFS-13418](https://issues.apache.org/jira/browse/HDFS-13418) |  NetworkTopology should be configurable when enable DFSNetworkTopology |  Major | . | Tao Jie | Tao Jie |
+| [HDFS-13439](https://issues.apache.org/jira/browse/HDFS-13439) | Add test case for read block operation when it is moved |  Major | . | Ajay Kumar | Ajay Kumar |
+| [HDFS-13462](https://issues.apache.org/jira/browse/HDFS-13462) | Add BIND\_HOST configuration for JournalNode's HTTP and RPC Servers |  Major | hdfs, journal-node | Lukas Majercak | Lukas Majercak |
+| [YARN-8140](https://issues.apache.org/jira/browse/YARN-8140) | Improve log message when launch cmd is ran for stopped yarn service |  Major | yarn-native-services | Yesha Vora | Eric Yang |
+| [MAPREDUCE-7086](https://issues.apache.org/jira/browse/MAPREDUCE-7086) | Add config to allow FileInputFormat to ignore directories when recursive=false |  Major | . | Sergey Shelukhin | Sergey Shelukhin |
+| [HDFS-12981](https://issues.apache.org/jira/browse/HDFS-12981) | renameSnapshot a Non-Existent snapshot to itself should throw error |  Minor | hdfs | Sailesh Patel | Kitti Nanasi |
+| [YARN-8239](https://issues.apache.org/jira/browse/YARN-8239) | [UI2] Clicking on Node Manager UI under AM container info / App Attempt page goes to old RM UI |  Major | yarn-ui-v2 | Sumana Sathish | Sunil Govindan |
+| [YARN-8260](https://issues.apache.org/jira/browse/YARN-8260) | [UI2] Per-application tracking URL is no longer available in YARN UI2 |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8201](https://issues.apache.org/jira/browse/YARN-8201) | Skip stacktrace of few exception from ClientRMService |  Minor | . | Bibin A Chundatt | Bilwa S T |
+| [HADOOP-15441](https://issues.apache.org/jira/browse/HADOOP-15441) | Log kms url and token service at debug level. |  Minor | . | Wei-Chiu Chuang | Gabor Bota |
+| [HDFS-13544](https://issues.apache.org/jira/browse/HDFS-13544) | Improve logging for JournalNode in federated cluster |  Major | federation, hdfs | Hanisha Koneru | Hanisha Koneru |
+| [YARN-8249](https://issues.apache.org/jira/browse/YARN-8249) | Few REST api's in RMWebServices are missing static user check |  Critical | webapp, yarn | Sunil Govindan | Sunil Govindan |
+| [HDFS-13512](https://issues.apache.org/jira/browse/HDFS-13512) | WebHdfs getFileStatus doesn't return ecPolicy |  Major | . | Ajay Kumar | Ajay Kumar |
+| [HADOOP-15250](https://issues.apache.org/jira/browse/HADOOP-15250) | Split-DNS MultiHomed Server Network Cluster Network IPC Client Bind Addr Wrong |  Critical | ipc, net | Greg Senia | Ajay Kumar |
+| [HDFS-13589](https://issues.apache.org/jira/browse/HDFS-13589) | Add dfsAdmin command to query if "upgrade" is finalized |  Major | hdfs | Hanisha Koneru | Hanisha Koneru |
+| [HADOOP-15486](https://issues.apache.org/jira/browse/HADOOP-15486) | Make NetworkTopology#netLock fair |  Major | net | Nanda kumar | Nanda kumar |
+| [YARN-8213](https://issues.apache.org/jira/browse/YARN-8213) | Add Capacity Scheduler performance metrics |  Critical | capacityscheduler, metrics | Weiwei Yang | Weiwei Yang |
+| [HDFS-13628](https://issues.apache.org/jira/browse/HDFS-13628) | Update Archival Storage doc for Provided Storage |  Major | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [HADOOP-15449](https://issues.apache.org/jira/browse/HADOOP-15449) | Increase default timeout of ZK session to avoid frequent NameNode failover |  Critical | common | Karthik Palanisamy | Karthik Palanisamy |
+| [YARN-8333](https://issues.apache.org/jira/browse/YARN-8333) | Load balance YARN services using RegistryDNS multiple A records |  Major | yarn-native-services | Eric Yang | Eric Yang |
+| [HDFS-13602](https://issues.apache.org/jira/browse/HDFS-13602) | Add checkOperation(WRITE) checks in FSNamesystem |  Major | ha, namenode | Erik Krogen | Chao Sun |
+| [HDFS-13155](https://issues.apache.org/jira/browse/HDFS-13155) | BlockPlacementPolicyDefault.chooseTargetInOrder Not Checking Return Value for NULL |  Minor | namenode | BELUGA BEHR | Zsolt Venczel |
+| [YARN-8389](https://issues.apache.org/jira/browse/YARN-8389) | Improve the description of machine-list property in Federation docs |  Major | documentation, federation | Takanobu Asanuma | Takanobu Asanuma |
+| [HDFS-13511](https://issues.apache.org/jira/browse/HDFS-13511) | Provide specialized exception when block length cannot be obtained |  Major | . | Ted Yu | Gabor Bota |
+| [HDFS-13659](https://issues.apache.org/jira/browse/HDFS-13659) | Add more test coverage for contentSummary for snapshottable path |  Major | namenode, test | Wei-Chiu Chuang | Wei-Chiu Chuang |
+| [YARN-8400](https://issues.apache.org/jira/browse/YARN-8400) | Fix typos in YARN Federation documentation page |  Trivial | . | Bibin A Chundatt | Giovanni Matteo Fumarola |
+| [HADOOP-15499](https://issues.apache.org/jira/browse/HADOOP-15499) | Performance severe drop when running RawErasureCoderBenchmark with NativeRSRawErasureCoder |  Major | . | Sammi Chen | Sammi Chen |
+| [HDFS-13653](https://issues.apache.org/jira/browse/HDFS-13653) | Make dfs.client.failover.random.order a per nameservice configuration |  Major | federation | Ekanth Sethuramalingam | Ekanth Sethuramalingam |
+| [YARN-8394](https://issues.apache.org/jira/browse/YARN-8394) | Improve data locality documentation for Capacity Scheduler |  Major | . | Weiwei Yang | Weiwei Yang |
+| [HDFS-13641](https://issues.apache.org/jira/browse/HDFS-13641) | Add metrics for edit log tailing |  Major | metrics | Chao Sun | Chao Sun |
+| [HDFS-13686](https://issues.apache.org/jira/browse/HDFS-13686) | Add overall metrics for FSNamesystemLock |  Major | hdfs, namenode | Lukas Majercak | Lukas Majercak |
+| [HDFS-13692](https://issues.apache.org/jira/browse/HDFS-13692) | StorageInfoDefragmenter floods log when compacting StorageInfo TreeSet |  Minor | . | Yiqun Lin | Bharat Viswanadham |
+| [YARN-8214](https://issues.apache.org/jira/browse/YARN-8214) | Change default RegistryDNS port |  Major | . | Billie Rinaldi | Billie Rinaldi |
+| [HDFS-13703](https://issues.apache.org/jira/browse/HDFS-13703) | Avoid allocation of CorruptedBlocks hashmap when no corrupted blocks are hit |  Major | performance | Todd Lipcon | Todd Lipcon |
+| [HADOOP-15554](https://issues.apache.org/jira/browse/HADOOP-15554) | Improve JIT performance for Configuration parsing |  Minor | conf, performance | Todd Lipcon | Todd Lipcon |
+| [HDFS-13714](https://issues.apache.org/jira/browse/HDFS-13714) | Fix TestNameNodePrunesMissingStorages test failures on Windows |  Major | hdfs, namenode, test | Lukas Majercak | Lukas Majercak |
+| [HDFS-13712](https://issues.apache.org/jira/browse/HDFS-13712) | BlockReaderRemote.read() logging improvement |  Minor | hdfs-client | Gergo Repas | Gergo Repas |
+| [YARN-8302](https://issues.apache.org/jira/browse/YARN-8302) | ATS v2 should handle HBase connection issue properly |  Major | ATSv2 | Yesha Vora | Billie Rinaldi |
+| [HDFS-13674](https://issues.apache.org/jira/browse/HDFS-13674) | Improve documentation on Metrics |  Minor | documentation, metrics | Chao Sun | Chao Sun |
+| [HDFS-13719](https://issues.apache.org/jira/browse/HDFS-13719) | Docs around dfs.image.transfer.timeout are misleading |  Major | . | Kitti Nanasi | Kitti Nanasi |
+| [HADOOP-15598](https://issues.apache.org/jira/browse/HADOOP-15598) | DataChecksum calculate checksum is contented on hashtable synchronization |  Major | common | Prasanth Jayachandran | Prasanth Jayachandran |
+| [YARN-8501](https://issues.apache.org/jira/browse/YARN-8501) | Reduce complexity of RMWebServices' getApps method |  Major | restapi | Szilard Nemeth | Szilard Nemeth |
+| [HADOOP-15547](https://issues.apache.org/jira/browse/HADOOP-15547) | WASB: improve listStatus performance |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [YARN-8155](https://issues.apache.org/jira/browse/YARN-8155) | Improve ATSv2 client logging in RM and NM publisher |  Major | . | Rohith Sharma K S | Abhishek Modi |
+| [HADOOP-15476](https://issues.apache.org/jira/browse/HADOOP-15476) | fix logging for split-dns multihome |  Major | . | Ajay Kumar | Ajay Kumar |
+
+
+### BUG FIXES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-8040](https://issues.apache.org/jira/browse/YARN-8040) | [UI2] New YARN UI webapp does not respect current pathname for REST api |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [HADOOP-15062](https://issues.apache.org/jira/browse/HADOOP-15062) | TestCryptoStreamsWithOpensslAesCtrCryptoCodec fails on Debian 9 |  Major | . | Miklos Szegedi | Miklos Szegedi |
+| [HDFS-11043](https://issues.apache.org/jira/browse/HDFS-11043) | TestWebHdfsTimeouts fails |  Major | webhdfs | Andrew Wang | Chao Sun |
+| [HADOOP-15331](https://issues.apache.org/jira/browse/HADOOP-15331) | Fix a race condition causing parsing error of java.io.BufferedInputStream in class org.apache.hadoop.conf.Configuration |  Major | common | Miklos Szegedi | Miklos Szegedi |
+| [HDFS-11900](https://issues.apache.org/jira/browse/HDFS-11900) | Hedged reads thread pool creation not synchronized |  Major | hdfs-client | John Zhuge | John Zhuge |
+| [YARN-8032](https://issues.apache.org/jira/browse/YARN-8032) | Yarn service should expose failuresValidityInterval to users and use it for launching containers |  Major | . | Chandni Singh | Chandni Singh |
+| [YARN-8043](https://issues.apache.org/jira/browse/YARN-8043) | Add the exception message for failed launches running under LCE |  Major | . | Shane Kumpf | Shane Kumpf |
+| [YARN-7734](https://issues.apache.org/jira/browse/YARN-7734) | YARN-5418 breaks TestContainerLogsPage.testContainerLogPageAccess |  Major | . | Miklos Szegedi | Tao Yang |
+| [HDFS-13087](https://issues.apache.org/jira/browse/HDFS-13087) | Snapshotted encryption zone information should be immutable |  Major | encryption | LiXin Ge | LiXin Ge |
+| [HADOOP-12862](https://issues.apache.org/jira/browse/HADOOP-12862) | LDAP Group Mapping over SSL can not specify trust store |  Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang |
+| [HADOOP-15317](https://issues.apache.org/jira/browse/HADOOP-15317) | Improve NetworkTopology chooseRandom's loop |  Major | . | Xiao Chen | Xiao Chen |
+| [HADOOP-15355](https://issues.apache.org/jira/browse/HADOOP-15355) | TestCommonConfigurationFields is broken by HADOOP-15312 |  Major | test | Konstantin Shvachko | LiXin Ge |
+| [YARN-7764](https://issues.apache.org/jira/browse/YARN-7764) | Findbugs warning: Resource#getResources may expose internal representation |  Major | api | Weiwei Yang | Weiwei Yang |
+| [YARN-8106](https://issues.apache.org/jira/browse/YARN-8106) | Update LogAggregationIndexedFileController to use readFull instead read to avoid IOException while loading log meta |  Critical | log-aggregation | Prabhu Joseph | Prabhu Joseph |
+| [YARN-8115](https://issues.apache.org/jira/browse/YARN-8115) | [UI2] URL data like nodeHTTPAddress must be encoded in UI before using to access NM |  Major | yarn-ui-v2 | Sunil Govindan | Sreenath Somarajapuram |
+| [HDFS-13350](https://issues.apache.org/jira/browse/HDFS-13350) | Negative legacy block ID will confuse Erasure Coding to be considered as striped block |  Major | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [YARN-8119](https://issues.apache.org/jira/browse/YARN-8119) | [UI2] Timeline Server address' url scheme should be removed while accessing via KNOX |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8083](https://issues.apache.org/jira/browse/YARN-8083) | [UI2] All YARN related configurations are paged together in conf page |  Major | yarn-ui-v2 | Zoltan Haindrich | Gergely Novák |
+| [HADOOP-15366](https://issues.apache.org/jira/browse/HADOOP-15366) | Add a helper shutdown routine in HadoopExecutor to ensure clean shutdown |  Minor | . | Shashikant Banerjee | Shashikant Banerjee |
+| [YARN-7905](https://issues.apache.org/jira/browse/YARN-7905) | Parent directory permission incorrect during public localization |  Critical | . | Bibin A Chundatt | Bilwa S T |
+| [HADOOP-15374](https://issues.apache.org/jira/browse/HADOOP-15374) | Add links of the new features of 3.1.0 to the top page |  Major | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-7804](https://issues.apache.org/jira/browse/YARN-7804) | Refresh action on Grid view page should not be redirected to graph view |  Major | yarn-ui-v2 | Yesha Vora | Gergely Novák |
+| [HDFS-13420](https://issues.apache.org/jira/browse/HDFS-13420) | License header is displayed in ArchivalStorage/MemoryStorage html pages |  Minor | documentation | Akira Ajisaka | Akira Ajisaka |
+| [HDFS-13328](https://issues.apache.org/jira/browse/HDFS-13328) | Abstract ReencryptionHandler recursive logic in separate class. |  Major | namenode | Surendra Singh Lilhore | Surendra Singh Lilhore |
+| [HADOOP-15357](https://issues.apache.org/jira/browse/HADOOP-15357) | Configuration.getPropsWithPrefix no longer does variable substitution |  Major | . | Jim Brennan | Jim Brennan |
+| [YARN-7984](https://issues.apache.org/jira/browse/YARN-7984) | Delete registry entries from ZK on ServiceClient stop and clean up stop/destroy behavior |  Critical | yarn-native-services | Billie Rinaldi | Billie Rinaldi |
+| [YARN-8133](https://issues.apache.org/jira/browse/YARN-8133) | Doc link broken for yarn-service from overview page. |  Blocker | yarn-native-services | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8116](https://issues.apache.org/jira/browse/YARN-8116) | Nodemanager fails with NumberFormatException: For input string: "" |  Critical | . | Yesha Vora | Chandni Singh |
+| [MAPREDUCE-7062](https://issues.apache.org/jira/browse/MAPREDUCE-7062) | Update mapreduce.job.tags description for making use for ATSv2 purpose. |  Major | . | Charan Hebri | Charan Hebri |
+| [YARN-8073](https://issues.apache.org/jira/browse/YARN-8073) | TimelineClientImpl doesn't honor yarn.timeline-service.versions configuration |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8127](https://issues.apache.org/jira/browse/YARN-8127) | Resource leak when async scheduling is enabled |  Critical | . | Weiwei Yang | Tao Yang |
+| [HDFS-13427](https://issues.apache.org/jira/browse/HDFS-13427) | Fix the section titles of transparent encryption document |  Minor | documentation | Akira Ajisaka | Akira Ajisaka |
+| [HDFS-7101](https://issues.apache.org/jira/browse/HDFS-7101) | Potential null dereference in DFSck#doWork() |  Minor | . | Ted Yu | skrho |
+| [HDFS-13426](https://issues.apache.org/jira/browse/HDFS-13426) | Fix javadoc in FsDatasetAsyncDiskService#removeVolume |  Minor | hdfs | Shashikant Banerjee | Shashikant Banerjee |
+| [YARN-8120](https://issues.apache.org/jira/browse/YARN-8120) | JVM can crash with SIGSEGV when exiting due to custom leveldb logger |  Major | nodemanager, resourcemanager | Jason Lowe | Jason Lowe |
+| [YARN-8147](https://issues.apache.org/jira/browse/YARN-8147) | TestClientRMService#testGetApplications sporadically fails |  Major | test | Jason Lowe | Jason Lowe |
+| [HDFS-13436](https://issues.apache.org/jira/browse/HDFS-13436) | Fix javadoc of package-info.java |  Major | documentation | Akira Ajisaka | Akira Ajisaka |
+| [HADOOP-15379](https://issues.apache.org/jira/browse/HADOOP-15379) | Make IrqHandler.bind() public |  Minor | util | Steve Loughran | Ajay Kumar |
+| [YARN-8154](https://issues.apache.org/jira/browse/YARN-8154) | Fix missing titles in PlacementConstraints document |  Minor | documentation | Akira Ajisaka | Weiwei Yang |
+| [YARN-8153](https://issues.apache.org/jira/browse/YARN-8153) | Guaranteed containers always stay in SCHEDULED on NM after restart |  Major | . | Yang Wang | Yang Wang |
+| [HADOOP-14970](https://issues.apache.org/jira/browse/HADOOP-14970) | MiniHadoopClusterManager doesn't respect lack of format option |  Minor | . | Erik Krogen | Erik Krogen |
+| [HDFS-13438](https://issues.apache.org/jira/browse/HDFS-13438) | Fix javadoc in FsVolumeList#removeVolume |  Minor | . | Shashikant Banerjee | Shashikant Banerjee |
+| [YARN-8142](https://issues.apache.org/jira/browse/YARN-8142) | yarn service application stops when AM is killed with SIGTERM |  Major | yarn-native-services | Yesha Vora | Billie Rinaldi |
+| [MAPREDUCE-7077](https://issues.apache.org/jira/browse/MAPREDUCE-7077) | Pipe mapreduce job fails with Permission denied for jobTokenPassword |  Critical | . | Yesha Vora | Akira Ajisaka |
+| [HDFS-13330](https://issues.apache.org/jira/browse/HDFS-13330) | ShortCircuitCache#fetchOrCreate never retries |  Major | . | Wei-Chiu Chuang | Gabor Bota |
+| [YARN-8156](https://issues.apache.org/jira/browse/YARN-8156) | Increase the default value of yarn.timeline-service.app-collector.linger-period.ms |  Major | . | Rohith Sharma K S | Charan Hebri |
+| [YARN-8165](https://issues.apache.org/jira/browse/YARN-8165) | Incorrect queue name logging in AbstractContainerAllocator |  Trivial | capacityscheduler | Weiwei Yang | Weiwei Yang |
+| [HDFS-12828](https://issues.apache.org/jira/browse/HDFS-12828) | OIV ReverseXML Processor fails with escaped characters |  Critical | hdfs | Erik Krogen | Erik Krogen |
+| [HADOOP-15391](https://issues.apache.org/jira/browse/HADOOP-15391) | Add missing css file in hadoop-aws, hadoop-aliyun, hadoop-azure and hadoop-azure-datalake modules |  Major | documentation | Yiqun Lin | Yiqun Lin |
+| [YARN-8171](https://issues.apache.org/jira/browse/YARN-8171) | [UI2] AM Node link from attempt page should not redirect to new tab |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8145](https://issues.apache.org/jira/browse/YARN-8145) | yarn rmadmin -getGroups doesn't return updated groups for user |  Major | . | Sumana Sathish | Sunil Govindan |
+| [HDFS-13463](https://issues.apache.org/jira/browse/HDFS-13463) | Fix javadoc in FsDatasetImpl#checkAndUpdate |  Minor | datanode | Shashikant Banerjee | Shashikant Banerjee |
+| [HDFS-13464](https://issues.apache.org/jira/browse/HDFS-13464) | Fix javadoc in FsVolumeList#handleVolumeFailures |  Minor | documentation | Shashikant Banerjee | Shashikant Banerjee |
+| [HADOOP-15396](https://issues.apache.org/jira/browse/HADOOP-15396) | Some java source files are executable |  Minor | . | Akira Ajisaka | Shashikant Banerjee |
+| [YARN-6827](https://issues.apache.org/jira/browse/YARN-6827) | [ATS1/1.5] NPE exception while publishing recovering applications into ATS during RM restart. |  Major | resourcemanager | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8182](https://issues.apache.org/jira/browse/YARN-8182) | [UI2] Proxy- Clicking on nodes under Nodes HeatMap gives 401 error |  Critical | . | Sumana Sathish | Sunil Govindan |
+| [YARN-8189](https://issues.apache.org/jira/browse/YARN-8189) | [UI2] Nodes page column headers are half truncated |  Major | . | Sunil Govindan | Sunil Govindan |
+| [YARN-7830](https://issues.apache.org/jira/browse/YARN-7830) | [UI2] Post selecting grid view in Attempt page, attempt info page should also be opened with grid view |  Major | yarn-ui-v2 | Yesha Vora | Gergely Novák |
+| [YARN-7786](https://issues.apache.org/jira/browse/YARN-7786) | NullPointerException while launching ApplicationMaster |  Major | . | lujie | lujie |
+| [HDFS-10183](https://issues.apache.org/jira/browse/HDFS-10183) | Prevent race condition during class initialization |  Minor | fs | Pavel Avgustinov | Pavel Avgustinov |
+| [HDFS-13388](https://issues.apache.org/jira/browse/HDFS-13388) | RequestHedgingProxyProvider calls multiple configured NNs all the time |  Major | hdfs-client | Jinglun | Jinglun |
+| [YARN-7956](https://issues.apache.org/jira/browse/YARN-7956) | [UI2] Avoid duplicating Components link under Services/\<ServiceName\>/Components |  Major | yarn-ui-v2 | Yesha Vora | Yesha Vora |
+| [HDFS-13433](https://issues.apache.org/jira/browse/HDFS-13433) | webhdfs requests can be routed incorrectly in federated cluster |  Critical | webhdfs | Arpit Agarwal | Arpit Agarwal |
+| [HDFS-13408](https://issues.apache.org/jira/browse/HDFS-13408) | MiniDFSCluster to support being built on randomized base directory |  Major | test | Xiao Liang | Xiao Liang |
+| [HDFS-13356](https://issues.apache.org/jira/browse/HDFS-13356) | Balancer:Set default value of minBlockSize to 10mb |  Major | balancer & mover | Bharat Viswanadham | Bharat Viswanadham |
+| [HADOOP-15390](https://issues.apache.org/jira/browse/HADOOP-15390) | Yarn RM logs flooded by DelegationTokenRenewer trying to renew KMS tokens |  Critical | . | Xiao Chen | Xiao Chen |
+| [HDFS-13336](https://issues.apache.org/jira/browse/HDFS-13336) | Test cases of TestWriteToReplica failed in windows |  Major | . | Xiao Liang | Xiao Liang |
+| [YARN-8196](https://issues.apache.org/jira/browse/YARN-8196) | yarn.webapp.api-service.enable should be highlighted in the quickstart |  Trivial | documentation | Davide  Vergari | Billie Rinaldi |
+| [YARN-8183](https://issues.apache.org/jira/browse/YARN-8183) | Fix ConcurrentModificationException inside RMAppAttemptMetrics#convertAtomicLongMaptoLongMap |  Critical | yarn | Sumana Sathish | Suma Shivaprasad |
+| [YARN-8188](https://issues.apache.org/jira/browse/YARN-8188) | RM Nodes UI data table index for sorting column need to be corrected post Application tags display |  Major | resourcemanager, webapp | Weiwei Yang | Weiwei Yang |
+| [HADOOP-15411](https://issues.apache.org/jira/browse/HADOOP-15411) | AuthenticationFilter should use Configuration.getPropsWithPrefix instead of iterator |  Critical | . | Suma Shivaprasad | Suma Shivaprasad |
+| [MAPREDUCE-7042](https://issues.apache.org/jira/browse/MAPREDUCE-7042) | Killed MR job data does not move to mapreduce.jobhistory.done-dir when ATS v2 is enabled |  Major | . | Yesha Vora | Xuan Gong |
+| [YARN-8205](https://issues.apache.org/jira/browse/YARN-8205) | Application State is not updated to ATS if AM launching is delayed. |  Critical | . | Sumana Sathish | Rohith Sharma K S |
+| [YARN-8004](https://issues.apache.org/jira/browse/YARN-8004) | Add unit tests for inter queue preemption for dominant resource calculator |  Critical | yarn | Sumana Sathish | Zian Chen |
+| [YARN-8208](https://issues.apache.org/jira/browse/YARN-8208) | Add log statement for Docker client configuration file at INFO level |  Minor | yarn-native-services | Yesha Vora | Yesha Vora |
+| [YARN-8211](https://issues.apache.org/jira/browse/YARN-8211) | Yarn registry dns log finds BufferUnderflowException on port ping |  Major | yarn-native-services | Yesha Vora | Eric Yang |
+| [YARN-8221](https://issues.apache.org/jira/browse/YARN-8221) | RMWebServices also need to honor yarn.resourcemanager.display.per-user-apps |  Major | webapp | Sunil Govindan | Sunil Govindan |
+| [YARN-8210](https://issues.apache.org/jira/browse/YARN-8210) | AMRMClient logging on every heartbeat to track updation of AM RM token causes too many log lines to be generated in AM logs |  Major | yarn | Suma Shivaprasad | Suma Shivaprasad |
+| [YARN-8005](https://issues.apache.org/jira/browse/YARN-8005) | Add unit tests for queue priority with dominant resource calculator |  Critical | . | Sumana Sathish | Zian Chen |
+| [YARN-8225](https://issues.apache.org/jira/browse/YARN-8225) | YARN precommit build failing in TestPlacementConstraintTransformations |  Critical | . | Billie Rinaldi | Shane Kumpf |
+| [HDFS-13509](https://issues.apache.org/jira/browse/HDFS-13509) | Bug fix for breakHardlinks() of ReplicaInfo/LocalReplica, and fix TestFileAppend failures on Windows |  Major | . | Xiao Liang | Xiao Liang |
+| [YARN-8187](https://issues.apache.org/jira/browse/YARN-8187) | [UI2] Individual Node page does not contain breadcrumb trail |  Critical | yarn-ui-v2 | Sumana Sathish | Zian Chen |
+| [YARN-7799](https://issues.apache.org/jira/browse/YARN-7799) | YARN Service dependency follow up work |  Critical | client, resourcemanager | Gour Saha | Billie Rinaldi |
+| [MAPREDUCE-7073](https://issues.apache.org/jira/browse/MAPREDUCE-7073) | Optimize TokenCache#obtainTokensForNamenodesInternal |  Major | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HADOOP-15406](https://issues.apache.org/jira/browse/HADOOP-15406) | hadoop-nfs dependencies for mockito and junit are not test scope |  Major | nfs | Jason Lowe | Jason Lowe |
+| [YARN-6385](https://issues.apache.org/jira/browse/YARN-6385) | Fix checkstyle warnings in TestFileSystemApplicationHistoryStore |  Minor | . | Yiqun Lin | Yiqun Lin |
+| [YARN-8222](https://issues.apache.org/jira/browse/YARN-8222) | Fix potential NPE when gets RMApp from RM context |  Critical | . | Tao Yang | Tao Yang |
+| [YARN-8209](https://issues.apache.org/jira/browse/YARN-8209) | NPE in DeletionService |  Critical | . | Chandni Singh | Eric Badger |
+| [HDFS-13481](https://issues.apache.org/jira/browse/HDFS-13481) | TestRollingFileSystemSinkWithHdfs#testFlushThread: test failed intermittently |  Major | hdfs | Gabor Bota | Gabor Bota |
+| [YARN-8217](https://issues.apache.org/jira/browse/YARN-8217) | RmAuthenticationFilterInitializer /TimelineAuthenticationFilterInitializer should use Configuration.getPropsWithPrefix instead of iterator |  Major | . | Suma Shivaprasad | Suma Shivaprasad |
+| [YARN-7818](https://issues.apache.org/jira/browse/YARN-7818) | Remove privileged operation warnings during container launch for the ContainerRuntimes |  Major | . | Yesha Vora | Shane Kumpf |
+| [YARN-8223](https://issues.apache.org/jira/browse/YARN-8223) | ClassNotFoundException when auxiliary service is loaded from HDFS |  Blocker | . | Charan Hebri | Zian Chen |
+| [YARN-8079](https://issues.apache.org/jira/browse/YARN-8079) | Support static and archive unmodified local resources in service AM |  Critical | . | Wangda Tan | Suma Shivaprasad |
+| [YARN-8025](https://issues.apache.org/jira/browse/YARN-8025) | UsersManangers#getComputedResourceLimitForActiveUsers throws NPE due to preComputedActiveUserLimit is empty |  Major | yarn | Jiandan Yang | Tao Yang |
+| [YARN-8251](https://issues.apache.org/jira/browse/YARN-8251) | [UI2] Clicking on Application link at the header goes to Diagnostics Tab instead of AppAttempt Tab |  Major | yarn-ui-v2 | Sumana Sathish | Yesha Vora |
+| [YARN-8232](https://issues.apache.org/jira/browse/YARN-8232) | RMContainer lost queue name when RM HA happens |  Major | resourcemanager | Hu Ziqian | Hu Ziqian |
+| [YARN-7894](https://issues.apache.org/jira/browse/YARN-7894) | Improve ATS response for DS\_CONTAINER when container launch fails |  Major | timelineserver | Charan Hebri | Chandni Singh |
+| [YARN-8264](https://issues.apache.org/jira/browse/YARN-8264) | [UI2 GPU] GPU Info tab disappears if we click any sub link under List of Applications or List of Containers |  Major | . | Sumana Sathish | Sunil Govindan |
+| [HDFS-13537](https://issues.apache.org/jira/browse/HDFS-13537) | TestHdfsHelper does not generate jceks path properly for relative path in Windows |  Major | . | Xiao Liang | Xiao Liang |
+| [YARN-8202](https://issues.apache.org/jira/browse/YARN-8202) | DefaultAMSProcessor should properly check units of requested custom resource types against minimum/maximum allocation |  Blocker | . | Szilard Nemeth | Szilard Nemeth |
+| [HADOOP-15446](https://issues.apache.org/jira/browse/HADOOP-15446) | WASB: PageBlobInputStream.skip breaks HBASE replication |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [YARN-7003](https://issues.apache.org/jira/browse/YARN-7003) | DRAINING state of queues is not recovered after RM restart |  Major | capacityscheduler | Tao Yang | Tao Yang |
+| [YARN-8244](https://issues.apache.org/jira/browse/YARN-8244) |  TestContainerSchedulerQueuing.testStartMultipleContainers failed |  Major | . | Miklos Szegedi | Jim Brennan |
+| [YARN-8265](https://issues.apache.org/jira/browse/YARN-8265) | Service AM should retrieve new IP for docker container relaunched by NM |  Critical | yarn-native-services | Eric Yang | Billie Rinaldi |
+| [YARN-8271](https://issues.apache.org/jira/browse/YARN-8271) | [UI2] Improve labeling of certain tables |  Major | yarn-ui-v2 | Yesha Vora | Yesha Vora |
+| [YARN-8288](https://issues.apache.org/jira/browse/YARN-8288) | Fix wrong number of table columns in Resource Model doc |  Major | . | Weiwei Yang | Weiwei Yang |
+| [HDFS-13539](https://issues.apache.org/jira/browse/HDFS-13539) | DFSStripedInputStream NPE when reportCheckSumFailure |  Major | . | Xiao Chen | Xiao Chen |
+| [YARN-8266](https://issues.apache.org/jira/browse/YARN-8266) | [UI2] Clicking on application from cluster view should redirect to application attempt page |  Major | yarn-ui-v2 | Yesha Vora | Yesha Vora |
+| [YARN-8166](https://issues.apache.org/jira/browse/YARN-8166) | [UI2] Service page header links are broken |  Major | yarn-ui-v2 | Yesha Vora | Yesha Vora |
+| [YARN-8236](https://issues.apache.org/jira/browse/YARN-8236) | Invalid kerberos principal file name cause NPE in native service |  Critical | yarn-native-services | Sunil Govindan | Gour Saha |
+| [YARN-8278](https://issues.apache.org/jira/browse/YARN-8278) | DistributedScheduling is not working in HA |  Blocker | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HADOOP-15442](https://issues.apache.org/jira/browse/HADOOP-15442) | ITestS3AMetrics.testMetricsRegister can't know metrics source's name |  Major | fs/s3, metrics | Sean Mackrory | Sean Mackrory |
+| [YARN-8300](https://issues.apache.org/jira/browse/YARN-8300) | Fix NPE in DefaultUpgradeComponentsFinder |  Major | yarn | Suma Shivaprasad | Suma Shivaprasad |
+| [HDFS-13581](https://issues.apache.org/jira/browse/HDFS-13581) | DN UI logs link is broken when https is enabled |  Minor | datanode | Namit Maheshwari | Shashikant Banerjee |
+| [YARN-8128](https://issues.apache.org/jira/browse/YARN-8128) | Document better the per-node per-app file limit in YARN log aggregation |  Major | . | Xuan Gong | Xuan Gong |
+| [YARN-8293](https://issues.apache.org/jira/browse/YARN-8293) | In YARN Services UI, "User Name for service" should be completely removed in secure clusters |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8141](https://issues.apache.org/jira/browse/YARN-8141) | YARN Native Service: Respect YARN\_CONTAINER\_RUNTIME\_DOCKER\_LOCAL\_RESOURCE\_MOUNTS specified in service spec |  Critical | yarn-native-services | Wangda Tan | Chandni Singh |
+| [YARN-8296](https://issues.apache.org/jira/browse/YARN-8296) | Update YarnServiceApi documentation and yarn service UI code to remove references to unique\_component\_support |  Major | yarn-native-services, yarn-ui-v2 | Suma Shivaprasad | Suma Shivaprasad |
+| [HDFS-13586](https://issues.apache.org/jira/browse/HDFS-13586) | Fsync fails on directories on Windows |  Critical | datanode, hdfs | Lukas Majercak | Lukas Majercak |
+| [HADOOP-15478](https://issues.apache.org/jira/browse/HADOOP-15478) | WASB: hflush() and hsync() regression |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [YARN-8179](https://issues.apache.org/jira/browse/YARN-8179) | Preemption does not happen due to natural\_termination\_factor when DRF is used |  Major | . | kyungwan nam | kyungwan nam |
+| [HADOOP-15450](https://issues.apache.org/jira/browse/HADOOP-15450) | Avoid fsync storm triggered by DiskChecker and handle disk full situation |  Blocker | . | Kihwal Lee | Arpit Agarwal |
+| [YARN-8290](https://issues.apache.org/jira/browse/YARN-8290) | SystemMetricsPublisher.appACLsUpdated should be invoked after application information is published to ATS to avoid "User is not set in the application report" Exception |  Critical | . | Yesha Vora | Eric Yang |
+| [YARN-8332](https://issues.apache.org/jira/browse/YARN-8332) | Incorrect min/max allocation property name in resource types doc |  Critical | documentation | Weiwei Yang | Weiwei Yang |
+| [HDFS-13601](https://issues.apache.org/jira/browse/HDFS-13601) | Optimize ByteString conversions in PBHelper |  Major | . | Andrew Wang | Andrew Wang |
+| [HDFS-13540](https://issues.apache.org/jira/browse/HDFS-13540) | DFSStripedInputStream should only allocate new buffers when reading |  Major | . | Xiao Chen | Xiao Chen |
+| [YARN-8297](https://issues.apache.org/jira/browse/YARN-8297) | Incorrect ATS Url used for Wire encrypted cluster |  Blocker | yarn-ui-v2 | Yesha Vora | Sunil Govindan |
+| [HDFS-13588](https://issues.apache.org/jira/browse/HDFS-13588) | Fix TestFsDatasetImpl test failures on Windows |  Major | . | Xiao Liang | Xiao Liang |
+| [YARN-8310](https://issues.apache.org/jira/browse/YARN-8310) | Handle old NMTokenIdentifier, AMRMTokenIdentifier, and ContainerTokenIdentifier formats |  Major | . | Robert Kanter | Robert Kanter |
+| [YARN-8344](https://issues.apache.org/jira/browse/YARN-8344) | Missing nm.stop() in TestNodeManagerResync to fix testKillContainersOnResync |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-8327](https://issues.apache.org/jira/browse/YARN-8327) | Fix TestAggregatedLogFormat#testReadAcontainerLogs1 on Windows |  Major | log-aggregation | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HDFS-13611](https://issues.apache.org/jira/browse/HDFS-13611) | Unsafe use of Text as a ConcurrentHashMap key in PBHelperClient |  Major | . | Andrew Wang | Andrew Wang |
+| [YARN-8316](https://issues.apache.org/jira/browse/YARN-8316) | Diagnostic message should improve when yarn service fails to launch due to ATS unavailability |  Major | yarn-native-services | Yesha Vora | Billie Rinaldi |
+| [YARN-8357](https://issues.apache.org/jira/browse/YARN-8357) | Yarn Service: NPE when service is saved first and then started. |  Critical | . | Chandni Singh | Chandni Singh |
+| [HDFS-13618](https://issues.apache.org/jira/browse/HDFS-13618) | Fix TestDataNodeFaultInjector test failures on Windows |  Major | test | Xiao Liang | Xiao Liang |
+| [HADOOP-15473](https://issues.apache.org/jira/browse/HADOOP-15473) | Configure serialFilter in KeyProvider to avoid UnrecoverableKeyException caused by JDK-8189997 |  Critical | kms | Gabor Bota | Gabor Bota |
+| [YARN-8292](https://issues.apache.org/jira/browse/YARN-8292) | Fix the dominant resource preemption cannot happen when some of the resource vector becomes negative |  Critical | yarn | Sumana Sathish | Wangda Tan |
+| [YARN-8338](https://issues.apache.org/jira/browse/YARN-8338) | TimelineService V1.5 doesn't come up after HADOOP-15406 |  Critical | . | Vinod Kumar Vavilapalli | Vinod Kumar Vavilapalli |
+| [YARN-8339](https://issues.apache.org/jira/browse/YARN-8339) | Service AM should localize static/archive resource types to container working directory instead of 'resources' |  Critical | yarn-native-services | Suma Shivaprasad | Suma Shivaprasad |
+| [YARN-8369](https://issues.apache.org/jira/browse/YARN-8369) | Javadoc build failed due to "bad use of '\>'" |  Critical | build, docs | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-8362](https://issues.apache.org/jira/browse/YARN-8362) | Number of remaining retries are updated twice after a container failure in NM |  Critical | . | Chandni Singh | Chandni Singh |
+| [YARN-8377](https://issues.apache.org/jira/browse/YARN-8377) | Javadoc build failed in hadoop-yarn-server-nodemanager |  Critical | build, docs | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-8368](https://issues.apache.org/jira/browse/YARN-8368) | yarn app start cli should print applicationId |  Critical | . | Yesha Vora | Rohith Sharma K S |
+| [YARN-8350](https://issues.apache.org/jira/browse/YARN-8350) | NPE in service AM related to placement policy |  Critical | yarn-native-services | Billie Rinaldi | Gour Saha |
+| [YARN-8367](https://issues.apache.org/jira/browse/YARN-8367) | Fix NPE in SingleConstraintAppPlacementAllocator when placement constraint in SchedulingRequest is null |  Major | scheduler | Gour Saha | Weiwei Yang |
+| [YARN-8197](https://issues.apache.org/jira/browse/YARN-8197) | Tracking URL in the app state does not get redirected to MR ApplicationMaster for Running applications |  Critical | yarn | Sumana Sathish | Sunil Govindan |
+| [YARN-8308](https://issues.apache.org/jira/browse/YARN-8308) | Yarn service app fails due to issues with Renew Token |  Major | yarn-native-services | Yesha Vora | Gour Saha |
+| [HDFS-13636](https://issues.apache.org/jira/browse/HDFS-13636) | Cross-Site Scripting vulnerability in HttpServer2 |  Major | . | Haibo Yan | Haibo Yan |
+| [YARN-7962](https://issues.apache.org/jira/browse/YARN-7962) | Race Condition When Stopping DelegationTokenRenewer causes RM crash during failover |  Critical | resourcemanager | BELUGA BEHR | BELUGA BEHR |
+| [YARN-8372](https://issues.apache.org/jira/browse/YARN-8372) | Distributed shell app master should not release containers when shutdown if keep-container is true |  Critical | distributed-shell | Charan Hebri | Suma Shivaprasad |
+| [YARN-8319](https://issues.apache.org/jira/browse/YARN-8319) | More YARN pages need to honor yarn.resourcemanager.display.per-user-apps |  Major | webapp | Vinod Kumar Vavilapalli | Sunil Govindan |
+| [MAPREDUCE-7097](https://issues.apache.org/jira/browse/MAPREDUCE-7097) | MapReduce JHS should honor yarn.webapp.filter-entity-list-by-user |  Major | . | Vinod Kumar Vavilapalli | Sunil Govindan |
+| [YARN-8276](https://issues.apache.org/jira/browse/YARN-8276) | [UI2] After version field became mandatory, form-based submission of new YARN service doesn't work |  Critical | yarn-ui-v2 | Gergely Novák | Gergely Novák |
+| [HDFS-13339](https://issues.apache.org/jira/browse/HDFS-13339) | Volume reference can't be released and may lead to deadlock when DataXceiver does a check volume |  Critical | datanode | liaoyuxiangqin | Zsolt Venczel |
+| [YARN-8382](https://issues.apache.org/jira/browse/YARN-8382) | cgroup file leak in NM |  Major | nodemanager | Hu Ziqian | Hu Ziqian |
+| [YARN-8365](https://issues.apache.org/jira/browse/YARN-8365) | Revisit the record type used by Registry DNS for upstream resolution |  Major | yarn-native-services | Shane Kumpf | Shane Kumpf |
+| [HDFS-13545](https://issues.apache.org/jira/browse/HDFS-13545) |  "guarded" is misspelled as "gaurded" in FSPermissionChecker.java |  Trivial | documentation | Jianchao Jia | Jianchao Jia |
+| [YARN-8396](https://issues.apache.org/jira/browse/YARN-8396) | Click on an individual container continuously spins and doesn't load the page |  Blocker | . | Charan Hebri | Sunil Govindan |
+| [MAPREDUCE-7103](https://issues.apache.org/jira/browse/MAPREDUCE-7103) | Fix TestHistoryViewerPrinter on windows due to a mismatch line separator |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HADOOP-15217](https://issues.apache.org/jira/browse/HADOOP-15217) | FsUrlConnection does not handle paths with spaces |  Major | fs | Joseph Fourny | Zsolt Venczel |
+| [HDFS-12950](https://issues.apache.org/jira/browse/HDFS-12950) | [oiv] ls will fail in  secure cluster |  Major | . | Brahma Reddy Battula | Wei-Chiu Chuang |
+| [YARN-8386](https://issues.apache.org/jira/browse/YARN-8386) |  App log can not be viewed from Logs tab in secure cluster |  Critical | yarn-ui-v2 | Yesha Vora | Sunil Govindan |
+| [YARN-8359](https://issues.apache.org/jira/browse/YARN-8359) | Exclude containermanager.linux test classes on Windows |  Major | . | Giovanni Matteo Fumarola | Jason Lowe |
+| [HDFS-13664](https://issues.apache.org/jira/browse/HDFS-13664) | Refactor ConfiguredFailoverProxyProvider to make inheritance easier |  Minor | hdfs-client | Chao Sun | Chao Sun |
+| [HDFS-12670](https://issues.apache.org/jira/browse/HDFS-12670) | can't renew HDFS tokens with only the hdfs client jar |  Critical | . | Thomas Graves | Arpit Agarwal |
+| [HDFS-13667](https://issues.apache.org/jira/browse/HDFS-13667) | Typo: Marking all "datandoes" as stale |  Trivial | namenode | Wei-Chiu Chuang | Nanda kumar |
+| [YARN-8413](https://issues.apache.org/jira/browse/YARN-8413) | Flow activity page is failing with "Timeline server failed with an error" |  Major | yarn-ui-v2 | Yesha Vora | Sunil Govindan |
+| [YARN-8405](https://issues.apache.org/jira/browse/YARN-8405) | RM zk-state-store.parent-path ACLs has been changed since HADOOP-14773 |  Major | . | Rohith Sharma K S | Íñigo Goiri |
+| [YARN-8419](https://issues.apache.org/jira/browse/YARN-8419) | [UI2] User cannot submit a new service as submit button is always disabled |  Major | . | Suma Shivaprasad | Suma Shivaprasad |
+| [MAPREDUCE-7108](https://issues.apache.org/jira/browse/MAPREDUCE-7108) | TestFileOutputCommitter fails on Windows |  Minor | test | Zuoming Zhang | Zuoming Zhang |
+| [MAPREDUCE-7101](https://issues.apache.org/jira/browse/MAPREDUCE-7101) | Add config parameter to allow JHS to alway scan user dir irrespective of modTime |  Critical | . | Wangda Tan | Thomas Marquardt |
+| [HADOOP-15527](https://issues.apache.org/jira/browse/HADOOP-15527) | loop until TIMEOUT before sending kill -9 |  Major | . | Vinod Kumar Vavilapalli | Vinod Kumar Vavilapalli |
+| [YARN-8404](https://issues.apache.org/jira/browse/YARN-8404) | Timeline event publish need to be async to avoid Dispatcher thread leak in case ATS is down |  Blocker | . | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8410](https://issues.apache.org/jira/browse/YARN-8410) | Registry DNS lookup fails to return for CNAMEs |  Major | yarn-native-services | Shane Kumpf | Shane Kumpf |
+| [HDFS-13675](https://issues.apache.org/jira/browse/HDFS-13675) | Speed up TestDFSAdminWithHA |  Major | hdfs, namenode | Lukas Majercak | Lukas Majercak |
+| [HDFS-13673](https://issues.apache.org/jira/browse/HDFS-13673) | TestNameNodeMetrics fails on Windows |  Minor | test | Zuoming Zhang | Zuoming Zhang |
+| [HDFS-13676](https://issues.apache.org/jira/browse/HDFS-13676) | TestEditLogRace fails on Windows |  Minor | test | Zuoming Zhang | Zuoming Zhang |
+| [HDFS-13174](https://issues.apache.org/jira/browse/HDFS-13174) | hdfs mover -p /path times out after 20 min |  Major | balancer & mover | Istvan Fajth | Istvan Fajth |
+| [HADOOP-15523](https://issues.apache.org/jira/browse/HADOOP-15523) | Shell command timeout given is in seconds whereas it is taken as millisec while scheduling |  Major | . | Bilwa S T | Bilwa S T |
+| [HDFS-13682](https://issues.apache.org/jira/browse/HDFS-13682) | Cannot create encryption zone after KMS auth token expires |  Critical | encryption, kms, namenode | Xiao Chen | Xiao Chen |
+| [YARN-8445](https://issues.apache.org/jira/browse/YARN-8445) | YARN native service doesn't allow service name equals to component name |  Major | . | Chandni Singh | Chandni Singh |
+| [YARN-8444](https://issues.apache.org/jira/browse/YARN-8444) | NodeResourceMonitor crashes on bad swapFree value |  Major | . | Jim Brennan | Jim Brennan |
+| [YARN-8326](https://issues.apache.org/jira/browse/YARN-8326) | Yarn 3.0 seems runs slower than Yarn 2.6 |  Major | yarn | Hsin-Liang Huang | Shane Kumpf |
+| [YARN-8443](https://issues.apache.org/jira/browse/YARN-8443) | Total #VCores in cluster metrics is wrong when CapacityScheduler reserved some containers |  Major | webapp | Tao Yang | Tao Yang |
+| [YARN-8457](https://issues.apache.org/jira/browse/YARN-8457) | Compilation is broken with -Pyarn-ui |  Major | webapp | Sunil Govindan | Sunil Govindan |
+| [YARN-8464](https://issues.apache.org/jira/browse/YARN-8464) | Async scheduling thread could be interrupted when there are no NodeManagers in cluster |  Blocker | capacity scheduler | Charan Hebri | Sunil Govindan |
+| [YARN-8423](https://issues.apache.org/jira/browse/YARN-8423) | GPU does not get released even though the application gets killed. |  Critical | yarn | Sumana Sathish | Sunil Govindan |
+| [YARN-8401](https://issues.apache.org/jira/browse/YARN-8401) | [UI2] new ui is not accessible with out internet connection |  Blocker | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HDFS-13705](https://issues.apache.org/jira/browse/HDFS-13705) | The native ISA-L library loading failure should be made warning rather than an error message |  Minor | erasure-coding | Nilotpal Nandi | Shashikant Banerjee |
+| [YARN-8409](https://issues.apache.org/jira/browse/YARN-8409) | ActiveStandbyElectorBasedElectorService is failing with NPE |  Major | . | Yesha Vora | Chandni Singh |
+| [YARN-8379](https://issues.apache.org/jira/browse/YARN-8379) | Improve balancing resources in already satisfied queues by using Capacity Scheduler preemption |  Major | . | Wangda Tan | Zian Chen |
+| [YARN-8455](https://issues.apache.org/jira/browse/YARN-8455) | Add basic ACL check for all ATS v2 REST APIs |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-8469](https://issues.apache.org/jira/browse/YARN-8469) | [UI2] URL needs to be trimmed to handle index.html redirection while accessing via knox |  Major | yarn-ui-v2 | Sunil Govindan | Sunil Govindan |
+| [YARN-8451](https://issues.apache.org/jira/browse/YARN-8451) | Multiple NM heartbeat thread created when a slow NM resync with RM |  Major | nodemanager | Botong Huang | Botong Huang |
+| [HADOOP-15548](https://issues.apache.org/jira/browse/HADOOP-15548) | Randomize local dirs |  Minor | . | Jim Brennan | Jim Brennan |
+| [HADOOP-15574](https://issues.apache.org/jira/browse/HADOOP-15574) | Suppress build error if there are no docs after excluding private annotations |  Major | . | Takanobu Asanuma | Takanobu Asanuma |
+| [HDFS-13702](https://issues.apache.org/jira/browse/HDFS-13702) | Remove HTrace hooks from DFSClient to reduce CPU usage |  Major | performance | Todd Lipcon | Todd Lipcon |
+| [HDFS-13635](https://issues.apache.org/jira/browse/HDFS-13635) | Incorrect message when block is not found |  Major | datanode | Wei-Chiu Chuang | Gabor Bota |
+| [YARN-8415](https://issues.apache.org/jira/browse/YARN-8415) | TimelineWebServices.getEntity should throw ForbiddenException instead of 404 when ACL checks fail |  Major | . | Sumana Sathish | Suma Shivaprasad |
+| [HDFS-13715](https://issues.apache.org/jira/browse/HDFS-13715) | diskbalancer does not work if one of the blockpools are empty on a Federated cluster |  Major | diskbalancer | Namit Maheshwari | Bharat Viswanadham |
+| [YARN-8459](https://issues.apache.org/jira/browse/YARN-8459) | Improve Capacity Scheduler logs to debug invalid states |  Major | capacity scheduler | Wangda Tan | Wangda Tan |
+| [HADOOP-15571](https://issues.apache.org/jira/browse/HADOOP-15571) | Multiple FileContexts created with the same configuration object should be allowed to have different umask |  Critical | . | Vinod Kumar Vavilapalli | Vinod Kumar Vavilapalli |
+| [HDFS-13121](https://issues.apache.org/jira/browse/HDFS-13121) | NPE when request file descriptors when SC read |  Minor | hdfs-client | Gang Xie | Zsolt Venczel |
+| [YARN-6265](https://issues.apache.org/jira/browse/YARN-6265) | yarn.resourcemanager.fail-fast is used inconsistently |  Major | resourcemanager | Daniel Templeton | Yuanbo Liu |
+| [YARN-8473](https://issues.apache.org/jira/browse/YARN-8473) | Containers being launched as app tears down can leave containers in NEW state |  Major | nodemanager | Jason Lowe | Jason Lowe |
+| [YARN-8512](https://issues.apache.org/jira/browse/YARN-8512) | ATSv2 entities are not published to HBase from second attempt onwards |  Major | . | Yesha Vora | Rohith Sharma K S |
+| [YARN-8491](https://issues.apache.org/jira/browse/YARN-8491) | TestServiceCLI#testEnableFastLaunch fail when umask is 077 |  Major | . | K G Bakthavachalam | K G Bakthavachalam |
+| [HADOOP-15541](https://issues.apache.org/jira/browse/HADOOP-15541) | AWS SDK can mistake stream timeouts for EOF and throw SdkClientExceptions |  Major | fs/s3 | Sean Mackrory | Sean Mackrory |
+| [HDFS-13723](https://issues.apache.org/jira/browse/HDFS-13723) | Occasional "Should be different group" error in TestRefreshUserMappings#testGroupMappingRefresh |  Major | security, test | Siyao Meng | Siyao Meng |
+| [HDFS-12837](https://issues.apache.org/jira/browse/HDFS-12837) | Intermittent failure in TestReencryptionWithKMS |  Major | encryption, test | Surendra Singh Lilhore | Xiao Chen |
+| [HDFS-13729](https://issues.apache.org/jira/browse/HDFS-13729) | Fix broken links to RBF documentation |  Minor | documentation | jwhitter | Gabor Bota |
+| [YARN-8518](https://issues.apache.org/jira/browse/YARN-8518) | test-container-executor test\_is\_empty() is broken |  Major | . | Jim Brennan | Jim Brennan |
+| [YARN-8515](https://issues.apache.org/jira/browse/YARN-8515) | container-executor can crash with SIGPIPE after nodemanager restart |  Major | . | Jim Brennan | Jim Brennan |
+| [YARN-8421](https://issues.apache.org/jira/browse/YARN-8421) | when moving app, activeUsers is increased, even though app does not have outstanding request |  Major | . | kyungwan nam |  |
+| [YARN-8511](https://issues.apache.org/jira/browse/YARN-8511) | When AM releases a container, RM removes allocation tags before it is released by NM |  Major | capacity scheduler | Weiwei Yang | Weiwei Yang |
+| [HDFS-13524](https://issues.apache.org/jira/browse/HDFS-13524) | Occasional "All datanodes are bad" error in TestLargeBlock#testLargeBlockSize |  Major | . | Wei-Chiu Chuang | Siyao Meng |
+| [YARN-8538](https://issues.apache.org/jira/browse/YARN-8538) | Fix valgrind leak check on container executor |  Major | . | Billie Rinaldi | Billie Rinaldi |
+| [HADOOP-15610](https://issues.apache.org/jira/browse/HADOOP-15610) | Hadoop Docker Image Pip Install Fails |  Critical | . | Jack Bearden | Jack Bearden |
+| [HADOOP-15614](https://issues.apache.org/jira/browse/HADOOP-15614) | TestGroupsCaching.testExceptionOnBackgroundRefreshHandled reliably fails |  Major | . | Kihwal Lee | Weiwei Yang |
+| [MAPREDUCE-7118](https://issues.apache.org/jira/browse/MAPREDUCE-7118) | Distributed cache conflicts breaks backwards compatability |  Blocker | mrv2 | Jason Lowe | Jason Lowe |
+| [YARN-8528](https://issues.apache.org/jira/browse/YARN-8528) | Final states in ContainerAllocation might be modified externally causing unexpected allocation results |  Major | capacity scheduler | Xintong Song | Xintong Song |
+| [YARN-8541](https://issues.apache.org/jira/browse/YARN-8541) | RM startup failure on recovery after user deletion |  Blocker | resourcemanager | yimeng | Bibin A Chundatt |
+| [HADOOP-15593](https://issues.apache.org/jira/browse/HADOOP-15593) | UserGroupInformation TGT renewer throws NPE |  Blocker | security | Wei-Chiu Chuang | Gabor Bota |
+| [HDFS-13765](https://issues.apache.org/jira/browse/HDFS-13765) | Fix javadoc for FSDirMkdirOp#createParentDirectories |  Minor | documentation | Lokesh Jain | Lokesh Jain |
+| [YARN-8508](https://issues.apache.org/jira/browse/YARN-8508) | On NodeManager container gets cleaned up before its pid file is created |  Critical | . | Sumana Sathish | Chandni Singh |
+| [YARN-8434](https://issues.apache.org/jira/browse/YARN-8434) | Update federation documentation of Nodemanager configurations |  Minor | . | Bibin A Chundatt | Bibin A Chundatt |
+| [YARN-8591](https://issues.apache.org/jira/browse/YARN-8591) | [ATSv2] NPE while checking for entity acl in non-secure cluster |  Major | timelinereader, timelineserver | Akhil PB | Rohith Sharma K S |
+| [YARN-8558](https://issues.apache.org/jira/browse/YARN-8558) | NM recovery level db not cleaned up properly on container finish |  Critical | . | Bibin A Chundatt | Bibin A Chundatt |
+| [YARN-8418](https://issues.apache.org/jira/browse/YARN-8418) | App local logs could leaked if log aggregation fails to initialize for the app |  Critical | . | Bibin A Chundatt | Bibin A Chundatt |
+| [YARN-8522](https://issues.apache.org/jira/browse/YARN-8522) | Application fails with InvalidResourceRequestException |  Critical | . | Yesha Vora | Zian Chen |
+| [YARN-8606](https://issues.apache.org/jira/browse/YARN-8606) | Opportunistic scheduling does not work post RM failover |  Blocker | . | Bibin A Chundatt | Bibin A Chundatt |
+| [YARN-8600](https://issues.apache.org/jira/browse/YARN-8600) | RegistryDNS hang when remote lookup does not reply |  Critical | yarn | Eric Yang | Eric Yang |
+
+
+### TESTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [MAPREDUCE-7066](https://issues.apache.org/jira/browse/MAPREDUCE-7066) | TestQueue fails on Java9 |  Major | test | Takanobu Asanuma | Takanobu Asanuma |
+| [HADOOP-15313](https://issues.apache.org/jira/browse/HADOOP-15313) | TestKMS should close providers |  Major | kms, test | Xiao Chen | Xiao Chen |
+| [HDFS-13129](https://issues.apache.org/jira/browse/HDFS-13129) | Add a test for DfsAdmin refreshSuperUserGroupsConfiguration |  Minor | namenode | Mukul Kumar Singh | Mukul Kumar Singh |
+| [HDFS-13503](https://issues.apache.org/jira/browse/HDFS-13503) | Fix TestFsck test failures on Windows |  Major | hdfs | Xiao Liang | Xiao Liang |
+| [HDFS-13315](https://issues.apache.org/jira/browse/HDFS-13315) | Add a test for the issue reported in HDFS-11481 which is fixed by HDFS-10997. |  Major | . | Yongjun Zhang | Yongjun Zhang |
+| [HDFS-13542](https://issues.apache.org/jira/browse/HDFS-13542) | TestBlockManager#testNeededReplicationWhileAppending fails due to improper cluster shutdown in TestBlockManager#testBlockManagerMachinesArray on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13551](https://issues.apache.org/jira/browse/HDFS-13551) | TestMiniDFSCluster#testClusterSetStorageCapacity does not shut down cluster |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-11700](https://issues.apache.org/jira/browse/HDFS-11700) | TestHDFSServerPorts#testBackupNodePorts doesn't pass on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13548](https://issues.apache.org/jira/browse/HDFS-13548) | TestResolveHdfsSymlink#testFcResolveAfs fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13567](https://issues.apache.org/jira/browse/HDFS-13567) | TestNameNodeMetrics#testGenerateEDEKTime,TestNameNodeMetrics#testResourceCheck should use a different cluster basedir |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13557](https://issues.apache.org/jira/browse/HDFS-13557) | TestDFSAdmin#testListOpenFiles fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13559](https://issues.apache.org/jira/browse/HDFS-13559) | TestBlockScanner does not close TestContext properly |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13570](https://issues.apache.org/jira/browse/HDFS-13570) | TestQuotaByStorageType,TestQuota,TestDFSOutputStream fail on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13558](https://issues.apache.org/jira/browse/HDFS-13558) | TestDatanodeHttpXFrame does not shut down cluster |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13554](https://issues.apache.org/jira/browse/HDFS-13554) | TestDatanodeRegistration#testForcedRegistration does not shut down cluster |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13556](https://issues.apache.org/jira/browse/HDFS-13556) | TestNestedEncryptionZones does not shut down cluster |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13560](https://issues.apache.org/jira/browse/HDFS-13560) | Insufficient system resources exist to complete the requested service for some tests on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13592](https://issues.apache.org/jira/browse/HDFS-13592) | TestNameNodePrunesMissingStorages#testNameNodePrunesUnreportedStorages does not shut down cluster properly |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13593](https://issues.apache.org/jira/browse/HDFS-13593) | TestBlockReaderLocalLegacy#testBlockReaderLocalLegacyWithAppend fails on Windows |  Minor | test | Anbang Hu | Anbang Hu |
+| [HDFS-13587](https://issues.apache.org/jira/browse/HDFS-13587) | TestQuorumJournalManager fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13619](https://issues.apache.org/jira/browse/HDFS-13619) | TestAuditLoggerWithCommands fails on Windows |  Minor | test | Anbang Hu | Anbang Hu |
+| [HDFS-13620](https://issues.apache.org/jira/browse/HDFS-13620) | Randomize the test directory path for TestHDFSFileSystemContract |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13627](https://issues.apache.org/jira/browse/HDFS-13627) | TestErasureCodingExerciseAPIs fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13591](https://issues.apache.org/jira/browse/HDFS-13591) | TestDFSShell#testSetrepLow fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13632](https://issues.apache.org/jira/browse/HDFS-13632) | Randomize baseDir for MiniJournalCluster in MiniQJMHACluster for TestDFSAdminWithHA |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13629](https://issues.apache.org/jira/browse/HDFS-13629) | Some tests in TestDiskBalancerCommand fail on Windows due to MiniDFSCluster path conflict and improper path usage |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13631](https://issues.apache.org/jira/browse/HDFS-13631) | TestDFSAdmin#testCheckNumOfBlocksInReportCommand should use a separate MiniDFSCluster path |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13651](https://issues.apache.org/jira/browse/HDFS-13651) | TestReencryptionHandler fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13648](https://issues.apache.org/jira/browse/HDFS-13648) | Fix TestGetConf#testGetJournalNodes on Windows due to a mismatch line separator |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [MAPREDUCE-7102](https://issues.apache.org/jira/browse/MAPREDUCE-7102) | Fix TestJavaSerialization for Windows due a mismatch line separator |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [MAPREDUCE-7105](https://issues.apache.org/jira/browse/MAPREDUCE-7105) | Fix TestNativeCollectorOnlyHandler.testOnCall on Windows because of the path format |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HDFS-13652](https://issues.apache.org/jira/browse/HDFS-13652) | Randomize baseDir for MiniDFSCluster in TestBlockScanner |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13649](https://issues.apache.org/jira/browse/HDFS-13649) | Randomize baseDir for MiniDFSCluster in TestReconstructStripedFile and TestReconstructStripedFileWithRandomECPolicy |  Minor | . | Anbang Hu | Anbang Hu |
+| [HDFS-13650](https://issues.apache.org/jira/browse/HDFS-13650) | Randomize baseDir for MiniDFSCluster in TestDFSStripedInputStream and TestDFSStripedInputStreamWithRandomECPolicy |  Minor | . | Anbang Hu | Anbang Hu |
+| [YARN-8370](https://issues.apache.org/jira/browse/YARN-8370) | Some Node Manager tests fail on Windows due to improper path/file separator |  Minor | . | Anbang Hu | Anbang Hu |
+| [YARN-8422](https://issues.apache.org/jira/browse/YARN-8422) | TestAMSimulator failing with NPE |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HADOOP-15532](https://issues.apache.org/jira/browse/HADOOP-15532) | TestBasicDiskValidator fails with NoSuchFileException |  Minor | . | Íñigo Goiri | Giovanni Matteo Fumarola |
+| [HDFS-13563](https://issues.apache.org/jira/browse/HDFS-13563) | TestDFSAdminWithHA times out on Windows |  Minor | . | Anbang Hu | Lukas Majercak |
+| [HDFS-13681](https://issues.apache.org/jira/browse/HDFS-13681) | Fix TestStartup.testNNFailToStartOnReadOnlyNNDir test failure on Windows |  Major | test | Xiao Liang | Xiao Liang |
+
+
+### SUB-TASKS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-8002](https://issues.apache.org/jira/browse/YARN-8002) | Support NOT\_SELF and ALL namespace types for allocation tag |  Major | resourcemanager | Weiwei Yang | Weiwei Yang |
+| [HDFS-13289](https://issues.apache.org/jira/browse/HDFS-13289) | RBF: TestConnectionManager#testCleanup() test case need correction |  Minor | . | Dibyendu Karmakar | Dibyendu Karmakar |
+| [YARN-8013](https://issues.apache.org/jira/browse/YARN-8013) | Support application tags when defining application namespaces for placement constraints |  Major | . | Weiwei Yang | Weiwei Yang |
+| [YARN-6936](https://issues.apache.org/jira/browse/YARN-6936) | [Atsv2] Retrospect storing entities into sub application table from client perspective |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [HDFS-13353](https://issues.apache.org/jira/browse/HDFS-13353) | RBF: TestRouterWebHDFSContractCreate failed |  Major | test | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-8107](https://issues.apache.org/jira/browse/YARN-8107) | Give an informative message when incorrect format is used in ATSv2 filter attributes |  Major | ATSv2 | Charan Hebri | Rohith Sharma K S |
+| [YARN-8110](https://issues.apache.org/jira/browse/YARN-8110) | AMRMProxy recover should catch for all throwable to avoid premature exit |  Major | . | Botong Huang | Botong Huang |
+| [YARN-8048](https://issues.apache.org/jira/browse/YARN-8048) | Support auto-spawning of admin configured services during bootstrap of rm/apiserver |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [HDFS-13402](https://issues.apache.org/jira/browse/HDFS-13402) | RBF: Fix  java doc for StateStoreFileSystemImpl |  Minor | hdfs | Yiran Wu | Yiran Wu |
+| [YARN-7574](https://issues.apache.org/jira/browse/YARN-7574) | Add support for Node Labels on Auto Created Leaf Queue Template |  Major | capacity scheduler | Suma Shivaprasad | Suma Shivaprasad |
+| [HDFS-13410](https://issues.apache.org/jira/browse/HDFS-13410) | RBF: Support federation with no subclusters |  Minor | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-13384](https://issues.apache.org/jira/browse/HDFS-13384) | RBF: Improve timeout RPC call mechanism |  Minor | . | Íñigo Goiri | Íñigo Goiri |
+| [HADOOP-15376](https://issues.apache.org/jira/browse/HADOOP-15376) | Remove double semi colons on imports that make Clover fall over. |  Minor | . | Ewan Higgs | Ewan Higgs |
+| [YARN-7973](https://issues.apache.org/jira/browse/YARN-7973) | Support ContainerRelaunch for Docker containers |  Major | . | Shane Kumpf | Shane Kumpf |
+| [YARN-7941](https://issues.apache.org/jira/browse/YARN-7941) | Transitive dependencies for component are not resolved |  Major | . | Rohith Sharma K S | Billie Rinaldi |
+| [HADOOP-15346](https://issues.apache.org/jira/browse/HADOOP-15346) | S3ARetryPolicy for 400/BadArgument to be "fail" |  Major | fs/s3 | Steve Loughran | Steve Loughran |
+| [HDFS-13045](https://issues.apache.org/jira/browse/HDFS-13045) | RBF: Improve error message returned from subcluster |  Minor | . | Wei Yan | Íñigo Goiri |
+| [HDFS-13428](https://issues.apache.org/jira/browse/HDFS-13428) | RBF: Remove LinkedList From StateStoreFileImpl.java |  Trivial | federation | BELUGA BEHR | BELUGA BEHR |
+| [HDFS-13386](https://issues.apache.org/jira/browse/HDFS-13386) | RBF: Wrong date information in list file(-ls) result |  Minor | . | Dibyendu Karmakar | Dibyendu Karmakar |
+| [YARN-7221](https://issues.apache.org/jira/browse/YARN-7221) | Add security check for privileged docker container |  Major | security | Eric Yang | Eric Yang |
+| [YARN-7936](https://issues.apache.org/jira/browse/YARN-7936) | Add default service AM Xmx |  Major | . | Jian He | Jian He |
+| [YARN-8018](https://issues.apache.org/jira/browse/YARN-8018) | Yarn Service Upgrade: Add support for initiating service upgrade |  Major | . | Chandni Singh | Chandni Singh |
+| [HADOOP-14999](https://issues.apache.org/jira/browse/HADOOP-14999) | AliyunOSS: provide one asynchronous multi-part based uploading mechanism |  Major | fs/oss | Genmao Yu | Genmao Yu |
+| [YARN-7142](https://issues.apache.org/jira/browse/YARN-7142) | Support placement policy in yarn native services |  Major | yarn-native-services | Billie Rinaldi | Gour Saha |
+| [YARN-8138](https://issues.apache.org/jira/browse/YARN-8138) | Add unit test to validate queue priority preemption works under node partition. |  Minor | . | Charan Hebri | Zian Chen |
+| [YARN-8060](https://issues.apache.org/jira/browse/YARN-8060) | Create default readiness check for service components |  Major | yarn-native-services | Billie Rinaldi | Billie Rinaldi |
+| [HDFS-13435](https://issues.apache.org/jira/browse/HDFS-13435) | RBF: Improve the error loggings for printing the stack trace |  Major | . | Yiqun Lin | Yiqun Lin |
+| [YARN-8126](https://issues.apache.org/jira/browse/YARN-8126) | Support auto-spawning of admin configured services during bootstrap of RM |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-7996](https://issues.apache.org/jira/browse/YARN-7996) | Allow user supplied Docker client configurations with YARN native services |  Major | . | Shane Kumpf | Shane Kumpf |
+| [HDFS-13466](https://issues.apache.org/jira/browse/HDFS-13466) | RBF: Add more router-related information to the UI |  Minor | . | Wei Yan | Wei Yan |
+| [YARN-5888](https://issues.apache.org/jira/browse/YARN-5888) | [UI2] Improve unit tests for new YARN UI |  Minor | yarn-ui-v2 | Akhil PB | Akhil PB |
+| [HDFS-13453](https://issues.apache.org/jira/browse/HDFS-13453) | RBF: getMountPointDates should fetch latest subdir time/date when parent dir is not present but /parent/child dirs are present in mount table |  Major | . | Dibyendu Karmakar | Dibyendu Karmakar |
+| [YARN-8111](https://issues.apache.org/jira/browse/YARN-8111) | Simplify PlacementConstraints API by removing allocationTagToIntraApp |  Minor | resourcemanager | Weiwei Yang | Weiwei Yang |
+| [YARN-8064](https://issues.apache.org/jira/browse/YARN-8064) | Docker ".cmd" files should not be put in hadoop.tmp.dir |  Critical | . | Eric Badger | Eric Badger |
+| [HDFS-13478](https://issues.apache.org/jira/browse/HDFS-13478) | RBF: Disabled Nameservice store API |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-8177](https://issues.apache.org/jira/browse/YARN-8177) | Documentation changes for auto creation of Leaf Queues with node label |  Major | . | Suma Shivaprasad | Suma Shivaprasad |
+| [HDFS-13490](https://issues.apache.org/jira/browse/HDFS-13490) | RBF: Fix setSafeMode in the Router |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-13484](https://issues.apache.org/jira/browse/HDFS-13484) | RBF: Disable Nameservices from the federation |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-7939](https://issues.apache.org/jira/browse/YARN-7939) | Yarn Service Upgrade: add support to upgrade a component instance |  Major | . | Chandni Singh | Chandni Singh |
+| [HDFS-13326](https://issues.apache.org/jira/browse/HDFS-13326) | RBF: Improve the interfaces to modify and view mount tables |  Minor | . | Wei Yan | Gang Li |
+| [YARN-8122](https://issues.apache.org/jira/browse/YARN-8122) | Component health threshold monitor |  Major | . | Gour Saha | Gour Saha |
+| [HDFS-13499](https://issues.apache.org/jira/browse/HDFS-13499) | RBF: Show disabled name services in the UI |  Minor | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-8215](https://issues.apache.org/jira/browse/YARN-8215) | ATS v2 returns invalid YARN\_CONTAINER\_ALLOCATED\_HOST\_HTTP\_ADDRESS from NM |  Critical | ATSv2 | Yesha Vora | Rohith Sharma K S |
+| [YARN-8152](https://issues.apache.org/jira/browse/YARN-8152) | Add chart in SLS to illustrate the throughput of the scheduler |  Major | scheduler-load-simulator | Weiwei Yang | Tao Yang |
+| [YARN-8204](https://issues.apache.org/jira/browse/YARN-8204) | Yarn Service Upgrade: Add a flag to disable upgrade |  Major | . | Chandni Singh | Chandni Singh |
+| [YARN-7781](https://issues.apache.org/jira/browse/YARN-7781) | Update YARN-Services-Examples.md to be in sync with the latest code |  Major | . | Gour Saha | Gour Saha |
+| [HDFS-13508](https://issues.apache.org/jira/browse/HDFS-13508) | RBF: Normalize paths (automatically) when adding, updating, removing or listing mount table entries |  Minor | . | Ekanth Sethuramalingam | Ekanth Sethuramalingam |
+| [HDFS-13434](https://issues.apache.org/jira/browse/HDFS-13434) | RBF: Fix dead links in RBF document |  Major | documentation | Akira Ajisaka | Chetna Chaudhari |
+| [YARN-8195](https://issues.apache.org/jira/browse/YARN-8195) | Fix constraint cardinality check in the presence of multiple target allocation tags |  Critical | . | Weiwei Yang | Weiwei Yang |
+| [YARN-8228](https://issues.apache.org/jira/browse/YARN-8228) | Docker does not support hostnames greater than 64 characters |  Critical | yarn-native-services | Yesha Vora | Shane Kumpf |
+| [YARN-8212](https://issues.apache.org/jira/browse/YARN-8212) | Pending backlog for async allocation threads should be configurable |  Major | . | Weiwei Yang | Tao Yang |
+| [YARN-2674](https://issues.apache.org/jira/browse/YARN-2674) | Distributed shell AM may re-launch containers if RM work preserving restart happens |  Major | applications, resourcemanager | Chun Chen | Shane Kumpf |
+| [HDFS-13488](https://issues.apache.org/jira/browse/HDFS-13488) | RBF: Reject requests when a Router is overloaded |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-8113](https://issues.apache.org/jira/browse/YARN-8113) | Update placement constraints doc with application namespaces and inter-app constraints |  Major | documentation | Weiwei Yang | Weiwei Yang |
+| [YARN-8194](https://issues.apache.org/jira/browse/YARN-8194) | Exception when reinitializing a container using LinuxContainerExecutor |  Blocker | . | Chandni Singh | Chandni Singh |
+| [YARN-7961](https://issues.apache.org/jira/browse/YARN-7961) | Improve status response when yarn application is destroyed |  Major | yarn-native-services | Yesha Vora | Gour Saha |
+| [HDFS-13525](https://issues.apache.org/jira/browse/HDFS-13525) | RBF: Add unit test TestStateStoreDisabledNameservice |  Major | . | Yiqun Lin | Yiqun Lin |
+| [YARN-5151](https://issues.apache.org/jira/browse/YARN-5151) | [UI2] Support kill application from new YARN UI |  Major | . | Wangda Tan | Gergely Novák |
+| [YARN-8253](https://issues.apache.org/jira/browse/YARN-8253) | HTTPS Ats v2 api call fails with "bad HTTP parsed" |  Critical | ATSv2 | Yesha Vora | Charan Hebri |
+| [YARN-8207](https://issues.apache.org/jira/browse/YARN-8207) | Docker container launch use popen have risk of shell expansion |  Blocker | yarn-native-services | Eric Yang | Eric Yang |
+| [YARN-8261](https://issues.apache.org/jira/browse/YARN-8261) | Docker container launch fails due to .cmd file creation failure |  Blocker | . | Eric Badger | Jason Lowe |
+| [HADOOP-15454](https://issues.apache.org/jira/browse/HADOOP-15454) | TestRollingFileSystemSinkWithLocal fails on Windows |  Major | test | Xiao Liang | Xiao Liang |
+| [HDFS-13346](https://issues.apache.org/jira/browse/HDFS-13346) | RBF: Fix synchronization of router quota and nameservice quota |  Major | . | liuhongtong | Yiqun Lin |
+| [YARN-8243](https://issues.apache.org/jira/browse/YARN-8243) | Flex down should remove instance with largest component instance ID first |  Critical | yarn-native-services | Gour Saha | Gour Saha |
+| [YARN-7654](https://issues.apache.org/jira/browse/YARN-7654) | Support ENTRY\_POINT for docker container |  Blocker | yarn | Eric Yang | Eric Yang |
+| [YARN-8247](https://issues.apache.org/jira/browse/YARN-8247) | Incorrect HTTP status code returned by ATSv2 for non-whitelisted users |  Critical | ATSv2 | Charan Hebri | Rohith Sharma K S |
+| [YARN-8130](https://issues.apache.org/jira/browse/YARN-8130) | Race condition when container events are published for KILLED applications |  Major | ATSv2 | Charan Hebri | Rohith Sharma K S |
+| [YARN-8081](https://issues.apache.org/jira/browse/YARN-8081) | Yarn Service Upgrade: Add support to upgrade a component |  Major | . | Chandni Singh | Chandni Singh |
+| [YARN-8284](https://issues.apache.org/jira/browse/YARN-8284) | get\_docker\_command refactoring |  Minor | . | Jason Lowe | Eric Badger |
+| [HADOOP-15469](https://issues.apache.org/jira/browse/HADOOP-15469) | S3A directory committer commit job fails if \_temporary directory created under dest |  Major | fs/s3 | Steve Loughran | Steve Loughran |
+| [YARN-8206](https://issues.apache.org/jira/browse/YARN-8206) | Sending a kill does not immediately kill docker containers |  Major | . | Eric Badger | Eric Badger |
+| [YARN-7960](https://issues.apache.org/jira/browse/YARN-7960) | Add no-new-privileges flag to docker run |  Major | . | Eric Badger | Eric Badger |
+| [YARN-7530](https://issues.apache.org/jira/browse/YARN-7530) | hadoop-yarn-services-api should be part of hadoop-yarn-services |  Blocker | yarn-native-services | Eric Yang | Chandni Singh |
+| [YARN-6919](https://issues.apache.org/jira/browse/YARN-6919) | Add default volume mount list |  Major | yarn | Eric Badger | Eric Badger |
+| [HADOOP-15498](https://issues.apache.org/jira/browse/HADOOP-15498) | TestHadoopArchiveLogs (#testGenerateScript, #testPrepareWorkingDir) fails on Windows |  Minor | . | Anbang Hu | Anbang Hu |
+| [YARN-8329](https://issues.apache.org/jira/browse/YARN-8329) | Docker client configuration can still be set incorrectly |  Major | . | Shane Kumpf | Shane Kumpf |
+| [HDFS-12978](https://issues.apache.org/jira/browse/HDFS-12978) | Fine-grained locking while consuming journal stream. |  Major | namenode | Konstantin Shvachko | Konstantin Shvachko |
+| [YARN-8384](https://issues.apache.org/jira/browse/YARN-8384) | stdout.txt, stderr.txt logs of a launched docker container is coming with primary group of submit user instead of hadoop |  Critical | yarn-native-services | Sunil Govindan | Eric Yang |
+| [YARN-8349](https://issues.apache.org/jira/browse/YARN-8349) | Remove YARN registry entries when a service is killed by the RM |  Critical | yarn-native-services | Shane Kumpf | Billie Rinaldi |
+| [HDFS-13637](https://issues.apache.org/jira/browse/HDFS-13637) | RBF: Router fails when threadIndex (in ConnectionPool) wraps around Integer.MIN\_VALUE |  Critical | federation | CR Hota | CR Hota |
+| [YARN-8342](https://issues.apache.org/jira/browse/YARN-8342) | Using docker image from a non-privileged registry, the launch\_command is not honored |  Critical | . | Wangda Tan | Eric Yang |
+| [HDFS-13281](https://issues.apache.org/jira/browse/HDFS-13281) | Namenode#createFile should be /.reserved/raw/ aware. |  Critical | encryption | Rushabh S Shah | Rushabh S Shah |
+| [YARN-4677](https://issues.apache.org/jira/browse/YARN-4677) | RMNodeResourceUpdateEvent update from scheduler can lead to race condition |  Major | graceful, resourcemanager, scheduler | Brook Zhou | Wilfred Spiegelenburg |
+| [HADOOP-15137](https://issues.apache.org/jira/browse/HADOOP-15137) | ClassNotFoundException: org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol when using hadoop-client-minicluster |  Major | . | Jeff Zhang | Bharat Viswanadham |
+| [HDFS-13547](https://issues.apache.org/jira/browse/HDFS-13547) | Add ingress port based sasl resolver |  Major | security | Chen Liang | Chen Liang |
+| [HADOOP-15514](https://issues.apache.org/jira/browse/HADOOP-15514) | NoClassDefFoundError for TimelineCollectorManager when starting MiniYARNCluster |  Major | . | Jeff Zhang | Rohith Sharma K S |
+| [HADOOP-15516](https://issues.apache.org/jira/browse/HADOOP-15516) | Add test cases to cover FileUtil#readLink |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HADOOP-15506](https://issues.apache.org/jira/browse/HADOOP-15506) | Upgrade Azure Storage Sdk version to 7.0.0 and update corresponding code blocks |  Minor | fs/azure | Esfandiar Manii | Esfandiar Manii |
+| [HADOOP-15529](https://issues.apache.org/jira/browse/HADOOP-15529) | ContainerLaunch#testInvalidEnvVariableSubstitutionType is not supported in Windows |  Minor | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-8411](https://issues.apache.org/jira/browse/YARN-8411) | Enable stopped system services to be started during RM start |  Critical | . | Billie Rinaldi | Billie Rinaldi |
+| [YARN-8259](https://issues.apache.org/jira/browse/YARN-8259) | Revisit liveliness checks for Docker containers |  Blocker | . | Shane Kumpf | Shane Kumpf |
+| [HADOOP-15533](https://issues.apache.org/jira/browse/HADOOP-15533) | Make WASB listStatus messages consistent |  Trivial | fs/azure | Esfandiar Manii | Esfandiar Manii |
+| [HADOOP-15458](https://issues.apache.org/jira/browse/HADOOP-15458) | TestLocalFileSystem#testFSOutputStreamBuilder fails on Windows |  Minor | test | Xiao Liang | Xiao Liang |
+| [YARN-8465](https://issues.apache.org/jira/browse/YARN-8465) | Dshell docker container gets marked as lost after NM restart |  Major | yarn-native-services | Yesha Vora | Shane Kumpf |
+| [YARN-8485](https://issues.apache.org/jira/browse/YARN-8485) | Priviledged container app launch is failing intermittently |  Major | yarn-native-services | Yesha Vora | Eric Yang |
+| [HDFS-13528](https://issues.apache.org/jira/browse/HDFS-13528) | RBF: If a directory exceeds quota limit then quota usage is not refreshed for other mount entries |  Major | . | Dibyendu Karmakar | Dibyendu Karmakar |
+| [HDFS-13710](https://issues.apache.org/jira/browse/HDFS-13710) | RBF:  setQuota and getQuotaUsage should check the dfs.federation.router.quota.enable |  Major | federation, hdfs | yanghuafeng | yanghuafeng |
+| [HADOOP-15384](https://issues.apache.org/jira/browse/HADOOP-15384) | distcp numListstatusThreads option doesn't get to -delete scan |  Major | tools/distcp | Steve Loughran | Steve Loughran |
+| [HDFS-13726](https://issues.apache.org/jira/browse/HDFS-13726) | RBF: Fix RBF configuration links |  Minor | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [HDFS-13475](https://issues.apache.org/jira/browse/HDFS-13475) | RBF: Admin cannot enforce Router enter SafeMode |  Major | . | Wei Yan | Chao Sun |
+| [HDFS-13733](https://issues.apache.org/jira/browse/HDFS-13733) | RBF: Add Web UI configurations and descriptions to RBF document |  Minor | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-8301](https://issues.apache.org/jira/browse/YARN-8301) | Yarn Service Upgrade: Add documentation |  Critical | . | Chandni Singh | Chandni Singh |
+| [YARN-8546](https://issues.apache.org/jira/browse/YARN-8546) | Resource leak caused by a reserved container being released more than once under async scheduling |  Major | capacity scheduler | Weiwei Yang | Tao Yang |
+
+
+### OTHER:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-8091](https://issues.apache.org/jira/browse/YARN-8091) | Revisit checkUserAccessToQueue RM REST API |  Critical | . | Wangda Tan | Wangda Tan |
+| [YARN-8274](https://issues.apache.org/jira/browse/YARN-8274) | Docker command error during container relaunch |  Critical | . | Billie Rinaldi | Jason Lowe |
+| [YARN-8080](https://issues.apache.org/jira/browse/YARN-8080) | YARN native service should support component restart policy |  Critical | . | Wangda Tan | Suma Shivaprasad |
+| [HADOOP-15483](https://issues.apache.org/jira/browse/HADOOP-15483) | Upgrade jquery to version 3.3.1 |  Major | . | Lokesh Jain | Lokesh Jain |
+| [YARN-8506](https://issues.apache.org/jira/browse/YARN-8506) | Make GetApplicationsRequestPBImpl thread safe |  Critical | . | Wangda Tan | Wangda Tan |
+
+


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


[37/50] [abbrv] hadoop git commit: HADOOP-15446. ABFS: tune imports & javadocs; stabilise tests. Contributed by Steve Loughran and Da Zhou.

Posted by tm...@apache.org.
HADOOP-15446. ABFS: tune imports & javadocs; stabilise tests.
Contributed by Steve Loughran and Da Zhou.


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

Branch: refs/heads/HADOOP-15407
Commit: 75b184c647a0b23dc56f41f91bf5102821c1ded7
Parents: 62ea8ad
Author: Thomas Marquardt <tm...@microsoft.com>
Authored: Wed Aug 8 18:52:12 2018 +0000
Committer: Thomas Marquardt <tm...@microsoft.com>
Committed: Sat Aug 11 03:42:27 2018 +0000

----------------------------------------------------------------------
 .../apache/hadoop/fs/RawLocalFileSystem.java    |   2 +-
 .../src/main/resources/core-default.xml         |  15 +
 .../src/site/markdown/filesystem/filesystem.md  |  11 +-
 .../fs/contract/AbstractContractAppendTest.java |  57 ++--
 .../fs/contract/AbstractContractConcatTest.java |  34 +--
 .../AbstractContractGetFileStatusTest.java      |  26 +-
 .../fs/contract/AbstractContractMkdirTest.java  |   8 +
 .../hadoop/fs/contract/AbstractFSContract.java  |   2 -
 .../hadoop/fs/contract/ContractTestUtils.java   |  19 +-
 .../org/apache/hadoop/fs/azurebfs/Abfs.java     |   4 +-
 .../org/apache/hadoop/fs/azurebfs/Abfss.java    |   4 +-
 .../hadoop/fs/azurebfs/AzureBlobFileSystem.java |  98 +++---
 .../fs/azurebfs/AzureBlobFileSystemStore.java   | 147 ++++-----
 .../fs/azurebfs/SecureAzureBlobFileSystem.java  |   4 +-
 .../azurebfs/constants/AbfsHttpConstants.java   |   2 +-
 .../constants/HttpHeaderConfigurations.java     |   2 +-
 .../fs/azurebfs/constants/HttpQueryParams.java  |   2 +-
 .../ConfigurationValidationAnnotations.java     |  14 +-
 .../diagnostics/ConfigurationValidator.java     |   6 +-
 .../AzureBlobFileSystemException.java           |   4 +-
 .../exceptions/InvalidUriException.java         |   4 +-
 ...Base64StringConfigurationBasicValidator.java |   2 +-
 .../BooleanConfigurationBasicValidator.java     |   4 +-
 .../ConfigurationBasicValidator.java            |   2 +-
 .../IntegerConfigurationBasicValidator.java     |   2 +-
 .../LongConfigurationBasicValidator.java        |   4 +-
 .../StringConfigurationBasicValidator.java      |   4 +-
 .../hadoop/fs/azurebfs/services/AbfsClient.java | 157 +++++-----
 .../fs/azurebfs/services/AbfsHttpOperation.java |   6 +-
 .../fs/azurebfs/services/AbfsInputStream.java   |   5 +-
 .../fs/azurebfs/services/AbfsOutputStream.java  | 125 ++++----
 .../fs/azurebfs/services/AbfsRestOperation.java |  24 +-
 .../azurebfs/services/AbfsUriQueryBuilder.java  |   6 +-
 .../services/ExponentialRetryPolicy.java        |   2 +-
 .../hadoop/fs/azurebfs/services/ReadBuffer.java |   4 +-
 .../fs/azurebfs/services/ReadBufferManager.java |  56 ++--
 .../fs/azurebfs/services/ReadBufferWorker.java  |   4 +-
 .../azurebfs/services/SharedKeyCredentials.java |  32 +-
 .../hadoop-azure/src/site/markdown/abfs.md      |  72 +++++
 .../src/site/markdown/testing_azure.md          |  76 +++++
 .../ITestAzureNativeContractAppend.java         |  23 ++
 .../azurebfs/AbstractAbfsIntegrationTest.java   | 304 +++++++++++++++++++
 .../fs/azurebfs/AbstractAbfsScaleTest.java      |  53 ++++
 .../fs/azurebfs/DependencyInjectedTest.java     | 206 -------------
 .../ITestAzureBlobFileSystemAppend.java         |  28 +-
 .../ITestAzureBlobFileSystemBackCompat.java     |  16 +-
 .../azurebfs/ITestAzureBlobFileSystemCopy.java  |  45 ++-
 .../ITestAzureBlobFileSystemCreate.java         |  54 ++--
 .../ITestAzureBlobFileSystemDelete.java         |  79 +++--
 .../azurebfs/ITestAzureBlobFileSystemE2E.java   |  66 ++--
 .../ITestAzureBlobFileSystemE2EScale.java       |  80 ++---
 .../ITestAzureBlobFileSystemFileStatus.java     |  45 ++-
 .../azurebfs/ITestAzureBlobFileSystemFlush.java | 209 +++++++------
 .../ITestAzureBlobFileSystemInitAndCreate.java  |  17 +-
 .../ITestAzureBlobFileSystemListStatus.java     | 123 +++++---
 .../azurebfs/ITestAzureBlobFileSystemMkDir.java |  55 +---
 .../azurebfs/ITestAzureBlobFileSystemOpen.java  |  41 ---
 .../ITestAzureBlobFileSystemRandomRead.java     |  48 +--
 .../ITestAzureBlobFileSystemRename.java         | 129 ++++----
 .../ITestAzureBlobFileSystemRenameUnicode.java  |  98 ++++++
 .../azurebfs/ITestFileSystemInitialization.java |  47 ++-
 .../fs/azurebfs/ITestFileSystemProperties.java  |  47 ++-
 .../azurebfs/ITestFileSystemRegistration.java   |  78 +++--
 .../fs/azurebfs/ITestWasbAbfsCompatibility.java | 166 +++++-----
 .../constants/TestConfigurationKeys.java        |  11 +-
 .../contract/ABFSContractTestBinding.java       |  64 ++++
 .../contract/AbfsFileSystemContract.java        |  65 ++++
 .../DependencyInjectedContractTest.java         |  63 ----
 .../contract/ITestAbfsFileSystemContract.java   |  54 ----
 .../ITestAbfsFileSystemContractAppend.java      |  14 +-
 .../ITestAbfsFileSystemContractConcat.java      |  14 +-
 .../ITestAbfsFileSystemContractCreate.java      |  10 +-
 .../ITestAbfsFileSystemContractDelete.java      |  12 +-
 .../ITestAbfsFileSystemContractDistCp.java      |  10 +-
 ...TestAbfsFileSystemContractGetFileStatus.java |  12 +-
 .../ITestAbfsFileSystemContractMkdir.java       |  12 +-
 .../ITestAbfsFileSystemContractOpen.java        |  12 +-
 .../ITestAbfsFileSystemContractRename.java      |  12 +-
 ...TestAbfsFileSystemContractRootDirectory.java |  12 +-
 ...ITestAbfsFileSystemContractSecureDistCp.java |  10 +-
 .../ITestAbfsFileSystemContractSeek.java        |  12 +-
 .../ITestAbfsFileSystemContractSetTimes.java    |  12 +-
 .../ITestAzureBlobFileSystemBasics.java         |  25 +-
 .../TestConfigurationValidators.java            |   5 +-
 .../services/ITestAbfsReadWriteAndSeek.java     |  91 ++++++
 .../services/ITestReadWriteAndSeek.java         |  78 -----
 .../TestAbfsConfigurationFieldsValidation.java  |   6 +-
 .../utils/CleanUpAbfsTestContainer.java         |   6 +-
 .../hadoop-azure/src/test/resources/abfs.xml    |   4 +-
 .../src/test/resources/azure-bfs-test.xml       |  14 +-
 .../src/test/resources/log4j.properties         |  32 ++
 91 files changed, 2140 insertions(+), 1641 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index bd003ae..6e9d433 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -291,7 +291,7 @@ public class RawLocalFileSystem extends FileSystem {
       Progressable progress) throws IOException {
     FileStatus status = getFileStatus(f);
     if (status.isDirectory()) {
-      throw new IOException("Cannot append to a diretory (=" + f + " )");
+      throw new FileAlreadyExistsException("Cannot append to a directory: " + f);
     }
     return new FSDataOutputStream(new BufferedOutputStream(
         createOutputStreamWithMode(f, true, null), bufferSize), statistics,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 93b8ddd..93649e7 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1615,16 +1615,31 @@
     SAS keys to communicate with Azure storage.
   </description>
 </property>
+
 <property>
   <name>fs.abfs.impl</name>
   <value>org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem</value>
   <description>The implementation class of the Azure Blob Filesystem</description>
 </property>
+
 <property>
   <name>fs.abfss.impl</name>
   <value>org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem</value>
   <description>The implementation class of the Secure Azure Blob Filesystem</description>
 </property>
+
+<property>
+  <name>fs.AbstractFileSystem.abfs.impl</name>
+  <value>org.apache.hadoop.fs.azurebfs.Abfs</value>
+  <description>AbstractFileSystem implementation class of abfs://</description>
+</property>
+
+<property>
+  <name>fs.AbstractFileSystem.abfss.impl</name>
+  <value>org.apache.hadoop.fs.azurebfs.Abfss</value>
+  <description>AbstractFileSystem implementation class of abfss://</description>
+</property>
+
 <property>
   <name>fs.azure.local.sas.key.mode</name>
   <value>false</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
index 2637f54..28c6fbe 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
@@ -544,15 +544,6 @@ atomic. The combined operation, including `mkdirs(parent(F))` MAY be atomic.
 The return value is always true&mdash;even if a new directory is not created
  (this is defined in HDFS).
 
-#### Implementation Notes: Local FileSystem
-
-The local FileSystem does not raise an exception if `mkdirs(p)` is invoked
-on a path that exists and is a file. Instead the operation returns false.
-
-    if isFile(FS, p):
-       FS' = FS
-       result = False
-
 ### <a name='FileSystem.create'></a> `FSDataOutputStream create(Path, ...)`
 
 
@@ -641,7 +632,7 @@ Implementations without a compliant call SHOULD throw `UnsupportedOperationExcep
 
     if not exists(FS, p) : raise FileNotFoundException
 
-    if not isFile(FS, p) : raise [FileNotFoundException, IOException]
+    if not isFile(FS, p) : raise [FileAlreadyExistsException, FileNotFoundException, IOException]
 
 #### Postconditions
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java
index d61b635..0be220e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java
@@ -18,7 +18,12 @@
 
 package org.apache.hadoop.fs.contract;
 
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -27,6 +32,7 @@ import org.slf4j.LoggerFactory;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test append -if supported
@@ -75,15 +81,10 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB
 
   @Test
   public void testAppendNonexistentFile() throws Throwable {
-    try {
-      FSDataOutputStream out = getFileSystem().append(target);
-      //got here: trouble
-      out.close();
-      fail("expected a failure");
-    } catch (Exception e) {
-      //expected
-      handleExpectedException(e);
-    }
+    //expected
+    handleExpectedException(
+        intercept(Exception.class,
+            () -> getFileSystem().append(target).close()));
   }
 
   @Test
@@ -116,15 +117,9 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB
 
   @Test
   public void testAppendMissingTarget() throws Throwable {
-    try {
-      FSDataOutputStream out = getFileSystem().append(target);
-      //got here: trouble
-      out.close();
-      fail("expected a failure");
-    } catch (Exception e) {
-      //expected
-      handleExpectedException(e);
-    }
+    handleExpectedException(
+        intercept(Exception.class,
+            () -> getFileSystem().append(target).close()));
   }
 
   @Test
@@ -149,4 +144,30 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB
                                                  dataset.length);
     ContractTestUtils.compareByteArrays(dataset, bytes, dataset.length);
   }
+
+  @Test
+  public void testAppendFileAfterDelete() throws Exception {
+    final FileSystem fs = getFileSystem();
+    final Path filePath = target;
+    fs.create(filePath);
+    fs.delete(filePath, false);
+    intercept(FileNotFoundException.class,
+        () -> fs.append(filePath));
+  }
+
+  @Test
+  public void testAppendDirectory() throws Exception {
+    final FileSystem fs = getFileSystem();
+
+    final Path folderPath = target;
+    fs.mkdirs(folderPath);
+    IOException ex = intercept(IOException.class,
+        () -> fs.append(folderPath));
+    if (ex instanceof FileAlreadyExistsException) {
+      handleExpectedException(ex);
+    } else {
+      handleRelaxedException("Append to a directory",
+          "FileAlreadyExistsException", ex);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java
index 7b12086..d30e0d6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java
@@ -19,15 +19,16 @@
 package org.apache.hadoop.fs.contract;
 
 import org.apache.hadoop.fs.Path;
+
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertFileHasLength;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.cleanup;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test concat -if supported
@@ -60,25 +61,15 @@ public abstract class AbstractContractConcatTest extends AbstractFSContractTestB
   @Test
   public void testConcatEmptyFiles() throws Throwable {
     touch(getFileSystem(), target);
-    try {
-      getFileSystem().concat(target, new Path[0]);
-      fail("expected a failure");
-    } catch (Exception e) {
-      //expected
-      handleExpectedException(e);
-    }
+    handleExpectedException(intercept(Exception.class,
+        () -> getFileSystem().concat(target, new Path[0])));
   }
 
   @Test
   public void testConcatMissingTarget() throws Throwable {
-    try {
-      getFileSystem().concat(target,
-                             new Path[] { zeroByteFile});
-      fail("expected a failure");
-    } catch (Exception e) {
-      //expected
-      handleExpectedException(e);
-    }
+    handleExpectedException(
+        intercept(Exception.class,
+            () -> getFileSystem().concat(target, new Path[]{zeroByteFile})));
   }
 
   @Test
@@ -98,15 +89,8 @@ public abstract class AbstractContractConcatTest extends AbstractFSContractTestB
   public void testConcatOnSelf() throws Throwable {
     byte[] block = dataset(TEST_FILE_LEN, 0, 255);
     createFile(getFileSystem(), target, false, block);
-    try {
-      getFileSystem().concat(target,
-                             new Path[]{target});
-    } catch (Exception e) {
-      //expected
-      handleExpectedException(e);
-    }
+    handleExpectedException(intercept(Exception.class,
+        () -> getFileSystem().concat(target, new Path[]{target})));
   }
 
-
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java
index 269e35e..cb706ede 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.junit.Test;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test getFileStatus and related listing operations.
@@ -275,35 +276,22 @@ public abstract class AbstractContractGetFileStatusTest extends
   @Test
   public void testLocatedStatusNoDir() throws Throwable {
     describe("test the LocatedStatus call on a path which is not present");
-    try {
-      RemoteIterator<LocatedFileStatus> iterator
-          = getFileSystem().listLocatedStatus(path("missing"));
-      fail("Expected an exception, got an iterator: " + iterator);
-    } catch (FileNotFoundException expected) {
-      // expected
-    }
+    intercept(FileNotFoundException.class,
+        () -> getFileSystem().listLocatedStatus(path("missing")));
   }
 
   @Test
   public void testListStatusNoDir() throws Throwable {
     describe("test the listStatus(path) call on a path which is not present");
-    try {
-      getFileSystem().listStatus(path("missing"));
-      fail("Expected an exception");
-    } catch (FileNotFoundException expected) {
-      // expected
-    }
+    intercept(FileNotFoundException.class,
+        () -> getFileSystem().listStatus(path("missing")));
   }
 
   @Test
   public void testListStatusFilteredNoDir() throws Throwable {
     describe("test the listStatus(path, filter) call on a missing path");
-    try {
-      getFileSystem().listStatus(path("missing"), ALL_PATHS);
-      fail("Expected an exception");
-    } catch (FileNotFoundException expected) {
-      // expected
-    }
+    intercept(FileNotFoundException.class,
+        () -> getFileSystem().listStatus(path("missing"), ALL_PATHS));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java
index c5a546d..de44bc2 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java
@@ -26,6 +26,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 
@@ -175,4 +176,11 @@ public abstract class AbstractContractMkdirTest extends AbstractFSContractTestBa
     }
   }
 
+  @Test
+  public void testCreateDirWithExistingDir() throws Exception {
+    Path path = path("testCreateDirWithExistingDir");
+    final FileSystem fs = getFileSystem();
+    assertMkdirs(fs, path);
+    assertMkdirs(fs, path);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java
index d3dafe9..f09496a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java
@@ -148,7 +148,6 @@ public abstract class AbstractFSContract extends Configured {
    * @param feature feature to query
    * @param defval default value
    * @return true if the feature is supported
-   * @throws IOException IO problems
    */
   public boolean isSupported(String feature, boolean defval) {
     return getConf().getBoolean(getConfKey(feature), defval);
@@ -160,7 +159,6 @@ public abstract class AbstractFSContract extends Configured {
    * @param feature feature to query
    * @param defval default value
    * @return true if the feature is supported
-   * @throws IOException IO problems
    */
   public int getLimit(String feature, int defval) {
     return getConf().getInt(getConfKey(feature), defval);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
index 38a6fb1..ba12048 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
@@ -187,8 +187,11 @@ public class ContractTestUtils extends Assert {
           (short) 1,
           buffersize);
     }
-    out.write(src, 0, len);
-    out.close();
+    try {
+      out.write(src, 0, len);
+    } finally {
+      out.close();
+    }
     assertFileHasLength(fs, path, len);
   }
 
@@ -1022,6 +1025,18 @@ public class ContractTestUtils extends Assert {
   }
 
   /**
+   * Execute {@link FileSystem#mkdirs(Path)}; expect {@code true} back.
+   * (Note: does not work for localFS if the directory already exists)
+   * Does not perform any validation of the created directory.
+   * @param fs filesystem
+   * @param dir directory to create
+   * @throws IOException IO Problem
+   */
+  public static void assertMkdirs(FileSystem fs, Path dir) throws IOException {
+    assertTrue("mkdirs(" + dir + ") returned false", fs.mkdirs(dir));
+  }
+
+  /**
    * Test for the host being an OSX machine.
    * @return true if the JVM thinks that is running on OSX
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java
index 707e264..32df942 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DelegateToFileSystem;
@@ -32,7 +31,6 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
  * Azure Blob File System implementation of AbstractFileSystem.
  * This impl delegates to the old FileSystem
  */
-@InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class Abfs extends DelegateToFileSystem {
 
@@ -45,4 +43,4 @@ public class Abfs extends DelegateToFileSystem {
   public int getUriDefaultPort() {
     return -1;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java
index 19c0f7a..c33265c 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DelegateToFileSystem;
@@ -32,7 +31,6 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
  * Azure Blob File System implementation of AbstractFileSystem.
  * This impl delegates to the old FileSystem
  */
-@InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class Abfss extends DelegateToFileSystem {
 
@@ -45,4 +43,4 @@ public class Abfss extends DelegateToFileSystem {
   public int getUriDefaultPort() {
     return -1;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
index cf5acbb..9f58f6b 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
@@ -36,14 +36,10 @@ import java.util.concurrent.Future;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.fs.PathIOException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 
 import org.apache.commons.lang.ArrayUtils;
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
@@ -54,13 +50,15 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
@@ -69,8 +67,7 @@ import org.apache.hadoop.util.Progressable;
  * A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
  * href="http://store.azure.com/">Windows Azure</a>
  */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
+@InterfaceStability.Evolving
 public class AzureBlobFileSystem extends FileSystem {
   public static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystem.class);
   private URI uri;
@@ -88,8 +85,7 @@ public class AzureBlobFileSystem extends FileSystem {
     super.initialize(uri, configuration);
     setConf(configuration);
 
-    this.LOG.debug(
-        "Initializing AzureBlobFileSystem for {}", uri);
+    LOG.debug("Initializing AzureBlobFileSystem for {}", uri);
 
     this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
     this.userGroupInformation = UserGroupInformation.getCurrentUser();
@@ -97,16 +93,24 @@ public class AzureBlobFileSystem extends FileSystem {
     this.primaryUserGroup = userGroupInformation.getPrimaryGroupName();
     this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation);
 
-    this.LOG.debug(
-        "Initializing NativeAzureFileSystem for {}", uri);
+    LOG.debug("Initializing NativeAzureFileSystem for {}", uri);
 
     this.setWorkingDirectory(this.getHomeDirectory());
 
     if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) {
       this.createFileSystem();
     }
+  }
 
-    this.mkdirs(this.workingDir);
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "AzureBlobFileSystem{");
+    sb.append("uri=").append(uri);
+    sb.append(", user='").append(user).append('\'');
+    sb.append(", primaryUserGroup='").append(primaryUserGroup).append('\'');
+    sb.append('}');
+    return sb.toString();
   }
 
   public boolean isSecure() {
@@ -120,8 +124,7 @@ public class AzureBlobFileSystem extends FileSystem {
 
   @Override
   public FSDataInputStream open(final Path path, final int bufferSize) throws IOException {
-    this.LOG.debug(
-        "AzureBlobFileSystem.open path: {} bufferSize: {}", path.toString(), bufferSize);
+    LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize);
 
     try {
       InputStream inputStream = abfsStore.openFileForRead(makeQualified(path), statistics);
@@ -135,9 +138,8 @@ public class AzureBlobFileSystem extends FileSystem {
   @Override
   public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize,
       final short replication, final long blockSize, final Progressable progress) throws IOException {
-    this.LOG.debug(
-        "AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}",
-        f.toString(),
+    LOG.debug("AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}",
+        f,
         permission,
         overwrite,
         blockSize);
@@ -196,7 +198,7 @@ public class AzureBlobFileSystem extends FileSystem {
 
   @Override
   public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException {
-    this.LOG.debug(
+    LOG.debug(
         "AzureBlobFileSystem.append path: {} bufferSize: {}",
         f.toString(),
         bufferSize);
@@ -211,7 +213,7 @@ public class AzureBlobFileSystem extends FileSystem {
   }
 
   public boolean rename(final Path src, final Path dst) throws IOException {
-    this.LOG.debug(
+    LOG.debug(
         "AzureBlobFileSystem.rename src: {} dst: {}", src.toString(), dst.toString());
 
     Path parentFolder = src.getParent();
@@ -250,7 +252,7 @@ public class AzureBlobFileSystem extends FileSystem {
 
   @Override
   public boolean delete(final Path f, final boolean recursive) throws IOException {
-    this.LOG.debug(
+    LOG.debug(
         "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive);
 
     if (f.isRoot()) {
@@ -273,7 +275,7 @@ public class AzureBlobFileSystem extends FileSystem {
 
   @Override
   public FileStatus[] listStatus(final Path f) throws IOException {
-    this.LOG.debug(
+    LOG.debug(
         "AzureBlobFileSystem.listStatus path: {}", f.toString());
 
     try {
@@ -287,8 +289,8 @@ public class AzureBlobFileSystem extends FileSystem {
 
   @Override
   public boolean mkdirs(final Path f, final FsPermission permission) throws IOException {
-    this.LOG.debug(
-        "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f.toString(), permission);
+    LOG.debug(
+        "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f, permission);
 
     final Path parentFolder = f.getParent();
     if (parentFolder == null) {
@@ -312,13 +314,13 @@ public class AzureBlobFileSystem extends FileSystem {
     }
 
     super.close();
-    this.LOG.debug("AzureBlobFileSystem.close");
+    LOG.debug("AzureBlobFileSystem.close");
     this.isClosed = true;
   }
 
   @Override
   public FileStatus getFileStatus(final Path f) throws IOException {
-    this.LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f.toString());
+    LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f);
 
     try {
       return abfsStore.getFileStatus(makeQualified(f));
@@ -350,7 +352,8 @@ public class AzureBlobFileSystem extends FileSystem {
   @Override
   public Path getHomeDirectory() {
     return makeQualified(new Path(
-            FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX + "/" + this.userGroupInformation.getShortUserName()));
+            FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX
+                + "/" + this.userGroupInformation.getShortUserName()));
   }
 
   /**
@@ -360,7 +363,7 @@ public class AzureBlobFileSystem extends FileSystem {
    */
   @Override
   public BlockLocation[] getFileBlockLocations(FileStatus file,
-      long start, long len) throws IOException {
+      long start, long len) {
     if (file == null) {
       return null;
     }
@@ -403,7 +406,7 @@ public class AzureBlobFileSystem extends FileSystem {
   }
 
   private boolean deleteRoot() throws IOException {
-    this.LOG.debug("Deleting root content");
+    LOG.debug("Deleting root content");
 
     final ExecutorService executorService = Executors.newFixedThreadPool(10);
 
@@ -441,15 +444,14 @@ public class AzureBlobFileSystem extends FileSystem {
   private FileStatus tryGetFileStatus(final Path f) {
     try {
       return getFileStatus(f);
-    }
-    catch (IOException ex) {
-      this.LOG.debug("File not found {}", f.toString());
+    } catch (IOException ex) {
+      LOG.debug("File not found {}", f);
       return null;
     }
   }
 
   private void createFileSystem() throws IOException {
-    this.LOG.debug(
+    LOG.debug(
         "AzureBlobFileSystem.createFileSystem uri: {}", uri);
     try {
       this.abfsStore.createFilesystem();
@@ -493,7 +495,8 @@ public class AzureBlobFileSystem extends FileSystem {
       return false;
     }
 
-    if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME) || scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) {
+    if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME)
+        || scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) {
       return true;
     }
 
@@ -501,34 +504,45 @@ public class AzureBlobFileSystem extends FileSystem {
   }
 
   @VisibleForTesting
-  <T> FileSystemOperation execute(
+  <T> FileSystemOperation<T> execute(
       final String scopeDescription,
       final Callable<T> callableFileOperation) throws IOException {
     return execute(scopeDescription, callableFileOperation, null);
   }
 
   @VisibleForTesting
-  <T> FileSystemOperation execute(
+  <T> FileSystemOperation<T> execute(
       final String scopeDescription,
       final Callable<T> callableFileOperation,
       T defaultResultValue) throws IOException {
 
     try {
       final T executionResult = callableFileOperation.call();
-      return new FileSystemOperation(executionResult, null);
+      return new FileSystemOperation<>(executionResult, null);
     } catch (AbfsRestOperationException abfsRestOperationException) {
-      return new FileSystemOperation(defaultResultValue, abfsRestOperationException);
+      return new FileSystemOperation<>(defaultResultValue, abfsRestOperationException);
     } catch (AzureBlobFileSystemException azureBlobFileSystemException) {
       throw new IOException(azureBlobFileSystemException);
     } catch (Exception exception) {
       if (exception instanceof ExecutionException) {
         exception = (Exception) getRootCause(exception);
       }
-      final FileSystemOperationUnhandledException fileSystemOperationUnhandledException = new FileSystemOperationUnhandledException(exception);
+      final FileSystemOperationUnhandledException fileSystemOperationUnhandledException
+          = new FileSystemOperationUnhandledException(exception);
       throw new IOException(fileSystemOperationUnhandledException);
     }
   }
 
+  /**
+   * Given a path and exception, choose which IOException subclass
+   * to create.
+   * Will return if and only iff the error code is in the list of allowed
+   * error codes.
+   * @param path path of operation triggering exception; may be null
+   * @param exception the exception caught
+   * @param allowedErrorCodesList varargs list of error codes.
+   * @throws IOException if the exception error code is not on the allowed list.
+   */
   private void checkException(final Path path,
                               final AzureBlobFileSystemException exception,
                               final AzureServiceErrorCode... allowedErrorCodesList) throws IOException {
@@ -542,9 +556,11 @@ public class AzureBlobFileSystem extends FileSystem {
 
       //AbfsRestOperationException.getMessage() contains full error info including path/uri.
       if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) {
-        throw new FileNotFoundException(ere.getMessage());
+        throw (IOException)new FileNotFoundException(ere.getMessage())
+            .initCause(exception);
       } else if (statusCode == HttpURLConnection.HTTP_CONFLICT) {
-        throw new FileAlreadyExistsException(ere.getMessage());
+        throw (IOException)new FileAlreadyExistsException(ere.getMessage())
+            .initCause(exception);
       } else {
         throw ere;
       }
@@ -601,4 +617,4 @@ public class AzureBlobFileSystem extends FileSystem {
   AzureBlobFileSystemStore getAbfsStore() {
     return this.abfsStore;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
index 134277f..8ac31ce 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
@@ -103,7 +103,7 @@ public class AzureBlobFileSystemStore {
   private final Set<String> azureAtomicRenameDirSet;
 
 
-  public AzureBlobFileSystemStore(URI uri, boolean isSeure, Configuration configuration, UserGroupInformation userGroupInformation)
+  public AzureBlobFileSystemStore(URI uri, boolean isSecure, Configuration configuration, UserGroupInformation userGroupInformation)
           throws AzureBlobFileSystemException {
     this.uri = uri;
     try {
@@ -113,9 +113,10 @@ public class AzureBlobFileSystemStore {
     }
 
     this.userGroupInformation = userGroupInformation;
-    this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
+    this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(
+        abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
 
-    initializeClient(uri, isSeure);
+    initializeClient(uri, isSecure);
   }
 
   @VisibleForTesting
@@ -134,8 +135,7 @@ public class AzureBlobFileSystemStore {
   }
 
   public Hashtable<String, String> getFilesystemProperties() throws AzureBlobFileSystemException {
-    this.LOG.debug(
-            "getFilesystemProperties for filesystem: {}",
+    LOG.debug("getFilesystemProperties for filesystem: {}",
             client.getFileSystem());
 
     final Hashtable<String, String> parsedXmsProperties;
@@ -148,13 +148,13 @@ public class AzureBlobFileSystemStore {
     return parsedXmsProperties;
   }
 
-  public void setFilesystemProperties(final Hashtable<String, String> properties) throws AzureBlobFileSystemException {
-    if (properties == null || properties.size() == 0) {
+  public void setFilesystemProperties(final Hashtable<String, String> properties)
+      throws AzureBlobFileSystemException {
+    if (properties == null || properties.isEmpty()) {
       return;
     }
 
-    this.LOG.debug(
-            "setFilesystemProperties for filesystem: {} with properties: {}",
+    LOG.debug("setFilesystemProperties for filesystem: {} with properties: {}",
             client.getFileSystem(),
             properties);
 
@@ -169,10 +169,9 @@ public class AzureBlobFileSystemStore {
   }
 
   public Hashtable<String, String> getPathProperties(final Path path) throws AzureBlobFileSystemException {
-    this.LOG.debug(
-            "getPathProperties for filesystem: {} path: {}",
+    LOG.debug("getPathProperties for filesystem: {} path: {}",
             client.getFileSystem(),
-            path.toString());
+           path);
 
     final Hashtable<String, String> parsedXmsProperties;
     final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
@@ -185,10 +184,9 @@ public class AzureBlobFileSystemStore {
   }
 
   public void setPathProperties(final Path path, final Hashtable<String, String> properties) throws AzureBlobFileSystemException {
-    this.LOG.debug(
-            "setFilesystemProperties for filesystem: {} path: {} with properties: {}",
+    LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}",
             client.getFileSystem(),
-            path.toString(),
+            path,
             properties);
 
     final String commaSeparatedProperties;
@@ -201,26 +199,23 @@ public class AzureBlobFileSystemStore {
   }
 
   public void createFilesystem() throws AzureBlobFileSystemException {
-    this.LOG.debug(
-            "createFilesystem for filesystem: {}",
+    LOG.debug("createFilesystem for filesystem: {}",
             client.getFileSystem());
 
     client.createFilesystem();
   }
 
   public void deleteFilesystem() throws AzureBlobFileSystemException {
-    this.LOG.debug(
-            "deleteFilesystem for filesystem: {}",
+    LOG.debug("deleteFilesystem for filesystem: {}",
             client.getFileSystem());
 
     client.deleteFilesystem();
   }
 
   public OutputStream createFile(final Path path, final boolean overwrite) throws AzureBlobFileSystemException {
-    this.LOG.debug(
-            "createFile filesystem: {} path: {} overwrite: {}",
+    LOG.debug("createFile filesystem: {} path: {} overwrite: {}",
             client.getFileSystem(),
-            path.toString(),
+            path,
             overwrite);
 
     client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite);
@@ -232,23 +227,19 @@ public class AzureBlobFileSystemStore {
     return outputStream;
   }
 
-  public Void createDirectory(final Path path) throws AzureBlobFileSystemException {
-    this.LOG.debug(
-            "createDirectory filesystem: {} path: {} overwrite: {}",
+  public void createDirectory(final Path path) throws AzureBlobFileSystemException {
+    LOG.debug("createDirectory filesystem: {} path: {}",
             client.getFileSystem(),
-            path.toString());
+            path);
 
     client.createPath("/" + getRelativePath(path), false, true);
-
-    return null;
   }
 
   public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException {
 
-    this.LOG.debug(
-            "openFileForRead filesystem: {} path: {}",
+    LOG.debug("openFileForRead filesystem: {} path: {}",
             client.getFileSystem(),
-            path.toString());
+            path);
 
     final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
 
@@ -266,16 +257,16 @@ public class AzureBlobFileSystemStore {
 
     // Add statistics for InputStream
     return new FSDataInputStream(
-            new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
+            new AbfsInputStream(client, statistics,
+                AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
                     abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag));
   }
 
   public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws
           AzureBlobFileSystemException {
-    this.LOG.debug(
-            "openFileForWrite filesystem: {} path: {} overwrite: {}",
+    LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}",
             client.getFileSystem(),
-            path.toString(),
+            path,
             overwrite);
 
     final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
@@ -304,23 +295,21 @@ public class AzureBlobFileSystemStore {
           AzureBlobFileSystemException {
 
     if (isAtomicRenameKey(source.getName())) {
-      this.LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename,"
+      LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename,"
               +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account.");
     }
 
-    this.LOG.debug(
-            "renameAsync filesystem: {} source: {} destination: {}",
+    LOG.debug("renameAsync filesystem: {} source: {} destination: {}",
             client.getFileSystem(),
-            source.toString(),
-            destination.toString());
+            source,
+            destination);
 
     String continuation = null;
     long deadline = now() + RENAME_TIMEOUT_MILISECONDS;
 
     do {
       if (now() > deadline) {
-        LOG.debug(
-                "Rename {} to {} timed out.",
+        LOG.debug("Rename {} to {} timed out.",
                 source,
                 destination);
 
@@ -334,13 +323,12 @@ public class AzureBlobFileSystemStore {
     } while (continuation != null && !continuation.isEmpty());
   }
 
-  public void delete(final Path path, final boolean recursive) throws
-          AzureBlobFileSystemException {
+  public void delete(final Path path, final boolean recursive)
+      throws AzureBlobFileSystemException {
 
-    this.LOG.debug(
-            "delete filesystem: {} path: {} recursive: {}",
+    LOG.debug("delete filesystem: {} path: {} recursive: {}",
             client.getFileSystem(),
-            path.toString(),
+            path,
             String.valueOf(recursive));
 
     String continuation = null;
@@ -348,13 +336,13 @@ public class AzureBlobFileSystemStore {
 
     do {
       if (now() > deadline) {
-        this.LOG.debug(
-                "Delete directory {} timed out.", path);
+        LOG.debug("Delete directory {} timed out.", path);
 
         throw new TimeoutException("Delete directory timed out.");
       }
 
-      AbfsRestOperation op = client.deletePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation);
+      AbfsRestOperation op = client.deletePath(
+          AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation);
       continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
 
     } while (continuation != null && !continuation.isEmpty());
@@ -362,10 +350,9 @@ public class AzureBlobFileSystemStore {
 
   public FileStatus getFileStatus(final Path path) throws IOException {
 
-    this.LOG.debug(
-            "getFileStatus filesystem: {} path: {}",
+    LOG.debug("getFileStatus filesystem: {} path: {}",
             client.getFileSystem(),
-            path.toString());
+           path);
 
     if (path.isRoot()) {
       AbfsRestOperation op = client.getFilesystemProperties();
@@ -405,10 +392,9 @@ public class AzureBlobFileSystemStore {
   }
 
   public FileStatus[] listStatus(final Path path) throws IOException {
-    this.LOG.debug(
-            "listStatus filesystem: {} path: {}",
+    LOG.debug("listStatus filesystem: {} path: {}",
             client.getFileSystem(),
-            path.toString());
+           path);
 
     String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path);
     String continuation = null;
@@ -480,10 +466,12 @@ public class AzureBlobFileSystemStore {
 
     final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
 
-    if (authorityParts.length < 2 || "".equals(authorityParts[0])) {
+    if (authorityParts.length < 2 || authorityParts[0] != null
+        && authorityParts[0].isEmpty()) {
       final String errMsg = String
-              .format("URI '%s' has a malformed authority, expected container name. "
-                              + "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
+              .format("'%s' has a malformed authority, expected container name. "
+                      + "Authority takes the form "
+                      + FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
                       uri.toString());
       throw new InvalidUriException(errMsg);
     }
@@ -499,11 +487,16 @@ public class AzureBlobFileSystemStore {
     try {
       baseUrl = new URL(url);
     } catch (MalformedURLException e) {
-      throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString()));
+      throw new InvalidUriException(uri.toString());
     }
 
+    int dotIndex = accountName.indexOf(AbfsHttpConstants.DOT);
+    if (dotIndex <= 0) {
+      throw new InvalidUriException(
+          uri.toString() + " - account name is not fully qualified.");
+    }
     SharedKeyCredentials creds =
-            new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)),
+            new SharedKeyCredentials(accountName.substring(0, dotIndex),
                     this.abfsConfiguration.getStorageAccountKey(accountName));
 
     this.client =  new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy());
@@ -513,7 +506,7 @@ public class AzureBlobFileSystemStore {
     Preconditions.checkNotNull(path, "path");
     final String relativePath = path.toUri().getPath();
 
-    if (relativePath.length() == 0) {
+    if (relativePath.isEmpty()) {
       return relativePath;
     }
 
@@ -537,7 +530,8 @@ public class AzureBlobFileSystemStore {
   }
 
   private boolean parseIsDirectory(final String resourceType) {
-    return resourceType == null ? false : resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
+    return resourceType != null
+        && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
   }
 
   private DateTime parseLastModifiedTime(final String lastModifiedTime) {
@@ -628,7 +622,7 @@ public class AzureBlobFileSystemStore {
           }
         }
       } catch (URISyntaxException e) {
-        this.LOG.info("URI syntax error creating URI for {}", dir);
+        LOG.info("URI syntax error creating URI for {}", dir);
       }
     }
 
@@ -658,20 +652,21 @@ public class AzureBlobFileSystemStore {
      */
     @Override
     public boolean equals(Object obj) {
-      if (obj == this) {
-        return true;
+      if (!(obj instanceof FileStatus)) {
+        return false;
       }
 
-      if (obj == null) {
+      FileStatus other = (FileStatus) obj;
+
+      if (!other.equals(this)) {// compare the path
         return false;
       }
 
-      if (this.getClass() == obj.getClass()) {
-        VersionedFileStatus other = (VersionedFileStatus) obj;
-        return this.getPath().equals(other.getPath()) && this.version.equals(other.version);
+      if (other instanceof VersionedFileStatus) {
+        return this.version.equals(((VersionedFileStatus)other).version);
       }
 
-      return false;
+      return true;
     }
 
     /**
@@ -695,6 +690,16 @@ public class AzureBlobFileSystemStore {
     public String getVersion() {
       return this.version;
     }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "VersionedFileStatus{");
+      sb.append(super.toString());
+      sb.append("; version='").append(version).append('\'');
+      sb.append('}');
+      return sb.toString();
+    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java
index a6ad829..15fe542 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.fs.azurebfs;
 
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 
@@ -26,7 +25,6 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
  * A secure {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
  * href="http://store.azure.com/">Windows Azure</a>
  */
-@InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class SecureAzureBlobFileSystem extends AzureBlobFileSystem {
   @Override
@@ -38,4 +36,4 @@ public class SecureAzureBlobFileSystem extends AzureBlobFileSystem {
   public String getScheme() {
     return FileSystemUriSchemes.ABFS_SECURE_SCHEME;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
index 2ec4db0..f80bc60 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * Responsible to keep all constant keys used in abfs rest client here
+ * Responsible to keep all constant keys used in abfs rest client here.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
index 9b7f9bc..4603b5f 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * Responsible to keep all abfs http headers here
+ * Responsible to keep all abfs http headers here.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
index a9f7d33..f58d33a 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * Responsible to keep all Http Query params here
+ * Responsible to keep all Http Query params here.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java
index 462ebbc..82c571a 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java
@@ -25,12 +25,12 @@ import java.lang.annotation.Target;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * Definitions of Annotations for all types of the validators
+ * Definitions of Annotations for all types of the validators.
  */
 @InterfaceStability.Evolving
 public class ConfigurationValidationAnnotations {
   /**
-   * Describes the requirements when validating the annotated int field
+   * Describes the requirements when validating the annotated int field.
    */
   @Target({ ElementType.FIELD })
   @Retention(RetentionPolicy.RUNTIME)
@@ -47,7 +47,7 @@ public class ConfigurationValidationAnnotations {
   }
 
   /**
-   * Describes the requirements when validating the annotated long field
+   * Describes the requirements when validating the annotated long field.
    */
   @Target({ ElementType.FIELD })
   @Retention(RetentionPolicy.RUNTIME)
@@ -64,7 +64,7 @@ public class ConfigurationValidationAnnotations {
   }
 
   /**
-   * Describes the requirements when validating the annotated String field
+   * Describes the requirements when validating the annotated String field.
    */
   @Target({ ElementType.FIELD })
   @Retention(RetentionPolicy.RUNTIME)
@@ -77,7 +77,7 @@ public class ConfigurationValidationAnnotations {
   }
 
   /**
-   * Describes the requirements when validating the annotated String field
+   * Describes the requirements when validating the annotated String field.
    */
   @Target({ ElementType.FIELD })
   @Retention(RetentionPolicy.RUNTIME)
@@ -90,7 +90,7 @@ public class ConfigurationValidationAnnotations {
   }
 
   /**
-   * Describes the requirements when validating the annotated boolean field
+   * Describes the requirements when validating the annotated boolean field.
    */
   @Target({ ElementType.FIELD })
   @Retention(RetentionPolicy.RUNTIME)
@@ -101,4 +101,4 @@ public class ConfigurationValidationAnnotations {
 
     boolean ThrowIfInvalid() default false;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java
index 796f785..d61229e 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java
@@ -18,19 +18,17 @@
 
 package org.apache.hadoop.fs.azurebfs.contracts.diagnostics;
 
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 
 /**
  * ConfigurationValidator to validate the value of a configuration key
- * @param <T> the type of the validator and the validated value
+ * @param <T> the type of the validator and the validated value.
  */
-@InterfaceAudience.Public
 @InterfaceStability.Evolving
 public interface ConfigurationValidator<T> {
   /**
-   * Validates the configValue
+   * Validates the configValue.
    * @return validated value of type T
    */
   T validate(String configValue) throws InvalidConfigurationValueException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java
index f31c680..9b1bead 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java
@@ -40,7 +40,7 @@ public abstract class AzureBlobFileSystemException extends IOException {
   @Override
   public String toString() {
     if (this.getMessage() == null && this.getCause() == null) {
-      return "";
+      return "AzureBlobFileSystemException";
     }
 
     if (this.getCause() == null) {
@@ -53,4 +53,4 @@ public abstract class AzureBlobFileSystemException extends IOException {
 
     return this.getMessage() + this.getCause().toString();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java
index a84495a..4fa0150 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java
@@ -28,6 +28,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Evolving
 public final class InvalidUriException extends AzureBlobFileSystemException {
   public InvalidUriException(String url) {
-    super(String.format("%s is invalid.", url));
+    super(String.format("Invalid URI %s", url));
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java
index 69288c5..6bb997b 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationVa
 import org.apache.commons.codec.binary.Base64;
 
 /**
-* String Base64 configuration value Validator
+* String Base64 configuration value Validator.
 */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java
index c9927ff..b16abdd 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 
 /**
- * Boolean configuration value validator
+ * Boolean configuration value validator.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
@@ -47,4 +47,4 @@ public class BooleanConfigurationBasicValidator extends ConfigurationBasicValida
 
     throw new InvalidConfigurationValueException(getConfigKey());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java
index 7da809c..8555a29 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java
@@ -38,7 +38,7 @@ abstract class ConfigurationBasicValidator<T> implements ConfigurationValidator
 
   /**
    * This method handles the base case where the configValue is null, based on the throwIfInvalid it either throws or returns the defaultVal,
-   * otherwise it returns null indicating that the configValue needs to be validated further
+   * otherwise it returns null indicating that the configValue needs to be validated further.
    * @param configValue the configuration value set by the user
    * @return the defaultVal in case the configValue is null and not required to be set, null in case the configValue not null
    * @throws InvalidConfigurationValueException in case the configValue is null and required to be set

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java
index ec38cd8..26c7d2f 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidato
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 
 /**
- * Integer configuration value Validator
+ * Integer configuration value Validator.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java
index 559dbc0..32ac14c 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java
@@ -17,15 +17,13 @@
  */
 package org.apache.hadoop.fs.azurebfs.diagnostics;
 
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 
 /**
- * Long configuration value Validator
+ * Long configuration value Validator.
  */
-@InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class LongConfigurationBasicValidator extends ConfigurationBasicValidator<Long> implements ConfigurationValidator {
   private final long min;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java
index d6f9c59..0d344d1 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java
@@ -18,15 +18,13 @@
 
 package org.apache.hadoop.fs.azurebfs.diagnostics;
 
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 
 /**
- * String configuration value Validator
+ * String configuration value Validator.
  */
-@InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class StringConfigurationBasicValidator extends ConfigurationBasicValidator<String> implements ConfigurationValidator{
 


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


[34/50] [abbrv] hadoop git commit: HADOOP-15446. ABFS: tune imports & javadocs; stabilise tests. Contributed by Steve Loughran and Da Zhou.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index 7010e74..a89c044 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -17,59 +17,62 @@
  */
 package org.apache.hadoop.fs.azurebfs;
 
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
 
-import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 
-import static junit.framework.TestCase.assertEquals;
-import static junit.framework.TestCase.assertFalse;
-import static junit.framework.TestCase.assertTrue;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
 
 /**
  * Test compatibility between ABFS client and WASB client.
  */
-public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
+public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
   private static final String WASB_TEST_CONTEXT = "wasb test file";
   private static final String ABFS_TEST_CONTEXT = "abfs test file";
   private static final String TEST_CONTEXT = "THIS IS FOR TEST";
 
-  public ITestWasbAbfsCompatibility() throws Exception {
-    super();
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestWasbAbfsCompatibility.class);
 
-    Assume.assumeFalse(this.isEmulator());
+  public ITestWasbAbfsCompatibility() throws Exception {
+    Assume.assumeFalse("Emulator is not supported", isEmulator());
   }
 
   @Test
   public void testListFileStatus() throws Exception {
     // crate file using abfs
-    AzureBlobFileSystem fs = this.getFileSystem();
-    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+    AzureBlobFileSystem fs = getFileSystem();
+    NativeAzureFileSystem wasb = getWasbFileSystem();
 
     Path path1 = new Path("/testfiles/~12/!008/3/abFsTestfile");
-    FSDataOutputStream abfsStream = fs.create(path1, true);
-    abfsStream.write(ABFS_TEST_CONTEXT.getBytes());
-    abfsStream.flush();
-    abfsStream.hsync();
-    abfsStream.close();
+    try(FSDataOutputStream abfsStream = fs.create(path1, true)) {
+      abfsStream.write(ABFS_TEST_CONTEXT.getBytes());
+      abfsStream.flush();
+      abfsStream.hsync();
+    }
 
     // create file using wasb
     Path path2 = new Path("/testfiles/~12/!008/3/nativeFsTestfile");
-    System.out.println(wasb.getUri());
-    FSDataOutputStream nativeFsStream = wasb.create(path2, true);
-    nativeFsStream.write(WASB_TEST_CONTEXT.getBytes());
-    nativeFsStream.flush();
-    nativeFsStream.hsync();
-    nativeFsStream.close();
+    LOG.info("{}", wasb.getUri());
+    try(FSDataOutputStream nativeFsStream = wasb.create(path2, true)) {
+      nativeFsStream.write(WASB_TEST_CONTEXT.getBytes());
+      nativeFsStream.flush();
+      nativeFsStream.hsync();
+    }
     // list file using abfs and wasb
     FileStatus[] abfsFileStatus = fs.listStatus(new Path("/testfiles/~12/!008/3/"));
     FileStatus[] nativeFsFileStatus = wasb.listStatus(new Path("/testfiles/~12/!008/3/"));
@@ -83,52 +86,34 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
     boolean[] createFileWithAbfs = new boolean[]{false, true, false, true};
     boolean[] readFileWithAbfs = new boolean[]{false, true, true, false};
 
-    AzureBlobFileSystem abfs = this.getFileSystem();
-    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+    AzureBlobFileSystem abfs = getFileSystem();
+    NativeAzureFileSystem wasb = getWasbFileSystem();
 
-    FileSystem fs;
-    BufferedReader br = null;
     for (int i = 0; i< 4; i++) {
-      try {
-        Path path = new Path("/testfiles/~12/!008/testfile" + i);
-        if (createFileWithAbfs[i]) {
-          fs = abfs;
-        } else {
-          fs = wasb;
-        }
-
-        // Write
-        FSDataOutputStream nativeFsStream = fs.create(path, true);
+      Path path = new Path("/testfiles/~12/!008/testfile" + i);
+      final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb;
+
+      // Write
+      try(FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
         nativeFsStream.write(TEST_CONTEXT.getBytes());
         nativeFsStream.flush();
         nativeFsStream.hsync();
-        nativeFsStream.close();
-
-        // Check file status
-        assertEquals(true, fs.exists(path));
-        assertEquals(false, fs.getFileStatus(path).isDirectory());
-
-        // Read
-        if (readFileWithAbfs[i]) {
-          fs = abfs;
-        } else {
-          fs = wasb;
-        }
-        FSDataInputStream inputStream = fs.open(path);
-        br = new BufferedReader(new InputStreamReader(fs.open(path)));
+      }
+
+      // Check file status
+      ContractTestUtils.assertIsFile(createFs, path);
+
+      // Read
+      final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb;
+
+      try(BufferedReader br =new BufferedReader(new InputStreamReader(readFs.open(path)))) {
         String line = br.readLine();
-        assertEquals(TEST_CONTEXT, line);
-
-        // Remove file
-        fs.delete(path, true);
-        assertFalse(fs.exists(path));
-      } catch (Exception e) {
-        e.printStackTrace();
-      } finally {
-        if (br != null) {
-          br.close();
-        }
+        assertEquals("Wrong text from " + readFs,
+            TEST_CONTEXT, line);
       }
+
+      // Remove file
+      assertDeleted(readFs, path, true);
     }
   }
 
@@ -137,32 +122,22 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
     boolean[] createDirWithAbfs = new boolean[]{false, true, false, true};
     boolean[] readDirWithAbfs = new boolean[]{false, true, true, false};
 
-    AzureBlobFileSystem abfs = this.getFileSystem();
-    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+    AzureBlobFileSystem abfs = getFileSystem();
+    NativeAzureFileSystem wasb = getWasbFileSystem();
 
-    FileSystem fs;
     for (int i = 0; i < 4; i++) {
       Path path = new Path("/testDir/t" + i);
       //create
-      if (createDirWithAbfs[i]) {
-        fs = abfs;
-      } else {
-        fs = wasb;
-      }
-      assertTrue(fs.mkdirs(path));
+      final FileSystem createFs = createDirWithAbfs[i] ? abfs : wasb;
+      assertTrue(createFs.mkdirs(path));
       //check
-      assertTrue(fs.exists(path));
+      assertPathExists(createFs, "Created dir not found with " + createFs, path);
       //read
-      if (readDirWithAbfs[i]) {
-        fs = abfs;
-      } else {
-        fs = wasb;
-      }
-      assertTrue(fs.exists(path));
-      FileStatus dirStatus = fs.getFileStatus(path);
-      assertTrue(dirStatus.isDirectory());
-      fs.delete(path, true);
-      assertFalse(fs.exists(path));
+      final FileSystem readFs = readDirWithAbfs[i] ? abfs : wasb;
+      assertPathExists(readFs, "Created dir not found with " + readFs,
+          path);
+      assertIsDirectory(readFs, path);
+      assertDeleted(readFs, path, true);
     }
   }
 
@@ -171,17 +146,18 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
   public void testUrlConversion(){
     String abfsUrl = "abfs://abcde-1111-1111-1111-1111@xxxx.dfs.xxx.xxx.xxxx.xxxx";
     String wabsUrl = "wasb://abcde-1111-1111-1111-1111@xxxx.blob.xxx.xxx.xxxx.xxxx";
-    Assert.assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl));
-    Assert.assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl));
+    assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl));
+    assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl));
   }
 
   @Test
   public void testSetWorkingDirectory() throws Exception {
     //create folders
-    AzureBlobFileSystem abfs = this.getFileSystem();
-    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+    AzureBlobFileSystem abfs = getFileSystem();
+    NativeAzureFileSystem wasb = getWasbFileSystem();
 
-    assertTrue(abfs.mkdirs(new Path("/d1/d2/d3/d4")));
+    Path d1d4 = new Path("/d1/d2/d3/d4");
+    assertMkdirs(abfs, d1d4);
 
     //set working directory to path1
     Path path1 = new Path("/d1/d2");
@@ -195,8 +171,8 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
     wasb.setWorkingDirectory(path2);
     abfs.setWorkingDirectory(path2);
 
-    Path path3 = new Path("/d1/d2/d3/d4");
+    Path path3 = d1d4;
     assertEquals(path3, wasb.getWorkingDirectory());
     assertEquals(path3, abfs.getWorkingDirectory());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
index 4b44765..fc7312a 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
@@ -18,14 +18,9 @@
 
 package org.apache.hadoop.fs.azurebfs.constants;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
 /**
  * Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file.
  */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class TestConfigurationKeys {
   public static final String FS_AZURE_TEST_ACCOUNT_NAME = "fs.azure.test.account.name";
   public static final String FS_AZURE_TEST_ACCOUNT_KEY_PREFIX = "fs.azure.test.account.key.";
@@ -33,5 +28,9 @@ public final class TestConfigurationKeys {
   public static final String FS_AZURE_TEST_HOST_PORT = "fs.azure.test.host.port";
   public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs";
 
+  public static final String ABFS_TEST_RESOURCE_XML = "azure-bfs-test.xml";
+
+  public static final String ABFS_TEST_CONTAINER_PREFIX = "abfs-testcontainer-";
+
   private TestConfigurationKeys() {}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java
new file mode 100644
index 0000000..ffd5bab
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.contract;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
+
+/**
+ * Bind ABFS contract tests to the Azure test setup/teardown.
+ */
+public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest {
+  private final URI testUri;
+
+  public ABFSContractTestBinding(final boolean secure) throws Exception {
+    this(secure, true);
+  }
+
+  public ABFSContractTestBinding(final boolean secure,
+      final boolean useExistingFileSystem) throws Exception{
+    super(secure);
+    if (useExistingFileSystem) {
+      Configuration configuration = getConfiguration();
+      String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI);
+
+      if (secure) {
+        testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME);
+      }
+      setTestUrl(testUrl);
+
+      this.testUri = new URI(testUrl);
+      //Get container for contract tests
+      configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString());
+      String[] splitAuthority = this.testUri.getAuthority().split("\\@");
+      setFileSystemName(splitAuthority[0]);
+    } else {
+      this.testUri = new URI(super.getTestUrl());
+    }
+  }
+
+  public Configuration getConfiguration() {
+    return super.getConfiguration();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
new file mode 100644
index 0000000..d365e6e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+
+/**
+ * Azure BlobFileSystem Contract. Test paths are created using any maven fork
+ * identifier, if defined. This guarantees paths unique to tests
+ * running in parallel.
+ */
+public class AbfsFileSystemContract extends AbstractBondedFSContract {
+
+  public static final String CONTRACT_XML = "abfs.xml";
+  private final boolean isSecure;
+
+  protected AbfsFileSystemContract(final Configuration conf, boolean secure) {
+    super(conf);
+    //insert the base features
+    addConfResource(CONTRACT_XML);
+    this.isSecure = secure;
+  }
+
+  @Override
+  public String getScheme() {
+    return isSecure ?
+        FileSystemUriSchemes.ABFS_SECURE_SCHEME
+        : FileSystemUriSchemes.ABFS_SCHEME;
+  }
+
+  @Override
+  public Path getTestPath() {
+    return new Path(UriUtils.generateUniqueTestPath());
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "AbfsFileSystemContract{");
+    sb.append("isSecure=").append(isSecure);
+    sb.append(super.toString());
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java
deleted file mode 100644
index 5fc81ce..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java
+++ /dev/null
@@ -1,63 +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.fs.azurebfs.contract;
-
-import java.net.URI;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
-
-/**
- * Dependency inject for ABFS contract tests.
- */
-public class DependencyInjectedContractTest extends DependencyInjectedTest {
-  private final URI testUri;
-
-  public DependencyInjectedContractTest(final boolean secure) throws Exception {
-    this(secure, true);
-  }
-
-  public DependencyInjectedContractTest(final boolean secure, final boolean useExistedFileSystem) throws Exception{
-    super(secure);
-    if (useExistedFileSystem) {
-      Configuration configuration = getConfiguration();
-      String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI);
-
-      if (secure) {
-        testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME);
-      }
-      updateTestUrl(testUrl);
-
-      this.testUri = new URI(testUrl);
-      //Get container for contract tests
-      configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString());
-      String[] splitAuthority = this.testUri.getAuthority().split("\\@");
-      updateFileSystemName(splitAuthority[0]);
-    } else {
-      this.testUri = new URI(super.getTestUrl());
-    }
-  }
-
-  public Configuration getConfiguration() {
-    return super.getConfiguration();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java
deleted file mode 100644
index 7f7a09a..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java
+++ /dev/null
@@ -1,54 +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.fs.azurebfs.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
-import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
-
-/**
- * Azure BlobFileSystem Contract. Test paths are created using any maven fork
- * identifier, if defined. This guarantees paths unique to tests
- * running in parallel.
- */
-public class ITestAbfsFileSystemContract extends AbstractBondedFSContract {
-
-  public static final String CONTRACT_XML = "abfs.xml";
-  private final boolean isSecure;
-
-  protected ITestAbfsFileSystemContract(final Configuration conf, boolean secure) {
-    super(conf);
-    //insert the base features
-    addConfResource(CONTRACT_XML);
-    this.isSecure = secure;
-  }
-
-  @Override
-  public String getScheme() {
-    return isSecure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME;
-  }
-
-  @Override
-  public Path getTestPath() {
-    Path path = new Path(UriUtils.generateUniqueTestPath());
-    return path;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
index d4cca14..a302fcc 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.azurebfs.contract;
 
 import java.util.Arrays;
 
+import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
@@ -40,31 +41,32 @@ public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
 
   @Override
+  @Test
   public void testRenameFileBeingAppended() throws Throwable {
     skip("Skipping as renaming an opened file is not supported");
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
index 4f724e2..c31a6d2 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
@@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
+    isSecure = secure;
+    binding = new ABFSContractTestBinding(isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
index 16b959f..ce4d229 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractCreate extends AbstractContractCreateTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractCreate(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
index fabd3273..310731c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractDelete extends AbstractContractDeleteTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractDelete(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
+    binding = new ABFSContractTestBinding(isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
index a1360e4..c2cf255 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
@@ -25,20 +25,20 @@ import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
  * Contract test for distCp operation.
  */
 public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTest {
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractDistCp() throws Exception {
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(false);
+    binding = new ABFSContractTestBinding(false);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
-  protected ITestAbfsFileSystemContract createContract(Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, false);
+  protected AbfsFileSystemContract createContract(Configuration conf) {
+    return new AbfsFileSystemContract(conf, false);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
index 5bb41ad..9ad3b21 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
@@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractGetFileStatus extends AbstractContractGe
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractGetFileStatus(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
+    binding = new ABFSContractTestBinding(isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return this.binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, this.isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
index 9d732d5..6265ca1 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractMkdir extends AbstractContractMkdirTest
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractMkdir(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(secure);
+    binding = new ABFSContractTestBinding(secure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
index a71149b..ae4bb2a 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractOpen extends AbstractContractOpenTest {
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractOpen(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
index 6d1c4ae..6e6a728 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractRename extends AbstractContractRenameTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractRename(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
index 46072ad..01dea2d 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
@@ -37,31 +37,31 @@ public class ITestAbfsFileSystemContractRootDirectory extends AbstractContractRo
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractRootDirectory(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(secure);
+    binding = new ABFSContractTestBinding(secure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
 
   @Override
   @Ignore("ABFS always return false when non-recursively remove root dir")
   public void testRmNonEmptyRootDirNonRecursive() throws Throwable {
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
index 1780f6f..5ed7466 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
@@ -25,20 +25,20 @@ import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
  * Contract test for secure distCP operation.
  */
 public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDistCpTest {
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractSecureDistCp() throws Exception {
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(true);
+    binding = new ABFSContractTestBinding(true);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
-  protected ITestAbfsFileSystemContract createContract(Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, true);
+  protected AbfsFileSystemContract createContract(Configuration conf) {
+    return new AbfsFileSystemContract(conf, true);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
index aeeb042..5e0ea0c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractSeek(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
index ea9392d..8d23b0b 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
@@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractSetTimes extends AbstractContractSetTime
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractSetTimes(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
index 9f3b4a7..d8854a2 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
@@ -22,8 +22,8 @@ import java.io.IOException;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 
-import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -35,26 +35,31 @@ import static org.junit.Assert.assertTrue;
  * Basic Contract test for Azure BlobFileSystem.
  */
 public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAzureBlobFileSystemBasics() throws Exception {
     // If all contract tests are running in parallel, some root level tests in FileSystemContractBaseTest will fail
     // due to the race condition. Hence for this contract test it should be tested in different container
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(false, false);
+    binding = new ABFSContractTestBinding(false, false);
   }
 
+
   @Before
   public void setUp() throws Exception {
-    this.dependencyInjectedContractTest.initialize();
-    fs = this.dependencyInjectedContractTest.getFileSystem();
+    binding.setup();
+    fs = binding.getFileSystem();
   }
 
-  @After
-  public void testCleanup() throws Exception {
+  @Override
+  public void tearDown() throws Exception {
     // This contract test is not using existing container for test,
     // instead it creates its own temp container for test, hence we need to destroy
     // it after the test.
-    this.dependencyInjectedContractTest.testCleanup();
+    try {
+      super.tearDown();
+    } finally {
+      binding.teardown();
+    }
   }
 
   @Test
@@ -82,7 +87,7 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
     Path filePath = path("testListStatus/file");
 
     assertTrue(fs.mkdirs(folderPath));
-    fs.create(filePath);
+    ContractTestUtils.touch(fs, filePath);
 
     FileStatus[] listFolderStatus;
     listFolderStatus = fs.listStatus(path("testListStatus"));
@@ -97,4 +102,4 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
   @Ignore("Not implemented in ABFS yet")
   public void testMkdirsWithUmask() throws Exception {
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
index 03320d6..e9e90c9 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
@@ -19,10 +19,11 @@
 package org.apache.hadoop.fs.azurebfs.diagnostics;
 
 import org.apache.commons.codec.Charsets;
+
+import org.junit.Assert;
 import org.junit.Test;
 
 
-import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.commons.codec.binary.Base64;
 
@@ -36,7 +37,7 @@ import static org.junit.Assert.assertEquals;
 /**
  * Test configuration validators.
  */
-public class TestConfigurationValidators extends DependencyInjectedTest {
+public class TestConfigurationValidators extends Assert {
 
   private static final String FAKE_KEY = "FakeKey";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java
new file mode 100644
index 0000000..dd06fe3
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsScaleTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
+
+/**
+ * Test read, write and seek.
+ * Uses package-private methods in AbfsConfiguration, which is why it is in
+ * this package.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
+  private static final Path TEST_PATH = new Path("/testfile");
+
+  @Parameterized.Parameters(name = "Size={0}")
+  public static Iterable<Object[]> sizes() {
+    return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE},
+        {DEFAULT_READ_BUFFER_SIZE},
+        {MAX_BUFFER_SIZE}});
+  }
+
+  private final int size;
+
+  public ITestAbfsReadWriteAndSeek(final int size) {
+    this.size = size;
+  }
+
+  @Test
+  public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception {
+    testReadWriteAndSeek(size);
+  }
+
+  private void testReadWriteAndSeek(int bufferSize) throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(getConfiguration());
+
+    abfsConfiguration.setWriteBufferSize(bufferSize);
+    abfsConfiguration.setReadBufferSize(bufferSize);
+
+
+    final byte[] b = new byte[2 * bufferSize];
+    new Random().nextBytes(b);
+    try(final FSDataOutputStream stream = fs.create(TEST_PATH)) {
+      stream.write(b);
+    }
+
+    final byte[] readBuffer = new byte[2 * bufferSize];
+    int result;
+    try(final FSDataInputStream inputStream = fs.open(TEST_PATH)) {
+      inputStream.seek(bufferSize);
+      result = inputStream.read(readBuffer, bufferSize, bufferSize);
+      assertNotEquals(-1, result);
+      inputStream.seek(0);
+      result = inputStream.read(readBuffer, 0, bufferSize);
+    }
+    assertNotEquals("data read in final read()", -1, result);
+    assertArrayEquals(readBuffer, b);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
deleted file mode 100644
index cabe049..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
+++ /dev/null
@@ -1,78 +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.fs.azurebfs.services;
-
-import java.util.Random;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
-import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
-
-import org.junit.Test;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertNotEquals;
-
-/**
- * Test read, write and seek.
- */
-public class ITestReadWriteAndSeek extends DependencyInjectedTest {
-  private static final Path TEST_PATH = new Path("/testfile");
-  public ITestReadWriteAndSeek() {
-    super();
-  }
-
-  @Test
-  public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception {
-    testReadWriteAndSeek(MIN_BUFFER_SIZE);
-    testReadWriteAndSeek(DEFAULT_READ_BUFFER_SIZE);
-    testReadWriteAndSeek(MAX_BUFFER_SIZE);
-  }
-
-  private void testReadWriteAndSeek(int bufferSize) throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(this.getConfiguration());
-
-    fs.create(TEST_PATH);
-    abfsConfiguration.setWriteBufferSize(bufferSize);
-    abfsConfiguration.setReadBufferSize(bufferSize);
-
-    final FSDataOutputStream stream = fs.create(TEST_PATH);
-
-    final byte[] b = new byte[2 * bufferSize];
-    new Random().nextBytes(b);
-    stream.write(b);
-    stream.close();
-
-    final byte[] r = new byte[2 * bufferSize];
-    final FSDataInputStream inputStream = fs.open(TEST_PATH);
-    inputStream.seek(bufferSize);
-    int result = inputStream.read(r, bufferSize, bufferSize);
-    assertNotEquals(-1, result);
-
-    inputStream.seek(0);
-    result = inputStream.read(r, 0, bufferSize);
-    assertNotEquals(-1, result);
-    assertArrayEquals(r, b);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
index 441a35a..ebaafa4 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
@@ -48,7 +48,7 @@ import org.junit.Test;
  * Test ConfigurationServiceFieldsValidation.
  */
 public class TestAbfsConfigurationFieldsValidation {
-  private AbfsConfiguration abfsConfiguration;
+  private final AbfsConfiguration abfsConfiguration;
 
   private static final String INT_KEY= "intKey";
   private static final String LONG_KEY= "longKey";
@@ -89,12 +89,10 @@ public class TestAbfsConfigurationFieldsValidation {
   private boolean boolField;
 
   public TestAbfsConfigurationFieldsValidation() throws Exception {
-    super();
     Base64 base64 = new Base64();
     this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
     this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
-    Configuration configuration = new Configuration();
-    configuration.addResource("azure-bfs-test.xml");
+    Configuration configuration = new Configuration(false);
     configuration.set(INT_KEY, "1234565");
     configuration.set(LONG_KEY, "4194304");
     configuration.set(STRING_KEY, "stringValue");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
index 2716bff..9051a72 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
@@ -24,7 +24,7 @@ import com.microsoft.azure.storage.blob.CloudBlobContainer;
 import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
 import org.junit.Test;
 
-import static org.apache.hadoop.fs.azurebfs.DependencyInjectedTest.TEST_CONTAINER_PREFIX;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ABFS_TEST_CONTAINER_PREFIX;
 
 /**
  * If unit tests were interrupted and crushed accidentally, the test containers won't be deleted.
@@ -38,7 +38,7 @@ public class CleanUpAbfsTestContainer {
     CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
     CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
     Iterable<CloudBlobContainer> containers
-            = blobClient.listContainers(TEST_CONTAINER_PREFIX);
+            = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX);
     for (CloudBlobContainer container : containers) {
       count++;
       System.out.println(String.format("Container %s URI %s",
@@ -54,7 +54,7 @@ public class CleanUpAbfsTestContainer {
     CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
     CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
     Iterable<CloudBlobContainer> containers
-            = blobClient.listContainers(TEST_CONTAINER_PREFIX);
+            = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX);
     for (CloudBlobContainer container : containers) {
       System.out.println(String.format("Container %s URI %s",
               container.getName(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
index caf8677..d065ace 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
+++ b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
@@ -1,3 +1,4 @@
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <!--
   ~ Licensed to the Apache Software Foundation (ASF) under one
   ~  or more contributor license agreements.  See the NOTICE file
@@ -15,7 +16,6 @@
   ~  See the License for the specific language governing permissions and
   ~  limitations under the License.
   -->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <configuration xmlns:xi="http://www.w3.org/2001/XInclude">
     <property>
         <name>fs.contract.test.root-tests-enabled</name>
@@ -61,4 +61,4 @@
         <name>fs.contract.supports-getfilestatus</name>
         <value>true</value>
     </property>
-</configuration>
\ No newline at end of file
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
index 508e5f6..464a8e6 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
+++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
@@ -1,4 +1,5 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <!--
   Licensed under the Apache License, Version 2.0 (the "License");
   you may not use this file except in compliance with the License.
@@ -13,7 +14,6 @@
   limitations under the License.
 -->
 
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <configuration xmlns:xi="http://www.w3.org/2001/XInclude">
 
   <!--
@@ -63,14 +63,4 @@
   <!--<value>false</value>-->
   <!--</property>-->
 
-  <property>
-    <name>fs.AbstractFileSystem.abfs.impl</name>
-    <value>org.apache.hadoop.fs.azurebfs.Abfs</value>
-  </property>
-
-  <property>
-    <name>fs.AbstractFileSystem.abfss.impl</name>
-    <value>org.apache.hadoop.fs.azurebfs.Abfss</value>
-  </property>
-
-</configuration>
\ No newline at end of file
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
index fd0cd9d..bac431d 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
@@ -26,3 +26,35 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG
 log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG
 log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE
 log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG
+
+# after here: turn off log messages from other parts of the system
+# which only clutter test reports.
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+log4j.logger.org.apache.hadoop.conf.Configuration.deprecation=WARN
+log4j.logger.org.apache.hadoop.util.GSet=WARN
+# MiniDFS clusters can be noisy
+log4j.logger.org.apache.hadoop.hdfs.server=ERROR
+log4j.logger.org.apache.hadoop.metrics2=WARN
+log4j.logger.org.apache.hadoop.net.NetworkTopology=WARN
+log4j.logger.org.apache.hadoop.util.JvmPauseMonitor=WARN
+log4j.logger.org.apache.hadoop.ipc=WARN
+log4j.logger.org.apache.hadoop.http=WARN
+log4j.logger.org.apache.hadoop.security.authentication.server.AuthenticationFilter=WARN
+log4j.logger.org.apache.hadoop.util.HostsFileReader=WARN
+log4j.logger.org.apache.commons.beanutils=WARN
+log4j.logger.org.apache.hadoop.hdfs.StateChange=WARN
+log4j.logger.BlockStateChange=WARN
+log4j.logger.org.apache.hadoop.hdfs.DFSUtil=WARN
+## YARN can be noisy too
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.scheduler=WARN
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager=WARN
+log4j.logger.org.apache.hadoop.yarn.event=WARN
+log4j.logger.org.apache.hadoop.yarn.util.ResourceCalculatorPlugin=ERROR
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN
+log4j.logger.org.apache.hadoop.mapred.IndexCache=WARN
+log4j.logger.org.apache.hadoop.yarn.webapp.WebApps=WARN
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.security=WARN
+log4j.logger.org.apache.hadoop.yarn.util.AbstractLivelinessMonitor=WARN
+log4j.logger.org.apache.hadoop.security.token.delegation=WARN
+log4j.logger.org.apache.hadoop.mapred.ShuffleHandler=WARN
+log4j.logger.org.apache.hadoop.ipc.Server=WARN


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


[12/50] [abbrv] hadoop git commit: HADOOP-15583. Stabilize S3A Assumed Role support. Contributed by Steve Loughran.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
index 3afd63f..8af0457 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
@@ -29,7 +29,7 @@ assumed roles for different buckets.
 *IAM Assumed Roles are unlikely to be supported by third-party systems
 supporting the S3 APIs.*
 
-## Using IAM Assumed Roles
+## <a name="using_assumed_roles"></a> Using IAM Assumed Roles
 
 ### Before You Begin
 
@@ -40,6 +40,8 @@ are, how to configure their policies, etc.
 * You need a pair of long-lived IAM User credentials, not the root account set.
 * Have the AWS CLI installed, and test that it works there.
 * Give the role access to S3, and, if using S3Guard, to DynamoDB.
+* For working with data encrypted with SSE-KMS, the role must
+have access to the appropriate KMS keys.
 
 Trying to learn how IAM Assumed Roles work by debugging stack traces from
 the S3A client is "suboptimal".
@@ -51,7 +53,7 @@ To use assumed roles, the client must be configured to use the
 in the configuration option `fs.s3a.aws.credentials.provider`.
 
 This AWS Credential provider will read in the `fs.s3a.assumed.role` options needed to connect to the
-Session Token Service [Assumed Role API](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html),
+Security Token Service [Assumed Role API](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html),
 first authenticating with the full credentials, then assuming the specific role
 specified. It will then refresh this login at the configured rate of
 `fs.s3a.assumed.role.session.duration`
@@ -69,7 +71,7 @@ which uses `fs.s3a.access.key` and `fs.s3a.secret.key`.
 Note: although you can list other AWS credential providers in  to the
 Assumed Role Credential Provider, it can only cause confusion.
 
-### <a name="using"></a> Using Assumed Roles
+### <a name="using"></a> Configuring Assumed Roles
 
 To use assumed roles, the S3A client credentials provider must be set to
 the `AssumedRoleCredentialProvider`, and `fs.s3a.assumed.role.arn` to
@@ -78,7 +80,6 @@ the previously created ARN.
 ```xml
 <property>
   <name>fs.s3a.aws.credentials.provider</name>
-  <value>org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider</value>
   <value>org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider</value>
 </property>
 
@@ -159,7 +160,18 @@ Here are the full set of configuration options.
   <name>fs.s3a.assumed.role.sts.endpoint</name>
   <value/>
   <description>
-    AWS Simple Token Service Endpoint. If unset, uses the default endpoint.
+    AWS Security Token Service Endpoint. If unset, uses the default endpoint.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.sts.endpoint.region</name>
+  <value>us-west-1</value>
+  <description>
+    AWS Security Token Service Endpoint's region;
+    Needed if fs.s3a.assumed.role.sts.endpoint points to an endpoint
+    other than the default one and the v4 signature is used.
     Only used if AssumedRoleCredentialProvider is the AWS credential provider.
   </description>
 </property>
@@ -194,39 +206,101 @@ These lists represent the minimum actions to which the client's principal
 must have in order to work with a bucket.
 
 
-### Read Access Permissions
+### <a name="read-permissions"></a> Read Access Permissions
 
 Permissions which must be granted when reading from a bucket:
 
 
-| Action | S3A operations |
-|--------|----------|
-| `s3:ListBucket` | `listStatus()`, `getFileStatus()` and elsewhere |
-| `s3:GetObject` | `getFileStatus()`, `open()` and elsewhere |
-| `s3:ListBucketMultipartUploads` |  Aborting/cleaning up S3A commit operations|
+```
+s3:Get*
+s3:ListBucket
+```
+
+When using S3Guard, the client needs the appropriate
+<a href="s3guard-permissions">DynamoDB access permissions</a>
+
+To use SSE-KMS encryption, the client needs the
+<a href="sse-kms-permissions">SSE-KMS Permissions</a> to access the
+KMS key(s).
+
+### <a name="write-permissions"></a> Write Access Permissions
+
+These permissions must all be granted for write access:
+
+```
+s3:Get*
+s3:Delete*
+s3:Put*
+s3:ListBucket
+s3:ListBucketMultipartUploads
+s3:AbortMultipartUpload
+```
+
+### <a name="sse-kms-permissions"></a> SSE-KMS Permissions
+
+When to read data encrypted using SSE-KMS, the client must have
+ `kms:Decrypt` permission for the specific key a file was encrypted with.
+
+```
+kms:Decrypt
+```
+
+To write data using SSE-KMS, the client must have all the following permissions.
+
+```
+kms:Decrypt
+kms:GenerateDataKey
+```
 
+This includes renaming: renamed files are encrypted with the encryption key
+of the current S3A client; it must decrypt the source file first.
 
-The `s3:ListBucketMultipartUploads` is only needed when committing work
-via the [S3A committers](committers.html).
-However, it must be granted to the root path in order to safely clean up jobs.
-It is simplest to permit this in all buckets, even if it is only actually
-needed when writing data.
+If the caller doesn't have these permissions, the operation will fail with an
+`AccessDeniedException`: the S3 Store does not provide the specifics of
+the cause of the failure.
 
+### <a name="s3guard-permissions"></a> S3Guard Permissions
 
-### Write Access Permissions
+To use S3Guard, all clients must have a subset of the
+[AWS DynamoDB Permissions](https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/api-permissions-reference.html).
 
-These permissions must *also* be granted for write access:
+To work with buckets protected with S3Guard, the client must have
+all the following rights on the DynamoDB Table used to protect that bucket.
 
+```
+dynamodb:BatchGetItem
+dynamodb:BatchWriteItem
+dynamodb:DeleteItem
+dynamodb:DescribeTable
+dynamodb:GetItem
+dynamodb:PutItem
+dynamodb:Query
+dynamodb:UpdateItem
+```
 
-| Action | S3A operations |
-|--------|----------|
-| `s3:PutObject` | `mkdir()`, `create()`, `rename()`, `delete()` |
-| `s3:DeleteObject` | `mkdir()`, `create()`, `rename()`, `delete()` |
-| `s3:AbortMultipartUpload` | S3A committer `abortJob()` and `cleanup()` operations |
-| `s3:ListMultipartUploadParts` | S3A committer `abortJob()` and `cleanup()` operations |
+This is true, *even if the client only has read access to the data*.
 
+For the `hadoop s3guard` table management commands, _extra_ permissions are required:
 
-### Mixed Permissions in a single S3 Bucket
+```
+dynamodb:CreateTable
+dynamodb:DescribeLimits
+dynamodb:DeleteTable
+dynamodb:Scan
+dynamodb:TagResource
+dynamodb:UntagResource
+dynamodb:UpdateTable
+```
+
+Without these permissions, tables cannot be created, destroyed or have their IO capacity
+changed through the `s3guard set-capacity` call.
+The `dynamodb:Scan` permission is needed for `s3guard prune`
+
+The `dynamodb:CreateTable` permission is needed by a client it tries to
+create the DynamoDB table on startup, that is
+`fs.s3a.s3guard.ddb.table.create` is `true` and the table does not already exist.
+
+### <a name="mixed-permissions"></a> Mixed Permissions in a single S3 Bucket
 
 Mixing permissions down the "directory tree" is limited
 only to the extent of supporting writeable directories under
@@ -274,7 +348,7 @@ This example has the base bucket read only, and a directory underneath,
     "Action" : [
       "s3:ListBucket",
       "s3:ListBucketMultipartUploads",
-      "s3:GetObject"
+      "s3:Get*"
       ],
     "Resource" : "arn:aws:s3:::example-bucket/*"
   }, {
@@ -320,7 +394,7 @@ the command line before trying to use the S3A client.
 `hadoop fs -mkdirs -p s3a://bucket/path/p1/`
 
 
-### <a name="no_role"></a>IOException: "Unset property fs.s3a.assumed.role.arn"
+### <a name="no_role"></a> IOException: "Unset property fs.s3a.assumed.role.arn"
 
 The Assumed Role Credential Provider is enabled, but `fs.s3a.assumed.role.arn` is unset.
 
@@ -339,7 +413,7 @@ java.io.IOException: Unset property fs.s3a.assumed.role.arn
   at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
 ```
 
-### <a name="not_authorized_for_assumed_role"></a>"Not authorized to perform sts:AssumeRole"
+### <a name="not_authorized_for_assumed_role"></a> "Not authorized to perform sts:AssumeRole"
 
 This can arise if the role ARN set in `fs.s3a.assumed.role.arn` is invalid
 or one to which the caller has no access.
@@ -399,7 +473,8 @@ Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceExc
 The value of `fs.s3a.assumed.role.session.duration` is out of range.
 
 ```
-java.lang.IllegalArgumentException: Assume Role session duration should be in the range of 15min - 1Hr
+java.lang.IllegalArgumentException: Assume Role session duration should be in the range of 15min
+- 1Hr
   at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$Builder.withRoleSessionDurationSeconds(STSAssumeRoleSessionCredentialsProvider.java:437)
   at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:86)
 ```
@@ -603,7 +678,7 @@ Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceExc
 
 ### <a name="invalid_token"></a> `AccessDeniedException/InvalidClientTokenId`: "The security token included in the request is invalid"
 
-The credentials used to authenticate with the AWS Simple Token Service are invalid.
+The credentials used to authenticate with the AWS Security Token Service are invalid.
 
 ```
 [ERROR] Failures:
@@ -682,26 +757,7 @@ org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.f
   at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
   at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
   at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
-  at org.apache.hadoop.fs.s3a.ITestAssumeRole.lambda$expectFileSystemFailure$0(ITestAssumeRole.java:70)
-  at org.apache.hadoop.fs.s3a.ITestAssumeRole.lambda$interceptC$1(ITestAssumeRole.java:84)
-  at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:491)
-  at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:377)
-  at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:446)
-  at org.apache.hadoop.fs.s3a.ITestAssumeRole.interceptC(ITestAssumeRole.java:82)
-  at org.apache.hadoop.fs.s3a.ITestAssumeRole.expectFileSystemFailure(ITestAssumeRole.java:68)
-  at org.apache.hadoop.fs.s3a.ITestAssumeRole.testAssumeRoleBadSession(ITestAssumeRole.java:216)
-  at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
-  at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
-  at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
-  at java.lang.reflect.Method.invoke(Method.java:498)
-  at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
-  at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
-  at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
-  at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
-  at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
-  at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
-  at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
-  at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
+
 Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
     1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName'
     failed to satisfy constraint:
@@ -742,10 +798,11 @@ Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceExc
 ### <a name="access_denied"></a> `java.nio.file.AccessDeniedException` within a FileSystem API call
 
 If an operation fails with an `AccessDeniedException`, then the role does not have
-the permission for the S3 Operation invoked during the call
+the permission for the S3 Operation invoked during the call.
 
 ```
-java.nio.file.AccessDeniedException: s3a://bucket/readonlyDir: rename(s3a://bucket/readonlyDir, s3a://bucket/renameDest)
+java.nio.file.AccessDeniedException: s3a://bucket/readonlyDir:
+  rename(s3a://bucket/readonlyDir, s3a://bucket/renameDest)
  on s3a://bucket/readonlyDir:
   com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied
   (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 2805F2ABF5246BB1;
@@ -795,3 +852,33 @@ check the path for the operation.
 Make sure that all the read and write permissions are allowed for any bucket/path
 to which data is being written to, and read permissions for all
 buckets read from.
+
+If the bucket is using SSE-KMS to encrypt data:
+
+1. The caller must have the `kms:Decrypt` permission to read the data.
+1. The caller needs `kms:Decrypt` and `kms:GenerateDataKey`.
+
+Without permissions, the request fails *and there is no explicit message indicating
+that this is an encryption-key issue*.
+
+### <a name="dynamodb_exception"></a> `AccessDeniedException` + `AmazonDynamoDBException`
+
+```
+java.nio.file.AccessDeniedException: bucket1:
+  com.amazonaws.services.dynamodbv2.model.AmazonDynamoDBException:
+  User: arn:aws:sts::980678866538:assumed-role/s3guard-test-role/test is not authorized to perform:
+  dynamodb:DescribeTable on resource: arn:aws:dynamodb:us-west-1:980678866538:table/bucket1
+   (Service: AmazonDynamoDBv2; Status Code: 400;
+```
+
+The caller is trying to access an S3 bucket which uses S3Guard, but the caller
+lacks the relevant DynamoDB access permissions.
+
+The `dynamodb:DescribeTable` operation is the first one used in S3Guard to access,
+the DynamoDB table, so it is often the first to fail. It can be a sign
+that the role has no permissions at all to access the table named in the exception,
+or just that this specific permission has been omitted.
+
+If the role policy requested for the assumed role didn't ask for any DynamoDB
+permissions, this is where all attempts to work with a S3Guarded bucket will
+fail. Check the value of `fs.s3a.assumed.role.policy`

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 7d0f67b..2dee10a 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -33,7 +33,7 @@ See also:
 * [Working with IAM Assumed Roles](./assumed_roles.html)
 * [Testing](./testing.html)
 
-##<a name="overview"></a> Overview
+## <a name="overview"></a> Overview
 
 Apache Hadoop's `hadoop-aws` module provides support for AWS integration.
 applications to easily use this support.
@@ -88,7 +88,7 @@ maintain it.
    This connector is no longer available: users must migrate to the newer `s3a:` client.
 
 
-##<a name="getting_started"></a> Getting Started
+## <a name="getting_started"></a> Getting Started
 
 S3A depends upon two JARs, alongside `hadoop-common` and its dependencies.
 
@@ -1698,6 +1698,6 @@ as configured by the value `fs.s3a.multipart.size`.
 To disable checksum verification in `distcp`, use the `-skipcrccheck` option:
 
 ```bash
-hadoop distcp -update -skipcrccheck /user/alice/datasets s3a://alice-backup/datasets
+hadoop distcp -update -skipcrccheck -numListstatusThreads 40 /user/alice/datasets s3a://alice-backup/datasets
 ```
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index aa6b5d8..3214c76 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -36,14 +36,6 @@ import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 import java.io.File;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
@@ -60,6 +52,9 @@ import org.junit.rules.TemporaryFolder;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
+import static org.junit.Assert.*;
 
 /**
  * S3A tests for configuration.
@@ -134,12 +129,26 @@ public class ITestS3AConfiguration {
     conf.setInt(Constants.PROXY_PORT, 1);
     String proxy =
         conf.get(Constants.PROXY_HOST) + ":" + conf.get(Constants.PROXY_PORT);
-    try {
-      fs = S3ATestUtils.createTestFileSystem(conf);
-      fail("Expected a connection error for proxy server at " + proxy);
-    } catch (AWSClientIOException e) {
-      // expected
-    }
+    expectFSCreateFailure(AWSClientIOException.class,
+        conf, "when using proxy " + proxy);
+  }
+
+  /**
+   * Expect a filesystem to not be created from a configuration
+   * @return the exception intercepted
+   * @throws Exception any other exception
+   */
+  private <E extends Throwable> E expectFSCreateFailure(
+      Class<E> clazz,
+      Configuration conf,
+      String text)
+      throws Exception {
+
+    return intercept(clazz,
+        () -> {
+          fs = S3ATestUtils.createTestFileSystem(conf);
+          return "expected failure creating FS " + text + " got " + fs;
+        });
   }
 
   @Test
@@ -148,15 +157,13 @@ public class ITestS3AConfiguration {
     conf.unset(Constants.PROXY_HOST);
     conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
     conf.setInt(Constants.PROXY_PORT, 1);
-    try {
-      fs = S3ATestUtils.createTestFileSystem(conf);
-      fail("Expected a proxy configuration error");
-    } catch (IllegalArgumentException e) {
-      String msg = e.toString();
-      if (!msg.contains(Constants.PROXY_HOST) &&
-          !msg.contains(Constants.PROXY_PORT)) {
-        throw e;
-      }
+    IllegalArgumentException e = expectFSCreateFailure(
+        IllegalArgumentException.class,
+        conf, "Expected a connection error for proxy server");
+    String msg = e.toString();
+    if (!msg.contains(Constants.PROXY_HOST) &&
+        !msg.contains(Constants.PROXY_PORT)) {
+      throw e;
     }
   }
 
@@ -167,19 +174,11 @@ public class ITestS3AConfiguration {
     conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
     conf.set(Constants.PROXY_HOST, "127.0.0.1");
     conf.set(Constants.SECURE_CONNECTIONS, "true");
-    try {
-      fs = S3ATestUtils.createTestFileSystem(conf);
-      fail("Expected a connection error for proxy server");
-    } catch (AWSClientIOException e) {
-      // expected
-    }
+    expectFSCreateFailure(AWSClientIOException.class,
+        conf, "Expected a connection error for proxy server");
     conf.set(Constants.SECURE_CONNECTIONS, "false");
-    try {
-      fs = S3ATestUtils.createTestFileSystem(conf);
-      fail("Expected a connection error for proxy server");
-    } catch (AWSClientIOException e) {
-      // expected
-    }
+    expectFSCreateFailure(AWSClientIOException.class,
+        conf, "Expected a connection error for proxy server");
   }
 
   @Test
@@ -189,31 +188,31 @@ public class ITestS3AConfiguration {
     conf.set(Constants.PROXY_HOST, "127.0.0.1");
     conf.setInt(Constants.PROXY_PORT, 1);
     conf.set(Constants.PROXY_USERNAME, "user");
-    try {
-      fs = S3ATestUtils.createTestFileSystem(conf);
-      fail("Expected a connection error for proxy server");
-    } catch (IllegalArgumentException e) {
-      String msg = e.toString();
-      if (!msg.contains(Constants.PROXY_USERNAME) &&
-          !msg.contains(Constants.PROXY_PASSWORD)) {
-        throw e;
-      }
+    IllegalArgumentException e = expectFSCreateFailure(
+        IllegalArgumentException.class,
+        conf, "Expected a connection error for proxy server");
+    assertIsProxyUsernameError(e);
+  }
+
+  private void assertIsProxyUsernameError(final IllegalArgumentException e) {
+    String msg = e.toString();
+    if (!msg.contains(Constants.PROXY_USERNAME) &&
+        !msg.contains(Constants.PROXY_PASSWORD)) {
+      throw e;
     }
+  }
+
+  @Test
+  public void testUsernameInconsistentWithPassword2() throws Exception {
     conf = new Configuration();
     conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
     conf.set(Constants.PROXY_HOST, "127.0.0.1");
     conf.setInt(Constants.PROXY_PORT, 1);
     conf.set(Constants.PROXY_PASSWORD, "password");
-    try {
-      fs = S3ATestUtils.createTestFileSystem(conf);
-      fail("Expected a connection error for proxy server");
-    } catch (IllegalArgumentException e) {
-      String msg = e.toString();
-      if (!msg.contains(Constants.PROXY_USERNAME) &&
-          !msg.contains(Constants.PROXY_PASSWORD)) {
-        throw e;
-      }
-    }
+    IllegalArgumentException e = expectFSCreateFailure(
+        IllegalArgumentException.class,
+        conf, "Expected a connection error for proxy server");
+    assertIsProxyUsernameError(e);
   }
 
   @Test
@@ -393,7 +392,7 @@ public class ITestS3AConfiguration {
       // Catch/pass standard path style access behaviour when live bucket
       // isn't in the same region as the s3 client default. See
       // http://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html
-      assertEquals(e.getStatusCode(), HttpStatus.SC_MOVED_PERMANENTLY);
+      assertEquals(HttpStatus.SC_MOVED_PERMANENTLY, e.getStatusCode());
     }
   }
 
@@ -428,8 +427,16 @@ public class ITestS3AConfiguration {
   public void testCloseIdempotent() throws Throwable {
     conf = new Configuration();
     fs = S3ATestUtils.createTestFileSystem(conf);
+    AWSCredentialProviderList credentials =
+        fs.shareCredentials("testCloseIdempotent");
+    credentials.close();
     fs.close();
+    assertTrue("Closing FS didn't close credentials " + credentials,
+        credentials.isClosed());
+    assertEquals("refcount not zero in " + credentials, 0, credentials.getRefCount());
     fs.close();
+    // and the numbers should not change
+    assertEquals("refcount not zero in " + credentials, 0, credentials.getRefCount());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java
index 44a2beb..afc4086 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java
@@ -19,15 +19,14 @@
 package org.apache.hadoop.fs.s3a;
 
 import java.io.IOException;
-import java.net.URI;
 
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
 import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
 import com.amazonaws.services.securitytoken.model.GetSessionTokenResult;
 import com.amazonaws.services.securitytoken.model.Credentials;
 
-import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+import org.apache.hadoop.fs.s3a.auth.STSClientFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.LambdaTestUtils;
 
@@ -55,6 +54,14 @@ public class ITestS3ATemporaryCredentials extends AbstractS3ATestBase {
 
   private static final long TEST_FILE_SIZE = 1024;
 
+  private AWSCredentialProviderList credentials;
+
+  @Override
+  public void teardown() throws Exception {
+    S3AUtils.closeAutocloseables(LOG, credentials);
+    super.teardown();
+  }
+
   /**
    * Test use of STS for requesting temporary credentials.
    *
@@ -63,7 +70,7 @@ public class ITestS3ATemporaryCredentials extends AbstractS3ATestBase {
    * S3A tests to request temporary credentials, then attempt to use those
    * credentials instead.
    *
-   * @throws IOException
+   * @throws IOException failure
    */
   @Test
   public void testSTS() throws IOException {
@@ -71,21 +78,20 @@ public class ITestS3ATemporaryCredentials extends AbstractS3ATestBase {
     if (!conf.getBoolean(TEST_STS_ENABLED, true)) {
       skip("STS functional tests disabled");
     }
-
-    S3xLoginHelper.Login login = S3AUtils.getAWSAccessKeys(
-        URI.create("s3a://foobar"), conf);
-    if (!login.hasLogin()) {
-      skip("testSTS disabled because AWS credentials not configured");
-    }
-    AWSCredentialsProvider parentCredentials = new BasicAWSCredentialsProvider(
-        login.getUser(), login.getPassword());
-
-    String stsEndpoint = conf.getTrimmed(TEST_STS_ENDPOINT, "");
-    AWSSecurityTokenServiceClient stsClient;
-    stsClient = new AWSSecurityTokenServiceClient(parentCredentials);
-    if (!stsEndpoint.isEmpty()) {
-      LOG.debug("STS Endpoint ={}", stsEndpoint);
-      stsClient.setEndpoint(stsEndpoint);
+    S3AFileSystem testFS = getFileSystem();
+    credentials = testFS.shareCredentials("testSTS");
+
+    String bucket = testFS.getBucket();
+    AWSSecurityTokenServiceClientBuilder builder = STSClientFactory.builder(
+        conf,
+        bucket,
+        credentials,
+        conf.getTrimmed(TEST_STS_ENDPOINT, ""), "");
+    AWSSecurityTokenService stsClient = builder.build();
+
+    if (!conf.getTrimmed(TEST_STS_ENDPOINT, "").isEmpty()) {
+      LOG.debug("STS Endpoint ={}", conf.getTrimmed(TEST_STS_ENDPOINT, ""));
+      stsClient.setEndpoint(conf.getTrimmed(TEST_STS_ENDPOINT, ""));
     }
     GetSessionTokenRequest sessionTokenRequest = new GetSessionTokenRequest();
     sessionTokenRequest.setDurationSeconds(900);
@@ -93,23 +99,28 @@ public class ITestS3ATemporaryCredentials extends AbstractS3ATestBase {
     sessionTokenResult = stsClient.getSessionToken(sessionTokenRequest);
     Credentials sessionCreds = sessionTokenResult.getCredentials();
 
-    String childAccessKey = sessionCreds.getAccessKeyId();
-    conf.set(ACCESS_KEY, childAccessKey);
-    String childSecretKey = sessionCreds.getSecretAccessKey();
-    conf.set(SECRET_KEY, childSecretKey);
-    String sessionToken = sessionCreds.getSessionToken();
-    conf.set(SESSION_TOKEN, sessionToken);
+    // clone configuration so changes here do not affect the base FS.
+    Configuration conf2 = new Configuration(conf);
+    S3AUtils.clearBucketOption(conf2, bucket, AWS_CREDENTIALS_PROVIDER);
+    S3AUtils.clearBucketOption(conf2, bucket, ACCESS_KEY);
+    S3AUtils.clearBucketOption(conf2, bucket, SECRET_KEY);
+    S3AUtils.clearBucketOption(conf2, bucket, SESSION_TOKEN);
+
+    conf2.set(ACCESS_KEY, sessionCreds.getAccessKeyId());
+    conf2.set(SECRET_KEY, sessionCreds.getSecretAccessKey());
+    conf2.set(SESSION_TOKEN, sessionCreds.getSessionToken());
 
-    conf.set(AWS_CREDENTIALS_PROVIDER, PROVIDER_CLASS);
+    conf2.set(AWS_CREDENTIALS_PROVIDER, PROVIDER_CLASS);
 
-    try(S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+    // with valid credentials, we can set properties.
+    try(S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf2)) {
       createAndVerifyFile(fs, path("testSTS"), TEST_FILE_SIZE);
     }
 
     // now create an invalid set of credentials by changing the session
     // token
-    conf.set(SESSION_TOKEN, "invalid-" + sessionToken);
-    try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+    conf2.set(SESSION_TOKEN, "invalid-" + sessionCreds.getSessionToken());
+    try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf2)) {
       createAndVerifyFile(fs, path("testSTSInvalidToken"), TEST_FILE_SIZE);
       fail("Expected an access exception, but file access to "
           + fs.getUri() + " was allowed: " + fs);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
index 763819b..a1df1a5 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 import com.amazonaws.services.s3.model.ListObjectsV2Request;
 import com.amazonaws.services.s3.model.ListObjectsV2Result;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -28,6 +29,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.s3a.S3AContract;
+
 import org.junit.Assume;
 import org.junit.Test;
 
@@ -37,6 +39,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
@@ -71,7 +74,9 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
     // Other configs would break test assumptions
     conf.set(FAIL_INJECT_INCONSISTENCY_KEY, DEFAULT_DELAY_KEY_SUBSTRING);
     conf.setFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY, 1.0f);
-    conf.setLong(FAIL_INJECT_INCONSISTENCY_MSEC, DEFAULT_DELAY_KEY_MSEC);
+    // this is a long value to guarantee that the inconsistency holds
+    // even over long-haul connections, and in the debugger too/
+    conf.setLong(FAIL_INJECT_INCONSISTENCY_MSEC, (long) (60 * 1000));
     return new S3AContract(conf);
   }
 
@@ -524,37 +529,60 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
 
     ListObjectsV2Result postDeleteDelimited = listObjectsV2(fs, key, "/");
     ListObjectsV2Result postDeleteUndelimited = listObjectsV2(fs, key, null);
-
-    assertEquals("InconsistentAmazonS3Client added back objects incorrectly " +
+    assertListSizeEqual(
+        "InconsistentAmazonS3Client added back objects incorrectly " +
             "in a non-recursive listing",
-        preDeleteDelimited.getObjectSummaries().size(),
-        postDeleteDelimited.getObjectSummaries().size()
-    );
-    assertEquals("InconsistentAmazonS3Client added back prefixes incorrectly " +
+        preDeleteDelimited.getObjectSummaries(),
+        postDeleteDelimited.getObjectSummaries());
+
+    assertListSizeEqual("InconsistentAmazonS3Client added back prefixes incorrectly " +
             "in a non-recursive listing",
-        preDeleteDelimited.getCommonPrefixes().size(),
-        postDeleteDelimited.getCommonPrefixes().size()
+        preDeleteDelimited.getCommonPrefixes(),
+        postDeleteDelimited.getCommonPrefixes()
     );
-    assertEquals("InconsistentAmazonS3Client added back objects incorrectly " +
+    assertListSizeEqual("InconsistentAmazonS3Client added back objects incorrectly " +
             "in a recursive listing",
-        preDeleteUndelimited.getObjectSummaries().size(),
-        postDeleteUndelimited.getObjectSummaries().size()
+        preDeleteUndelimited.getObjectSummaries(),
+        postDeleteUndelimited.getObjectSummaries()
     );
-    assertEquals("InconsistentAmazonS3Client added back prefixes incorrectly " +
+
+    assertListSizeEqual("InconsistentAmazonS3Client added back prefixes incorrectly " +
             "in a recursive listing",
-        preDeleteUndelimited.getCommonPrefixes().size(),
-        postDeleteUndelimited.getCommonPrefixes().size()
+        preDeleteUndelimited.getCommonPrefixes(),
+        postDeleteUndelimited.getCommonPrefixes()
     );
   }
 
   /**
-   * retrying v2 list.
-   * @param fs
-   * @param key
-   * @param delimiter
-   * @return
+   * Assert that the two list sizes match; failure message includes the lists.
+   * @param message text for the assertion
+   * @param expected expected list
+   * @param actual actual list
+   * @param <T> type of list
+   */
+  private <T> void assertListSizeEqual(String message,
+      List<T> expected,
+      List<T> actual) {
+    String leftContents = expected.stream()
+        .map(n -> n.toString())
+        .collect(Collectors.joining("\n"));
+    String rightContents = actual.stream()
+        .map(n -> n.toString())
+        .collect(Collectors.joining("\n"));
+    String summary = "\nExpected:" + leftContents
+        + "\n-----------\nActual:" + rightContents;
+    assertEquals(message + summary, expected.size(), actual.size());
+  }
+
+  /**
+   * Retrying v2 list directly through the s3 client.
+   * @param fs filesystem
+   * @param key key to list under
+   * @param delimiter any delimiter
+   * @return the listing
    * @throws IOException on error
    */
+  @Retries.RetryRaw
   private ListObjectsV2Result listObjectsV2(S3AFileSystem fs,
       String key, String delimiter) throws IOException {
     ListObjectsV2Request k = fs.createListObjectsRequest(key, delimiter)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
index c8a54b8..d5cd4d4 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
@@ -65,11 +65,12 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     // delete the existing directory (in case of last test failure)
     noS3Guard.delete(directory, true);
     // Create a directory on S3 only
-    noS3Guard.mkdirs(new Path(directory, "OnS3"));
+    Path onS3 = new Path(directory, "OnS3");
+    noS3Guard.mkdirs(onS3);
     // Create a directory on both S3 and metadata store
-    Path p = new Path(directory, "OnS3AndMS");
-    ContractTestUtils.assertPathDoesNotExist(noWriteBack, "path", p);
-    noWriteBack.mkdirs(p);
+    Path onS3AndMS = new Path(directory, "OnS3AndMS");
+    ContractTestUtils.assertPathDoesNotExist(noWriteBack, "path", onS3AndMS);
+    noWriteBack.mkdirs(onS3AndMS);
 
     FileStatus[] fsResults;
     DirListingMetadata mdResults;
@@ -83,6 +84,8 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     // Metadata store without write-back should still only contain /OnS3AndMS,
     // because newly discovered /OnS3 is not written back to metadata store
     mdResults = noWriteBack.getMetadataStore().listChildren(directory);
+    assertNotNull("No results from noWriteBack listChildren " + directory,
+        mdResults);
     assertEquals("Metadata store without write back should still only know "
             + "about /OnS3AndMS, but it has: " + mdResults,
         1, mdResults.numEntries());
@@ -102,8 +105,7 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
 
     // If we don't clean this up, the next test run will fail because it will
     // have recorded /OnS3 being deleted even after it's written to noS3Guard.
-    getFileSystem().getMetadataStore().forgetMetadata(
-        new Path(directory, "OnS3"));
+    getFileSystem().getMetadataStore().forgetMetadata(onS3);
   }
 
   /**
@@ -118,26 +120,33 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
 
     // Create a FileSystem that is S3-backed only
     conf = createConfiguration();
-    S3ATestUtils.disableFilesystemCaching(conf);
     String host = fsURI.getHost();
-    if (disableS3Guard) {
-      conf.set(Constants.S3_METADATA_STORE_IMPL,
-          Constants.S3GUARD_METASTORE_NULL);
-      S3AUtils.setBucketOption(conf, host,
-          S3_METADATA_STORE_IMPL,
-          S3GUARD_METASTORE_NULL);
-    } else {
-      S3ATestUtils.maybeEnableS3Guard(conf);
-      conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritativeMeta);
-      S3AUtils.setBucketOption(conf, host,
-          METADATASTORE_AUTHORITATIVE,
-          Boolean.toString(authoritativeMeta));
-      S3AUtils.setBucketOption(conf, host,
-          S3_METADATA_STORE_IMPL,
-          conf.get(S3_METADATA_STORE_IMPL));
+    String metastore;
+
+    metastore = S3GUARD_METASTORE_NULL;
+    if (!disableS3Guard) {
+      // pick up the metadata store used by the main test
+      metastore = getFileSystem().getConf().get(S3_METADATA_STORE_IMPL);
+      assertNotEquals(S3GUARD_METASTORE_NULL, metastore);
     }
-    FileSystem fs = FileSystem.get(fsURI, conf);
-    return asS3AFS(fs);
+
+    conf.set(Constants.S3_METADATA_STORE_IMPL, metastore);
+    conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritativeMeta);
+    S3AUtils.setBucketOption(conf, host,
+        METADATASTORE_AUTHORITATIVE,
+        Boolean.toString(authoritativeMeta));
+    S3AUtils.setBucketOption(conf, host,
+        S3_METADATA_STORE_IMPL, metastore);
+
+    S3AFileSystem fs = asS3AFS(FileSystem.newInstance(fsURI, conf));
+    // do a check to verify that everything got through
+    assertEquals("Metadata store should have been disabled: " + fs,
+        disableS3Guard, !fs.hasMetadataStore());
+    assertEquals("metastore option did not propagate",
+        metastore, fs.getConf().get(S3_METADATA_STORE_IMPL));
+
+    return fs;
+
   }
 
   private static S3AFileSystem asS3AFS(FileSystem fs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
index b746bfe5..dbf228d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
@@ -23,6 +23,7 @@ import static org.mockito.Mockito.*;
 import java.net.URI;
 import java.util.ArrayList;
 
+import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.MultipartUploadListing;
 import com.amazonaws.services.s3.model.Region;
@@ -34,8 +35,9 @@ import com.amazonaws.services.s3.model.Region;
 public class MockS3ClientFactory implements S3ClientFactory {
 
   @Override
-  public AmazonS3 createS3Client(URI name) {
-    String bucket = name.getHost();
+  public AmazonS3 createS3Client(URI name,
+      final String bucket,
+      final AWSCredentialsProvider credentialSet) {
     AmazonS3 s3 = mock(AmazonS3.class);
     when(s3.doesBucketExist(bucket)).thenReturn(true);
     // this listing is used in startup if purging is enabled, so

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
index d731ae7..b28925c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 import java.io.IOException;
 import java.net.URI;
+import java.nio.file.AccessDeniedException;
 import java.util.Arrays;
 import java.util.List;
 
@@ -34,11 +35,15 @@ import org.junit.rules.ExpectedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider;
+import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.test.GenericTestUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.junit.Assert.*;
 
 /**
@@ -221,14 +226,13 @@ public class TestS3AAWSCredentialsProvider {
   }
 
   private void expectProviderInstantiationFailure(String option,
-      String expectedErrorText) throws IOException {
+      String expectedErrorText) throws Exception {
     Configuration conf = new Configuration();
     conf.set(AWS_CREDENTIALS_PROVIDER, option);
     Path testFile = new Path(
         conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
-    expectException(IOException.class, expectedErrorText);
-    URI uri = testFile.toUri();
-    S3AUtils.createAWSCredentialProviderSet(uri, conf);
+    intercept(IOException.class, expectedErrorText,
+        () -> S3AUtils.createAWSCredentialProviderSet(testFile.toUri(), conf));
   }
 
   /**
@@ -288,4 +292,68 @@ public class TestS3AAWSCredentialsProvider {
         authenticationContains(conf, AssumedRoleCredentialProvider.NAME));
   }
 
+  @Test
+  public void testExceptionLogic() throws Throwable {
+    AWSCredentialProviderList providers
+        = new AWSCredentialProviderList();
+    // verify you can't get credentials from it
+    NoAuthWithAWSException noAuth = intercept(NoAuthWithAWSException.class,
+        AWSCredentialProviderList.NO_AWS_CREDENTIAL_PROVIDERS,
+        () -> providers.getCredentials());
+    // but that it closes safely
+    providers.close();
+
+    S3ARetryPolicy retryPolicy = new S3ARetryPolicy(new Configuration());
+    assertEquals("Expected no retry on auth failure",
+        RetryPolicy.RetryAction.FAIL.action,
+        retryPolicy.shouldRetry(noAuth, 0, 0, true).action);
+
+    try {
+      throw S3AUtils.translateException("login", "", noAuth);
+    } catch (AccessDeniedException expected) {
+      // this is what we want; other exceptions will be passed up
+      assertEquals("Expected no retry on AccessDeniedException",
+          RetryPolicy.RetryAction.FAIL.action,
+          retryPolicy.shouldRetry(expected, 0, 0, true).action);
+    }
+
+  }
+
+  @Test
+  public void testRefCounting() throws Throwable {
+    AWSCredentialProviderList providers
+        = new AWSCredentialProviderList();
+    assertEquals("Ref count for " + providers,
+        1, providers.getRefCount());
+    AWSCredentialProviderList replicate = providers.share();
+    assertEquals(providers, replicate);
+    assertEquals("Ref count after replication for " + providers,
+        2, providers.getRefCount());
+    assertFalse("Was closed " + providers, providers.isClosed());
+    providers.close();
+    assertFalse("Was closed " + providers, providers.isClosed());
+    assertEquals("Ref count after close() for " + providers,
+        1, providers.getRefCount());
+
+    // this should now close it
+    providers.close();
+    assertTrue("Was not closed " + providers, providers.isClosed());
+    assertEquals("Ref count after close() for " + providers,
+        0, providers.getRefCount());
+    assertEquals("Ref count after second close() for " + providers,
+        0, providers.getRefCount());
+    intercept(IllegalStateException.class, "closed",
+        () -> providers.share());
+    // final call harmless
+    providers.close();
+    assertEquals("Ref count after close() for " + providers,
+        0, providers.getRefCount());
+    providers.refresh();
+
+    intercept(NoAuthWithAWSException.class,
+        AWSCredentialProviderList.CREDENTIALS_REQUESTED_WHEN_CLOSED,
+        () -> providers.getCredentials());
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
index c6985b0..7451ef1 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
@@ -61,6 +61,7 @@ import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.*;
 import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
 import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
 import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden;
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 
 /**
@@ -85,6 +86,24 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
    */
   private S3AFileSystem roleFS;
 
+  /**
+   * Duration range exception text on SDKs which check client-side.
+   */
+  protected static final String E_DURATION_RANGE_1
+      = "Assume Role session duration should be in the range of 15min - 1Hr";
+
+  /**
+   * Duration range too high text on SDKs which check on the server.
+   */
+  protected static final String E_DURATION_RANGE_2
+      = "Member must have value less than or equal to 43200";
+
+  /**
+   * Duration range too low text on SDKs which check on the server.
+   */
+  protected static final String E_DURATION_RANGE_3
+      = "Member must have value greater than or equal to 900";
+
   @Override
   public void setup() throws Exception {
     super.setup();
@@ -112,13 +131,14 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
    * @param clazz class of exception to expect
    * @param text text in exception
    * @param <E> type of exception as inferred from clazz
+   * @return the caught exception if it was of the expected type and contents
    * @throws Exception if the exception was the wrong class
    */
-  private <E extends Throwable> void expectFileSystemCreateFailure(
+  private <E extends Throwable> E expectFileSystemCreateFailure(
       Configuration conf,
       Class<E> clazz,
       String text) throws Exception {
-    interceptClosing(clazz,
+    return interceptClosing(clazz,
         text,
         () -> new Path(getFileSystem().getUri()).getFileSystem(conf));
   }
@@ -246,6 +266,60 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
         "Member must satisfy regular expression pattern");
   }
 
+  /**
+   * A duration >1h is forbidden client-side in AWS SDK 1.11.271;
+   * with the ability to extend durations deployed in March 2018,
+   * duration checks will need to go server-side, and, presumably,
+   * later SDKs will remove the client side checks.
+   * This code exists to see when this happens.
+   */
+  @Test
+  public void testAssumeRoleThreeHourSessionDuration() throws Exception {
+    describe("Try to authenticate with a long session duration");
+
+    Configuration conf = createAssumedRoleConfig();
+    // add a duration of three hours
+    conf.setInt(ASSUMED_ROLE_SESSION_DURATION, 3 * 60 * 60);
+    try {
+      new Path(getFileSystem().getUri()).getFileSystem(conf).close();
+      LOG.info("Successfully created token of a duration >3h");
+    } catch (IOException ioe) {
+      assertExceptionContains(E_DURATION_RANGE_1, ioe);
+    }
+  }
+
+  /**
+   * A duration >1h is forbidden client-side in AWS SDK 1.11.271;
+   * with the ability to extend durations deployed in March 2018.
+   * with the later SDKs, the checks go server-side and
+   * later SDKs will remove the client side checks.
+   * This test asks for a duration which will still be rejected, and
+   * looks for either of the error messages raised.
+   */
+  @Test
+  public void testAssumeRoleThirtySixHourSessionDuration() throws Exception {
+    describe("Try to authenticate with a long session duration");
+
+    Configuration conf = createAssumedRoleConfig();
+    conf.setInt(ASSUMED_ROLE_SESSION_DURATION, 36 * 60 * 60);
+    IOException ioe = expectFileSystemCreateFailure(conf,
+        IOException.class, null);
+    assertIsRangeException(ioe);
+  }
+
+  /**
+   * Look for either the client-side or STS-side range exception
+   * @param e exception
+   * @throws Exception the exception, if its text doesn't match
+   */
+  private void assertIsRangeException(final Exception e) throws Exception {
+    String message = e.toString();
+    if (!message.contains(E_DURATION_RANGE_1)
+        && !message.contains(E_DURATION_RANGE_2)
+        && !message.contains(E_DURATION_RANGE_3)) {
+      throw e;
+    }
+  }
 
   /**
    * Create the assumed role configuration.
@@ -280,11 +354,11 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
     describe("Expect the constructor to fail if the session is to short");
     Configuration conf = new Configuration();
     conf.set(ASSUMED_ROLE_SESSION_DURATION, "30s");
-    interceptClosing(IllegalArgumentException.class, "",
+    Exception ex = interceptClosing(Exception.class, "",
         () -> new AssumedRoleCredentialProvider(uri, conf));
+    assertIsRangeException(ex);
   }
 
-
   @Test
   public void testAssumeRoleCreateFS() throws IOException {
     describe("Create an FS client with the role and do some basic IO");
@@ -296,24 +370,32 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
         conf.get(ACCESS_KEY), roleARN);
 
     try (FileSystem fs = path.getFileSystem(conf)) {
-      fs.getFileStatus(new Path("/"));
+      fs.getFileStatus(ROOT);
       fs.mkdirs(path("testAssumeRoleFS"));
     }
   }
 
   @Test
   public void testAssumeRoleRestrictedPolicyFS() throws Exception {
-    describe("Restrict the policy for this session; verify that reads fail");
+    describe("Restrict the policy for this session; verify that reads fail.");
 
+    // there's some special handling of S3Guard here as operations
+    // which only go to DDB don't fail the way S3 would reject them.
     Configuration conf = createAssumedRoleConfig();
     bindRolePolicy(conf, RESTRICTED_POLICY);
     Path path = new Path(getFileSystem().getUri());
+    boolean guarded = getFileSystem().hasMetadataStore();
     try (FileSystem fs = path.getFileSystem(conf)) {
-      forbidden("getFileStatus",
-          () -> fs.getFileStatus(new Path("/")));
-      forbidden("getFileStatus",
-          () -> fs.listStatus(new Path("/")));
-      forbidden("getFileStatus",
+      if (!guarded) {
+        // when S3Guard is enabled, the restricted policy still
+        // permits S3Guard record lookup, so getFileStatus calls
+        // will work iff the record is in the database.
+        forbidden("getFileStatus",
+            () -> fs.getFileStatus(ROOT));
+      }
+      forbidden("",
+          () -> fs.listStatus(ROOT));
+      forbidden("",
           () -> fs.mkdirs(path("testAssumeRoleFS")));
     }
   }
@@ -333,7 +415,11 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
     Configuration conf = createAssumedRoleConfig();
 
     bindRolePolicy(conf,
-        policy(statement(false, S3_ALL_BUCKETS, S3_GET_OBJECT_TORRENT)));
+        policy(
+            statement(false, S3_ALL_BUCKETS, S3_GET_OBJECT_TORRENT),
+            ALLOW_S3_GET_BUCKET_LOCATION,
+            STATEMENT_S3GUARD_CLIENT,
+            STATEMENT_ALLOW_SSE_KMS_RW));
     Path path = path("testAssumeRoleStillIncludesRolePerms");
     roleFS = (S3AFileSystem) path.getFileSystem(conf);
     assertTouchForbidden(roleFS, path);
@@ -342,6 +428,8 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
   /**
    * After blocking all write verbs used by S3A, try to write data (fail)
    * and read data (succeed).
+   * For S3Guard: full DDB RW access is retained.
+   * SSE-KMS key access is set to decrypt only.
    */
   @Test
   public void testReadOnlyOperations() throws Throwable {
@@ -352,7 +440,9 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
     bindRolePolicy(conf,
         policy(
             statement(false, S3_ALL_BUCKETS, S3_PATH_WRITE_OPERATIONS),
-            STATEMENT_ALL_S3, STATEMENT_ALL_DDB));
+            STATEMENT_ALL_S3,
+            STATEMENT_S3GUARD_CLIENT,
+            STATEMENT_ALLOW_SSE_KMS_READ));
     Path path = methodPath();
     roleFS = (S3AFileSystem) path.getFileSystem(conf);
     // list the root path, expect happy
@@ -399,8 +489,9 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
     Configuration conf = createAssumedRoleConfig();
 
     bindRolePolicyStatements(conf,
-        STATEMENT_ALL_DDB,
+        STATEMENT_S3GUARD_CLIENT,
         statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
+        STATEMENT_ALLOW_SSE_KMS_RW,
         new Statement(Effects.Allow)
           .addActions(S3_ALL_OPERATIONS)
           .addResources(directory(restrictedDir)));
@@ -447,7 +538,7 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
   }
 
   /**
-   * Execute a sequence of rename operations.
+   * Execute a sequence of rename operations with access locked down.
    * @param conf FS configuration
    */
   public void executeRestrictedRename(final Configuration conf)
@@ -461,7 +552,8 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
     fs.delete(basePath, true);
 
     bindRolePolicyStatements(conf,
-        STATEMENT_ALL_DDB,
+        STATEMENT_S3GUARD_CLIENT,
+        STATEMENT_ALLOW_SSE_KMS_RW,
         statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
         new Statement(Effects.Allow)
           .addActions(S3_PATH_RW_OPERATIONS)
@@ -503,6 +595,25 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
   }
 
   /**
+   * Without simulation of STS failures, and with STS overload likely to
+   * be very rare, there'll be no implicit test coverage of
+   * {@link AssumedRoleCredentialProvider#operationRetried(String, Exception, int, boolean)}.
+   * This test simply invokes the callback for both the first and second retry event.
+   *
+   * If the handler ever adds more than logging, this test ensures that things
+   * don't break.
+   */
+  @Test
+  public void testAssumedRoleRetryHandler() throws Throwable {
+    try(AssumedRoleCredentialProvider provider
+            = new AssumedRoleCredentialProvider(getFileSystem().getUri(),
+        createAssumedRoleConfig())) {
+      provider.operationRetried("retry", new IOException("failure"), 0, true);
+      provider.operationRetried("retry", new IOException("failure"), 1, true);
+    }
+  }
+
+  /**
    * Execute a sequence of rename operations where the source
    * data is read only to the client calling rename().
    * This will cause the inner delete() operations to fail, whose outcomes
@@ -534,7 +645,7 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
     touch(fs, readOnlyFile);
 
     bindRolePolicyStatements(conf,
-        STATEMENT_ALL_DDB,
+        STATEMENT_S3GUARD_CLIENT,
         statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
           new Statement(Effects.Allow)
             .addActions(S3_PATH_RW_OPERATIONS)
@@ -614,7 +725,8 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
     fs.mkdirs(readOnlyDir);
 
     bindRolePolicyStatements(conf,
-        STATEMENT_ALL_DDB,
+        STATEMENT_S3GUARD_CLIENT,
+        STATEMENT_ALLOW_SSE_KMS_RW,
         statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
         new Statement(Effects.Allow)
             .addActions(S3_PATH_RW_OPERATIONS)
@@ -752,7 +864,8 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
     fs.delete(destDir, true);
 
     bindRolePolicyStatements(conf,
-        STATEMENT_ALL_DDB,
+        STATEMENT_S3GUARD_CLIENT,
+        STATEMENT_ALLOW_SSE_KMS_RW,
         statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS),
         new Statement(Effects.Deny)
             .addActions(S3_PATH_WRITE_OPERATIONS)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java
index bb66268..834826e 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java
@@ -72,7 +72,8 @@ public class ITestAssumedRoleCommitOperations extends ITestCommitOperations {
     Configuration conf = newAssumedRoleConfig(getConfiguration(),
         getAssumedRoleARN());
     bindRolePolicyStatements(conf,
-        STATEMENT_ALL_DDB,
+        STATEMENT_S3GUARD_CLIENT,
+        STATEMENT_ALLOW_SSE_KMS_RW,
         statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
         new RoleModel.Statement(RoleModel.Effects.Allow)
             .addActions(S3_PATH_RW_OPERATIONS)
@@ -81,7 +82,6 @@ public class ITestAssumedRoleCommitOperations extends ITestCommitOperations {
     roleFS = (S3AFileSystem) restrictedDir.getFileSystem(conf);
   }
 
-
   @Override
   public void teardown() throws Exception {
     S3AUtils.closeAll(LOG, roleFS);
@@ -122,7 +122,6 @@ public class ITestAssumedRoleCommitOperations extends ITestCommitOperations {
     return new Path(restrictedDir, filepath);
   }
 
-
   private String getAssumedRoleARN() {
     return getContract().getConf().getTrimmed(ASSUMED_ROLE_ARN, "");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java
index 9fa2600..854e7ec 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java
@@ -58,14 +58,23 @@ public final class RoleTestUtils {
 
 
   /** Deny GET requests to all buckets. */
-  public static final Statement DENY_GET_ALL =
+  public static final Statement DENY_S3_GET_OBJECT =
       statement(false, S3_ALL_BUCKETS, S3_GET_OBJECT);
 
+  public static final Statement ALLOW_S3_GET_BUCKET_LOCATION
+      =  statement(true, S3_ALL_BUCKETS, S3_GET_BUCKET_LOCATION);
+
   /**
-   * This is AWS policy removes read access.
+   * This is AWS policy removes read access from S3, leaves S3Guard access up.
+   * This will allow clients to use S3Guard list/HEAD operations, even
+   * the ability to write records, but not actually access the underlying
+   * data.
+   * The client does need {@link RolePolicies#S3_GET_BUCKET_LOCATION} to
+   * get the bucket location.
    */
-  public static final Policy RESTRICTED_POLICY = policy(DENY_GET_ALL);
-
+  public static final Policy RESTRICTED_POLICY = policy(
+      DENY_S3_GET_OBJECT, STATEMENT_ALL_DDB, ALLOW_S3_GET_BUCKET_LOCATION
+      );
 
   /**
    * Error message to get from the AWS SDK if you can't assume the role.
@@ -145,7 +154,7 @@ public final class RoleTestUtils {
     Configuration conf = new Configuration(srcConf);
     conf.set(AWS_CREDENTIALS_PROVIDER, AssumedRoleCredentialProvider.NAME);
     conf.set(ASSUMED_ROLE_ARN, roleARN);
-    conf.set(ASSUMED_ROLE_SESSION_NAME, "valid");
+    conf.set(ASSUMED_ROLE_SESSION_NAME, "test");
     conf.set(ASSUMED_ROLE_SESSION_DURATION, "15m");
     disableFilesystemCaching(conf);
     return conf;
@@ -163,9 +172,8 @@ public final class RoleTestUtils {
       String contained,
       Callable<T> eval)
       throws Exception {
-    AccessDeniedException ex = intercept(AccessDeniedException.class, eval);
-    GenericTestUtils.assertExceptionContains(contained, ex);
-    return ex;
+    return intercept(AccessDeniedException.class,
+        contained, eval);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
index f591e32..9185fc5 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
@@ -32,6 +32,7 @@ import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.util.StopWatch;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.FileSystem;
@@ -51,6 +52,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.StringUtils;
 
+import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE;
 import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_NAME_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_METASTORE_NULL;
 import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
@@ -144,8 +146,11 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
 
     // Also create a "raw" fs without any MetadataStore configured
     Configuration conf = new Configuration(getConfiguration());
-    conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
     URI fsUri = getFileSystem().getUri();
+    conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
+    S3AUtils.setBucketOption(conf,fsUri.getHost(),
+        METADATASTORE_AUTHORITATIVE,
+        S3GUARD_METASTORE_NULL);
     rawFs = (S3AFileSystem) FileSystem.newInstance(fsUri, conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
index c6838a0..22a1efd 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
@@ -40,8 +40,10 @@ import org.junit.rules.Timeout;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
 import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
 import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
 
 import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY;
 
@@ -80,81 +82,102 @@ public class ITestS3GuardConcurrentOps extends AbstractS3ATestBase {
 
   @Test
   public void testConcurrentTableCreations() throws Exception {
-    final Configuration conf = getConfiguration();
+    S3AFileSystem fs = getFileSystem();
+    final Configuration conf = fs.getConf();
     Assume.assumeTrue("Test only applies when DynamoDB is used for S3Guard",
         conf.get(Constants.S3_METADATA_STORE_IMPL).equals(
             Constants.S3GUARD_METASTORE_DYNAMO));
 
-    DynamoDBMetadataStore ms = new DynamoDBMetadataStore();
-    ms.initialize(getFileSystem());
-    DynamoDB db = ms.getDynamoDB();
-
-    String tableName = "testConcurrentTableCreations" + new Random().nextInt();
-    conf.setBoolean(Constants.S3GUARD_DDB_TABLE_CREATE_KEY, true);
-    conf.set(Constants.S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    AWSCredentialProviderList sharedCreds =
+        fs.shareCredentials("testConcurrentTableCreations");
+    // close that shared copy.
+    sharedCreds.close();
+    // this is the original reference count.
+    int originalRefCount = sharedCreds.getRefCount();
 
-    String region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
-    if (StringUtils.isEmpty(region)) {
-      // no region set, so pick it up from the test bucket
-      conf.set(S3GUARD_DDB_REGION_KEY, getFileSystem().getBucketLocation());
-    }
-    int concurrentOps = 16;
-    int iterations = 4;
+    //now init the store; this should increment the ref count.
+    DynamoDBMetadataStore ms = new DynamoDBMetadataStore();
+    ms.initialize(fs);
 
-    failIfTableExists(db, tableName);
+    // the ref count should have gone up
+    assertEquals("Credential Ref count unchanged after initializing metastore "
+        + sharedCreds,
+        originalRefCount + 1, sharedCreds.getRefCount());
+    try {
+      DynamoDB db = ms.getDynamoDB();
 
-    for (int i = 0; i < iterations; i++) {
-      ExecutorService executor = Executors.newFixedThreadPool(
-          concurrentOps, new ThreadFactory() {
-            private AtomicInteger count = new AtomicInteger(0);
+      String tableName = "testConcurrentTableCreations" + new Random().nextInt();
+      conf.setBoolean(Constants.S3GUARD_DDB_TABLE_CREATE_KEY, true);
+      conf.set(Constants.S3GUARD_DDB_TABLE_NAME_KEY, tableName);
 
-            public Thread newThread(Runnable r) {
-              return new Thread(r,
-                  "testConcurrentTableCreations" + count.getAndIncrement());
+      String region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
+      if (StringUtils.isEmpty(region)) {
+        // no region set, so pick it up from the test bucket
+        conf.set(S3GUARD_DDB_REGION_KEY, fs.getBucketLocation());
+      }
+      int concurrentOps = 16;
+      int iterations = 4;
+
+      failIfTableExists(db, tableName);
+
+      for (int i = 0; i < iterations; i++) {
+        ExecutorService executor = Executors.newFixedThreadPool(
+            concurrentOps, new ThreadFactory() {
+              private AtomicInteger count = new AtomicInteger(0);
+
+              public Thread newThread(Runnable r) {
+                return new Thread(r,
+                    "testConcurrentTableCreations" + count.getAndIncrement());
+              }
+            });
+        ((ThreadPoolExecutor) executor).prestartAllCoreThreads();
+        Future<Exception>[] futures = new Future[concurrentOps];
+        for (int f = 0; f < concurrentOps; f++) {
+          final int index = f;
+          futures[f] = executor.submit(new Callable<Exception>() {
+            @Override
+            public Exception call() throws Exception {
+
+              ContractTestUtils.NanoTimer timer =
+                  new ContractTestUtils.NanoTimer();
+
+              Exception result = null;
+              try (DynamoDBMetadataStore store = new DynamoDBMetadataStore()) {
+                store.initialize(conf);
+              } catch (Exception e) {
+                LOG.error(e.getClass() + ": " + e.getMessage());
+                result = e;
+              }
+
+              timer.end("Parallel DynamoDB client creation %d", index);
+              LOG.info("Parallel DynamoDB client creation {} ran from {} to {}",
+                  index, timer.getStartTime(), timer.getEndTime());
+              return result;
             }
           });
-      ((ThreadPoolExecutor) executor).prestartAllCoreThreads();
-      Future<Exception>[] futures = new Future[concurrentOps];
-      for (int f = 0; f < concurrentOps; f++) {
-        final int index = f;
-        futures[f] = executor.submit(new Callable<Exception>() {
-          @Override
-          public Exception call() throws Exception {
-
-            ContractTestUtils.NanoTimer timer =
-                new ContractTestUtils.NanoTimer();
-
-            Exception result = null;
-            try (DynamoDBMetadataStore store = new DynamoDBMetadataStore()) {
-              store.initialize(conf);
-            } catch (Exception e) {
-              LOG.error(e.getClass() + ": " + e.getMessage());
-              result = e;
-            }
-
-            timer.end("Parallel DynamoDB client creation %d", index);
-            LOG.info("Parallel DynamoDB client creation {} ran from {} to {}",
-                index, timer.getStartTime(), timer.getEndTime());
-            return result;
+        }
+        List<Exception> exceptions = new ArrayList<>(concurrentOps);
+        for (int f = 0; f < concurrentOps; f++) {
+          Exception outcome = futures[f].get();
+          if (outcome != null) {
+            exceptions.add(outcome);
           }
-        });
-      }
-      List<Exception> exceptions = new ArrayList<>(concurrentOps);
-      for (int f = 0; f < concurrentOps; f++) {
-        Exception outcome = futures[f].get();
-        if (outcome != null) {
-          exceptions.add(outcome);
+        }
+        deleteTable(db, tableName);
+        int exceptionsThrown = exceptions.size();
+        if (exceptionsThrown > 0) {
+          // at least one exception was thrown. Fail the test & nest the first
+          // exception caught
+          throw new AssertionError(exceptionsThrown + "/" + concurrentOps +
+              " threads threw exceptions while initializing on iteration " + i,
+              exceptions.get(0));
         }
       }
-      deleteTable(db, tableName);
-      int exceptionsThrown = exceptions.size();
-      if (exceptionsThrown > 0) {
-        // at least one exception was thrown. Fail the test & nest the first
-        // exception caught
-        throw new AssertionError(exceptionsThrown + "/" + concurrentOps +
-            " threads threw exceptions while initializing on iteration " + i,
-            exceptions.get(0));
-      }
+    } finally {
+      ms.close();
     }
+    assertEquals("Credential Ref count unchanged after closing metastore: "
+            + sharedCreds,
+        originalRefCount, sharedCreds.getRefCount());
   }
 }


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


[44/50] [abbrv] hadoop git commit: HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit.

Posted by tm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
new file mode 100644
index 0000000..c17a5c1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -0,0 +1,402 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.UnsupportedEncodingException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * AbfsClient
+ */
+public class AbfsClient {
+  public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+  private final URL baseUrl;
+  private final SharedKeyCredentials sharedKeyCredentials;
+  private final String xMsVersion = "2018-03-28";
+  private final ExponentialRetryPolicy retryPolicy;
+  private final String filesystem;
+  private final ConfigurationService configurationService;
+  private final String userAgent;
+
+  public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
+                    final ConfigurationService configurationService,
+                    final ExponentialRetryPolicy exponentialRetryPolicy) {
+    this.baseUrl = baseUrl;
+    this.sharedKeyCredentials = sharedKeyCredentials;
+    String baseUrlString = baseUrl.toString();
+    this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(AbfsHttpConstants.FORWARD_SLASH) + 1);
+    this.configurationService = configurationService;
+    this.retryPolicy = exponentialRetryPolicy;
+    this.userAgent = initializeUserAgent();
+  }
+
+  public String getFileSystem() {
+    return filesystem;
+  }
+
+  ExponentialRetryPolicy getRetryPolicy() {
+    return retryPolicy;
+  }
+
+  SharedKeyCredentials getSharedKeyCredentials() {
+    return sharedKeyCredentials;
+  }
+
+  List<AbfsHttpHeader> createDefaultHeaders() {
+    final List<AbfsHttpHeader> requestHeaders = new ArrayList<AbfsHttpHeader>();
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_VERSION, xMsVersion));
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT, AbfsHttpConstants.APPLICATION_JSON
+            + AbfsHttpConstants.COMMA + AbfsHttpConstants.SINGLE_WHITE_SPACE + AbfsHttpConstants.APPLICATION_OCTET_STREAM));
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT_CHARSET,
+            AbfsHttpConstants.UTF_8));
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.CONTENT_TYPE, AbfsHttpConstants.EMPTY_STRING));
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.USER_AGENT, userAgent));
+    return requestHeaders;
+  }
+
+  AbfsUriQueryBuilder createDefaultUriQueryBuilder() {
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_TIMEOUT, AbfsHttpConstants.DEFAULT_TIMEOUT);
+    return abfsUriQueryBuilder;
+  }
+
+  public AbfsRestOperation createFilesystem() throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation setFilesystemProperties(final String properties) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
+            AbfsHttpConstants.HTTP_METHOD_PATCH));
+
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES,
+            properties));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation listPath(final String relativePath, final boolean recursive, final int listMaxResults,
+                                    final String continuation) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath));
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation);
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults));
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_GET,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation getFilesystemProperties() throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_HEAD,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation deleteFilesystem() throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_DELETE,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite)
+          throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    if (!overwrite) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, "*"));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, isFile ? AbfsHttpConstants.FILE : AbfsHttpConstants.DIRECTORY);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation renamePath(final String source, final String destination, final String continuation)
+          throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final String encodedRenameSource = urlEncode(AbfsHttpConstants.FORWARD_SLASH + this.getFileSystem() + source);
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_RENAME_SOURCE, encodedRenameSource));
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.STAR));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation);
+
+    final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset,
+                                  final int length) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
+            AbfsHttpConstants.HTTP_METHOD_PATCH));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.APPEND_ACTION);
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position));
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_PUT,
+            url,
+            requestHeaders, buffer, offset, length);
+    op.execute();
+    return op;
+  }
+
+
+  public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
+            AbfsHttpConstants.HTTP_METHOD_PATCH));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.FLUSH_ACTION);
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position));
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData));
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation setPathProperties(final String path, final String properties) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
+            AbfsHttpConstants.HTTP_METHOD_PATCH));
+
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES, properties));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_PROPERTIES_ACTION);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation getPathProperties(final String path) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_HEAD,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset,
+                                final int bufferLength, final String eTag) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.RANGE,
+            String.format("bytes=%d-%d", position, position + bufferLength - 1)));
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_GET,
+            url,
+            requestHeaders,
+            buffer,
+            bufferOffset,
+            bufferLength);
+    op.execute();
+
+    return op;
+  }
+
+  public AbfsRestOperation deletePath(final String path, final boolean recursive, final String continuation)
+          throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            this,
+            AbfsHttpConstants.HTTP_METHOD_DELETE,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  private URL createRequestUrl(final String query) throws AzureBlobFileSystemException {
+    return createRequestUrl(AbfsHttpConstants.EMPTY_STRING, query);
+  }
+
+  private URL createRequestUrl(final String path, final String query)
+          throws AzureBlobFileSystemException {
+    final String base = baseUrl.toString();
+    String encodedPath = path;
+    try {
+      encodedPath = urlEncode(path);
+    } catch (AzureBlobFileSystemException ex) {
+      this.LOG.debug(
+              "Unexpected error.", ex);
+    }
+
+    final StringBuilder sb = new StringBuilder();
+    sb.append(base);
+    sb.append(encodedPath);
+    sb.append(query);
+
+    final URL url;
+    try {
+      url = new URL(sb.toString());
+    } catch (MalformedURLException ex) {
+      throw new InvalidUriException(sb.toString());
+    }
+    return url;
+  }
+
+  private static String urlEncode(final String value) throws AzureBlobFileSystemException {
+    String encodedString = null;
+    try {
+      encodedString =  URLEncoder.encode(value, AbfsHttpConstants.UTF_8)
+          .replace(AbfsHttpConstants.PLUS, AbfsHttpConstants.PLUS_ENCODE)
+          .replace(AbfsHttpConstants.FORWARD_SLASH_ENCODE, AbfsHttpConstants.FORWARD_SLASH);
+    } catch (UnsupportedEncodingException ex) {
+        throw new InvalidUriException(value);
+    }
+
+    return encodedString;
+  }
+
+  private String initializeUserAgent() {
+    final String userAgentComment = String.format(Locale.ROOT,
+            "(JavaJRE %s; %s %s)",
+            System.getProperty(AbfsHttpConstants.JAVA_VERSION),
+            System.getProperty(AbfsHttpConstants.OS_NAME)
+                    .replaceAll(AbfsHttpConstants.SINGLE_WHITE_SPACE, AbfsHttpConstants.EMPTY_STRING),
+            System.getProperty(AbfsHttpConstants.OS_VERSION));
+
+    return String.format(AbfsHttpConstants.CLIENT_VERSION + " %s", userAgentComment);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java
new file mode 100644
index 0000000..9e4c27b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.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.azurebfs.services;
+
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URL;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+
+@Singleton
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+class AbfsHttpClientFactoryImpl implements AbfsHttpClientFactory {
+  private final ConfigurationService configurationService;
+
+  @Inject
+  AbfsHttpClientFactoryImpl(
+      final ConfigurationService configurationService) {
+
+    Preconditions.checkNotNull(configurationService, "configurationService");
+
+    this.configurationService = configurationService;
+  }
+
+  @VisibleForTesting
+  URIBuilder getURIBuilder(final String hostName, final FileSystem fs) {
+    final AzureBlobFileSystem abfs = (AzureBlobFileSystem) fs;
+
+    String scheme = FileSystemUriSchemes.HTTP_SCHEME;
+
+    if (abfs.isSecure()) {
+      scheme = FileSystemUriSchemes.HTTPS_SCHEME;
+    }
+
+    final URIBuilder uriBuilder = new URIBuilder();
+    uriBuilder.setScheme(scheme);
+    uriBuilder.setHost(hostName);
+
+    return uriBuilder;
+  }
+
+  public AbfsClient create(final AzureBlobFileSystem fs) throws AzureBlobFileSystemException {
+    final URI uri = fs.getUri();
+    final String authority = uri.getRawAuthority();
+    if (null == authority) {
+      throw new InvalidUriAuthorityException(uri.toString());
+    }
+
+    if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) {
+      throw new InvalidUriAuthorityException(uri.toString());
+    }
+
+    final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
+
+    if (authorityParts.length < 2 || "".equals(authorityParts[0])) {
+      final String errMsg = String
+          .format("URI '%s' has a malformed authority, expected container name. "
+                  + "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
+              uri.toString());
+      throw new InvalidUriException(errMsg);
+    }
+
+    final String fileSystemName = authorityParts[0];
+    final String accountName = authorityParts[1];
+
+    final URIBuilder uriBuilder = getURIBuilder(accountName, fs);
+
+    final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName;
+
+    URL baseUrl;
+    try {
+      baseUrl = new URL(url);
+    } catch (MalformedURLException e) {
+      throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString()));
+    }
+
+    SharedKeyCredentials creds =
+        new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)),
+                this.configurationService.getStorageAccountKey(accountName));
+
+    return new AbfsClient(baseUrl, creds, configurationService, new ExponentialRetryPolicy());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java
new file mode 100644
index 0000000..46b4c6d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+/**
+ * The Http Request / Response Headers for Rest AbfsClient
+ */
+public class AbfsHttpHeader {
+  private final String name;
+  private final String value;
+
+  public AbfsHttpHeader(final String name, final String value) {
+    this.name = name;
+    this.value = value;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getValue() {
+    return value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
new file mode 100644
index 0000000..0ea9365
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.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.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.List;
+import java.util.UUID;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represents an HTTP operation.
+ */
+public class AbfsHttpOperation {
+  private static final Logger LOG = LoggerFactory.getLogger(AbfsHttpOperation.class);
+
+  private static final int CONNECT_TIMEOUT = 30 * 1000;
+  private static final int READ_TIMEOUT = 30 * 1000;
+
+  private static final int CLEAN_UP_BUFFER_SIZE = 64 * 1024;
+
+  private static final int ONE_THOUSAND = 1000;
+  private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND;
+
+  private final String method;
+  private final URL url;
+
+  private HttpURLConnection connection;
+  private int statusCode;
+  private String statusDescription;
+  private String storageErrorCode = "";
+  private String storageErrorMessage  = "";
+  private String clientRequestId = "";
+  private String requestId  = "";
+  private ListResultSchema listResultSchema = null;
+
+  // metrics
+  private int bytesSent;
+  private long bytesReceived;
+
+  // optional trace enabled metrics
+  private final boolean isTraceEnabled;
+  private long connectionTimeMs;
+  private long sendRequestTimeMs;
+  private long recvResponseTimeMs;
+
+  protected  HttpURLConnection getConnection() {
+    return connection;
+  }
+
+  public String getMethod() {
+    return method;
+  }
+
+  public URL getUrl() {
+    return url;
+  }
+
+  public int getStatusCode() {
+    return statusCode;
+  }
+
+  public String getStatusDescription() {
+    return statusDescription;
+  }
+
+  public String getStorageErrorCode() {
+    return storageErrorCode;
+  }
+
+  public String getStorageErrorMessage() {
+    return storageErrorMessage;
+  }
+
+  public String getClientRequestId() {
+    return clientRequestId;
+  }
+
+  public String getRequestId() {
+    return requestId;
+  }
+
+  public int getBytesSent() {
+    return bytesSent;
+  }
+
+  public long getBytesReceived() {
+    return bytesReceived;
+  }
+
+  public ListResultSchema getListResultSchema() {
+    return listResultSchema;
+  }
+
+  public String getResponseHeader(String httpHeader) {
+    return connection.getHeaderField(httpHeader);
+  }
+
+  // Returns a trace message for the request
+  @Override
+  public String toString() {
+    final String urlStr = url.toString();
+    final StringBuilder sb = new StringBuilder();
+    sb.append(statusCode);
+    sb.append(",");
+    sb.append(storageErrorCode);
+    sb.append(",cid=");
+    sb.append(clientRequestId);
+    sb.append(",rid=");
+    sb.append(requestId);
+    if (isTraceEnabled) {
+      sb.append(",connMs=");
+      sb.append(connectionTimeMs);
+      sb.append(",sendMs=");
+      sb.append(sendRequestTimeMs);
+      sb.append(",recvMs=");
+      sb.append(recvResponseTimeMs);
+    }
+    sb.append(",sent=");
+    sb.append(bytesSent);
+    sb.append(",recv=");
+    sb.append(bytesReceived);
+    sb.append(",");
+    sb.append(method);
+    sb.append(",");
+    sb.append(urlStr);
+    return sb.toString();
+  }
+
+  /**
+   * Initializes a new HTTP request and opens the connection.
+   *
+   * @param url The full URL including query string parameters.
+   * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
+   * @param requestHeaders The HTTP request headers.READ_TIMEOUT
+   *
+   * @throws IOException if an error occurs.
+   */
+  public AbfsHttpOperation(final URL url, final String method, final List<AbfsHttpHeader> requestHeaders)
+      throws IOException {
+    this.isTraceEnabled = this.LOG.isTraceEnabled();
+    this.url = url;
+    this.method = method;
+    this.clientRequestId = UUID.randomUUID().toString();
+
+    this.connection = openConnection();
+
+    this.connection.setConnectTimeout(CONNECT_TIMEOUT);
+    this.connection.setReadTimeout(READ_TIMEOUT);
+
+    this.connection.setRequestMethod(method);
+
+    for (AbfsHttpHeader header : requestHeaders) {
+      this.connection.setRequestProperty(header.getName(), header.getValue());
+    }
+
+    this.connection.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, clientRequestId);
+  }
+
+  /**
+   * Sends the HTTP request.  Note that HttpUrlConnection requires that an
+   * empty buffer be sent in order to set the "Content-Length: 0" header, which
+   * is required by our endpoint.
+   *
+   * @param buffer the request entity body.
+   * @param offset an offset into the buffer where the data beings.
+   * @param length the length of the data in the buffer.
+   *
+   * @throws IOException if an error occurs.
+   */
+  public void sendRequest(byte[] buffer, int offset, int length) throws IOException {
+    this.connection.setDoOutput(true);
+    this.connection.setFixedLengthStreamingMode(length);
+    if (buffer == null) {
+      // An empty buffer is sent to set the "Content-Length: 0" header, which
+      // is required by our endpoint.
+      buffer = new byte[]{};
+      offset = 0;
+      length = 0;
+    }
+
+    // send the request body
+
+    long startTime = 0;
+    if (this.isTraceEnabled) {
+      startTime = System.nanoTime();
+    }
+    try (OutputStream outputStream = this.connection.getOutputStream()) {
+      // update bytes sent before they are sent so we may observe
+      // attempted sends as well as successful sends via the
+      // accompanying statusCode
+      this.bytesSent = length;
+      outputStream.write(buffer, offset, length);
+    } finally {
+      if (this.isTraceEnabled) {
+        this.sendRequestTimeMs = elapsedTimeMs(startTime);
+      }
+    }
+  }
+
+  /**
+   * Gets and processes the HTTP response.
+   *
+   * @throws IOException if an error occurs.
+   */
+  public void processResponse(final byte[] buffer, final int offset, final int length) throws IOException {
+
+    // get the response
+    long startTime = 0;
+    if (this.isTraceEnabled) {
+      startTime = System.nanoTime();
+    }
+
+    this.statusCode = this.connection.getResponseCode();
+
+    if (this.isTraceEnabled) {
+      this.recvResponseTimeMs = elapsedTimeMs(startTime);
+    }
+
+    this.statusDescription = this.connection.getResponseMessage();
+
+    this.requestId = this.connection.getHeaderField(HttpHeaderConfigurations.X_MS_REQUEST_ID);
+    if (this.requestId == null) {
+      this.requestId = AbfsHttpConstants.EMPTY_STRING;
+    }
+
+    if (AbfsHttpConstants.HTTP_METHOD_HEAD.equals(this.method)) {
+      // If it is HEAD, and it is ERROR
+      return;
+    }
+
+    if (this.isTraceEnabled) {
+      startTime = System.nanoTime();
+    }
+
+    if (statusCode >= HttpURLConnection.HTTP_BAD_REQUEST) {
+      processStorageErrorResponse();
+      if (this.isTraceEnabled) {
+        this.recvResponseTimeMs += elapsedTimeMs(startTime);
+      }
+      this.bytesReceived = this.connection.getHeaderFieldLong(HttpHeaderConfigurations.CONTENT_LENGTH, 0);
+    } else {
+      // consume the input stream to release resources
+      int totalBytesRead = 0;
+
+      try (InputStream stream = this.connection.getInputStream()) {
+        if (isNullInputStream(stream)) {
+          return;
+        }
+        boolean endOfStream = false;
+
+        // this is a list operation and need to retrieve the data
+        // need a better solution
+        if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method) && buffer == null) {
+          parseListFilesResponse(stream);
+        } else {
+          if (buffer != null) {
+            while (totalBytesRead < length) {
+              int bytesRead = stream.read(buffer, offset + totalBytesRead, length - totalBytesRead);
+              if (bytesRead == -1) {
+                endOfStream = true;
+                break;
+              }
+              totalBytesRead += bytesRead;
+            }
+          }
+          if (!endOfStream && stream.read() != -1) {
+            // read and discard
+            int bytesRead = 0;
+            byte[] b = new byte[CLEAN_UP_BUFFER_SIZE];
+            while ((bytesRead = stream.read(b)) >= 0) {
+              totalBytesRead += bytesRead;
+            }
+          }
+        }
+      } catch (IOException ex) {
+        this.LOG.error("UnexpectedError: ", ex);
+        throw ex;
+      } finally {
+        if (this.isTraceEnabled) {
+          this.recvResponseTimeMs += elapsedTimeMs(startTime);
+        }
+        this.bytesReceived = totalBytesRead;
+      }
+    }
+  }
+
+
+  /**
+   * Open the HTTP connection.
+   *
+   * @throws IOException if an error occurs.
+   */
+  private HttpURLConnection openConnection() throws IOException {
+    if (!isTraceEnabled) {
+      return (HttpURLConnection) url.openConnection();
+    }
+    long start = System.nanoTime();
+    try {
+      return (HttpURLConnection) url.openConnection();
+    } finally {
+      connectionTimeMs = elapsedTimeMs(start);
+    }
+  }
+
+  /**
+   * When the request fails, this function is used to parse the responseAbfsHttpClient.LOG.debug("ExpectedError: ", ex);
+   * and extract the storageErrorCode and storageErrorMessage.  Any errors
+   * encountered while attempting to process the error response are logged,
+   * but otherwise ignored.
+   *
+   * For storage errors, the response body *usually* has the following format:
+   *
+   * {
+   *   "error":
+   *   {
+   *     "code": "string",
+   *     "message": "string"
+   *   }
+   * }
+   *
+   */
+  private void processStorageErrorResponse() {
+    try (InputStream stream = connection.getErrorStream()) {
+      if (stream == null) {
+        return;
+      }
+      JsonFactory jf = new JsonFactory();
+      try (JsonParser jp = jf.createParser(stream)) {
+        String fieldName, fieldValue;
+        jp.nextToken();  // START_OBJECT - {
+        jp.nextToken();  // FIELD_NAME - "error":
+        jp.nextToken();  // START_OBJECT - {
+        jp.nextToken();
+        while (jp.hasCurrentToken()) {
+          if (jp.getCurrentToken() == JsonToken.FIELD_NAME) {
+            fieldName = jp.getCurrentName();
+            jp.nextToken();
+            fieldValue = jp.getText();
+            switch (fieldName) {
+              case "code":
+                storageErrorCode = fieldValue;
+                break;
+              case "message":
+                storageErrorMessage = fieldValue;
+                break;
+              default:
+                break;
+            }
+          }
+          jp.nextToken();
+        }
+      }
+    } catch (IOException ex) {
+      // Ignore errors that occur while attempting to parse the storage
+      // error, since the response may have been handled by the HTTP driver
+      // or for other reasons have an unexpected
+      this.LOG.debug("ExpectedError: ", ex);
+    }
+  }
+
+  /**
+   * Returns the elapsed time in milliseconds.
+   */
+  private long elapsedTimeMs(final long startTime) {
+    return (System.nanoTime() - startTime) / ONE_MILLION;
+  }
+
+  /**
+   * Parse the list file response
+   *
+   * @param stream InputStream contains the list results.
+   * @throws IOException
+   */
+  private void parseListFilesResponse(final InputStream stream) throws IOException {
+    if (stream == null) {
+      return;
+    }
+
+    if (listResultSchema != null) {
+      // already parse the response
+      return;
+    }
+
+    try {
+      final ObjectMapper objectMapper = new ObjectMapper();
+      this.listResultSchema = objectMapper.readValue(stream, ListResultSchema.class);
+    } catch (IOException ex) {
+      this.LOG.error("Unable to deserialize list results", ex);
+      throw ex;
+    }
+  }
+
+  /**
+   * Check null stream, this is to pass findbugs's redundant check for NULL
+   * @param stream InputStream
+   */
+  private boolean isNullInputStream(InputStream stream) {
+    return stream == null ? true : false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java
new file mode 100644
index 0000000..06e1a8a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java
@@ -0,0 +1,693 @@
+/**
+ * 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.azurebfs.services;
+
+import javax.xml.bind.DatatypeConverter;
+import java.io.File;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CharsetEncoder;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Hashtable;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.google.common.base.Preconditions;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
+import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.util.Time.now;
+
+@Singleton
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+final class AbfsHttpServiceImpl implements AbfsHttpService {
+  public static final Logger LOG = LoggerFactory.getLogger(AbfsHttpService.class);
+  private static final String DATE_TIME_PATTERN = "E, dd MMM yyyy HH:mm:ss 'GMT'";
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int LIST_MAX_RESULTS = 5000;
+  private static final int DELETE_DIRECTORY_TIMEOUT_MILISECONDS = 180000;
+  private static final int RENAME_TIMEOUT_MILISECONDS = 180000;
+
+  private final AbfsHttpClientFactory abfsHttpClientFactory;
+  private final ConcurrentHashMap<AzureBlobFileSystem, AbfsClient> clientCache;
+  private final ConfigurationService configurationService;
+  private final Set<String> azureAtomicRenameDirSet;
+
+  @Inject
+  AbfsHttpServiceImpl(
+      final ConfigurationService configurationService,
+      final AbfsHttpClientFactory abfsHttpClientFactory,
+      final TracingService tracingService) {
+    Preconditions.checkNotNull(abfsHttpClientFactory, "abfsHttpClientFactory");
+    Preconditions.checkNotNull(configurationService, "configurationService");
+    Preconditions.checkNotNull(tracingService, "tracingService");
+
+    this.configurationService = configurationService;
+    this.clientCache = new ConcurrentHashMap<>();
+    this.abfsHttpClientFactory = abfsHttpClientFactory;
+    this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(configurationService.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
+  }
+
+  @Override
+  public Hashtable<String, String> getFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem)
+      throws AzureBlobFileSystemException{
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "getFilesystemProperties for filesystem: {}",
+        client.getFileSystem());
+
+    final Hashtable<String, String> parsedXmsProperties;
+
+    final AbfsRestOperation op = client.getFilesystemProperties();
+    final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
+
+    parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
+
+    return parsedXmsProperties;
+  }
+
+  @Override
+  public void setFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem, final Hashtable<String, String> properties) throws
+      AzureBlobFileSystemException {
+    if (properties == null || properties.size() == 0) {
+      return;
+    }
+
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "setFilesystemProperties for filesystem: {} with properties: {}",
+        client.getFileSystem(),
+        properties);
+
+    final String commaSeparatedProperties;
+    try {
+      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
+    } catch (CharacterCodingException ex) {
+      throw new InvalidAbfsRestOperationException(ex);
+    }
+    client.setFilesystemProperties(commaSeparatedProperties);
+  }
+
+  @Override
+  public Hashtable<String, String> getPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws
+      AzureBlobFileSystemException {
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "getPathProperties for filesystem: {} path: {}",
+        client.getFileSystem(),
+        path.toString());
+
+    final Hashtable<String, String> parsedXmsProperties;
+    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+    final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
+
+    parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
+
+    return parsedXmsProperties;
+  }
+
+  @Override
+  public void setPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final Hashtable<String,
+      String> properties) throws
+      AzureBlobFileSystemException {
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "setFilesystemProperties for filesystem: {} path: {} with properties: {}",
+        client.getFileSystem(),
+        path.toString(),
+        properties);
+
+    final String commaSeparatedProperties;
+    try {
+      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
+    } catch (CharacterCodingException ex) {
+      throw new InvalidAbfsRestOperationException(ex);
+    }
+    client.setPathProperties("/" + getRelativePath(path), commaSeparatedProperties);
+  }
+
+  @Override
+  public void createFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "createFilesystem for filesystem: {}",
+        client.getFileSystem());
+
+    client.createFilesystem();
+  }
+
+  @Override
+  public void deleteFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "deleteFilesystem for filesystem: {}",
+        client.getFileSystem());
+
+    client.deleteFilesystem();
+  }
+
+  @Override
+  public OutputStream createFile(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws
+      AzureBlobFileSystemException {
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "createFile filesystem: {} path: {} overwrite: {}",
+        client.getFileSystem(),
+        path.toString(),
+        overwrite);
+
+    client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite);
+
+    final OutputStream outputStream;
+    outputStream = new FSDataOutputStream(
+        new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0,
+            configurationService.getWriteBufferSize()), null);
+    return outputStream;
+  }
+
+  @Override
+  public Void createDirectory(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "createDirectory filesystem: {} path: {} overwrite: {}",
+        client.getFileSystem(),
+        path.toString());
+
+    client.createPath("/" + getRelativePath(path), false, true);
+
+    return null;
+  }
+
+  @Override
+  public InputStream openFileForRead(final AzureBlobFileSystem azureBlobFileSystem, final Path path,
+      final FileSystem.Statistics statistics) throws AzureBlobFileSystemException {
+    final AbfsClient client = getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "openFileForRead filesystem: {} path: {}",
+        client.getFileSystem(),
+        path.toString());
+
+    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+    final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+    final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
+    final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+
+    if (parseIsDirectory(resourceType)) {
+      throw new AbfsRestOperationException(
+          AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+          AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+          "openFileForRead must be used with files and not directories",
+          null);
+    }
+
+    // Add statistics for InputStream
+    return new FSDataInputStream(
+        new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
+            configurationService.getReadBufferSize(), configurationService.getReadAheadQueueDepth(), eTag));
+  }
+
+  @Override
+  public OutputStream openFileForWrite(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws
+      AzureBlobFileSystemException {
+    final AbfsClient client = getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "openFileForWrite filesystem: {} path: {} overwrite: {}",
+        client.getFileSystem(),
+        path.toString(),
+        overwrite);
+
+    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+    final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+    final Long contentLength = Long.valueOf(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
+
+    if (parseIsDirectory(resourceType)) {
+      throw new AbfsRestOperationException(
+          AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+          AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+          "openFileForRead must be used with files and not directories",
+          null);
+    }
+
+    final long offset = overwrite ? 0 : contentLength;
+
+    final OutputStream outputStream;
+    outputStream = new FSDataOutputStream(
+        new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
+            offset, configurationService.getWriteBufferSize()), null);
+    return outputStream;
+  }
+
+  @Override
+  public void rename(final AzureBlobFileSystem azureBlobFileSystem, final Path source, final Path destination) throws
+      AzureBlobFileSystemException {
+
+    if (isAtomicRenameKey(source.getName())) {
+      this.LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename,"
+          +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account.");
+    }
+
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "renameAsync filesystem: {} source: {} destination: {}",
+        client.getFileSystem(),
+        source.toString(),
+        destination.toString());
+
+    String continuation = null;
+    long deadline = now() + RENAME_TIMEOUT_MILISECONDS;
+
+    do {
+      if (now() > deadline) {
+        LOG.debug(
+            "Rename {} to {} timed out.",
+            source,
+            destination);
+
+        throw new TimeoutException("Rename timed out.");
+      }
+
+      AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source),
+          AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation);
+      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+
+    } while (continuation != null && !continuation.isEmpty());
+  }
+
+  @Override
+  public void delete(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean recursive) throws
+      AzureBlobFileSystemException {
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "delete filesystem: {} path: {} recursive: {}",
+        client.getFileSystem(),
+        path.toString(),
+        String.valueOf(recursive));
+
+    String continuation = null;
+    long deadline = now() + DELETE_DIRECTORY_TIMEOUT_MILISECONDS;
+
+    do {
+      if (now() > deadline) {
+        this.LOG.debug(
+            "Delete directory {} timed out.", path);
+
+        throw new TimeoutException("Delete directory timed out.");
+      }
+
+      AbfsRestOperation op = client.deletePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation);
+      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+
+    } while (continuation != null && !continuation.isEmpty());
+  }
+
+  @Override
+  public FileStatus getFileStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "getFileStatus filesystem: {} path: {}",
+        client.getFileSystem(),
+        path.toString());
+
+    if (path.isRoot()) {
+      AbfsRestOperation op = client.getFilesystemProperties();
+      final long blockSize = configurationService.getAzureBlockSize();
+      final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+      final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
+      return new VersionedFileStatus(
+          azureBlobFileSystem.getOwnerUser(),
+          azureBlobFileSystem.getOwnerUserPrimaryGroup(),
+          0,
+          true,
+          1,
+          blockSize,
+          parseLastModifiedTime(lastModified).getMillis(),
+          path,
+          eTag);
+    } else {
+      AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+      final long blockSize = configurationService.getAzureBlockSize();
+      final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+      final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
+      final String contentLength = op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH);
+      final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+
+      return new VersionedFileStatus(
+          azureBlobFileSystem.getOwnerUser(),
+          azureBlobFileSystem.getOwnerUserPrimaryGroup(),
+          parseContentLength(contentLength),
+          parseIsDirectory(resourceType),
+          1,
+          blockSize,
+          parseLastModifiedTime(lastModified).getMillis(),
+          path,
+          eTag);
+    }
+  }
+
+  @Override
+  public FileStatus[] listStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
+    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+
+    this.LOG.debug(
+        "listStatus filesystem: {} path: {}",
+        client.getFileSystem(),
+        path.toString());
+
+    String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path);
+    String continuation = null;
+    ArrayList<FileStatus> fileStatuses = new ArrayList<>();
+
+    do {
+      AbfsRestOperation op = client.listPath(relativePath, false, LIST_MAX_RESULTS, continuation);
+      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+      ListResultSchema retrievedSchema = op.getResult().getListResultSchema();
+      if (retrievedSchema == null) {
+        throw new AbfsRestOperationException(
+            AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+            AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+            "listStatusAsync path not found",
+            null, op.getResult());
+      }
+
+      long blockSize = configurationService.getAzureBlockSize();
+
+      for (ListResultEntrySchema entry : retrievedSchema.paths()) {
+        long lastModifiedMillis = 0;
+        long contentLength = entry.contentLength() == null ? 0 : entry.contentLength();
+        boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory();
+        if (entry.lastModified() != null && !entry.lastModified().isEmpty()) {
+          final DateTime dateTime = DateTime.parse(
+              entry.lastModified(),
+              DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
+          lastModifiedMillis = dateTime.getMillis();
+        }
+
+        fileStatuses.add(
+            new VersionedFileStatus(
+                azureBlobFileSystem.getOwnerUser(),
+                azureBlobFileSystem.getOwnerUserPrimaryGroup(),
+                contentLength,
+                isDirectory,
+                1,
+                blockSize,
+                lastModifiedMillis,
+                azureBlobFileSystem.makeQualified(new Path(File.separator + entry.name())),
+                entry.eTag()));
+      }
+
+    } while (continuation != null && !continuation.isEmpty());
+
+    return fileStatuses.toArray(new FileStatus[0]);
+  }
+
+  @Override
+  public synchronized void closeFileSystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
+    this.clientCache.remove(azureBlobFileSystem);
+  }
+
+  @Override
+  public boolean isAtomicRenameKey(String key) {
+    return isKeyForDirectorySet(key, azureAtomicRenameDirSet);
+  }
+
+  private String getRelativePath(final Path path) {
+    Preconditions.checkNotNull(path, "path");
+    final String relativePath = path.toUri().getPath();
+
+    if (relativePath.length() == 0) {
+      return relativePath;
+    }
+
+    if (relativePath.charAt(0) == Path.SEPARATOR_CHAR) {
+      if (relativePath.length() == 1) {
+        return AbfsHttpConstants.EMPTY_STRING;
+      }
+
+      return relativePath.substring(1);
+    }
+
+    return relativePath;
+  }
+
+  private synchronized AbfsClient getOrCreateClient(final AzureBlobFileSystem azureBlobFileSystem) throws
+      AzureBlobFileSystemException {
+    Preconditions.checkNotNull(azureBlobFileSystem, "azureBlobFileSystem");
+
+    AbfsClient client = this.clientCache.get(azureBlobFileSystem);
+
+    if (client != null) {
+      return client;
+    }
+
+    client = abfsHttpClientFactory.create(azureBlobFileSystem);
+    this.clientCache.put(
+        azureBlobFileSystem,
+        client);
+    return client;
+  }
+
+  private long parseContentLength(final String contentLength) {
+    if (contentLength == null) {
+      return -1;
+    }
+
+    return Long.parseLong(contentLength);
+  }
+
+  private boolean parseIsDirectory(final String resourceType) {
+    return resourceType == null ? false : resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
+  }
+
+  private DateTime parseLastModifiedTime(final String lastModifiedTime) {
+    return DateTime.parse(
+        lastModifiedTime,
+        DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
+  }
+
+  private String convertXmsPropertiesToCommaSeparatedString(final Hashtable<String, String> properties) throws
+      CharacterCodingException {
+    StringBuilder commaSeparatedProperties = new StringBuilder();
+
+    final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING).newEncoder();
+
+    for (Map.Entry<String, String> propertyEntry : properties.entrySet()) {
+      String key = propertyEntry.getKey();
+      String value = propertyEntry.getValue();
+
+      Boolean canEncodeValue = encoder.canEncode(value);
+      if (!canEncodeValue) {
+        throw new CharacterCodingException();
+      }
+
+      String encodedPropertyValue = DatatypeConverter.printBase64Binary(encoder.encode(CharBuffer.wrap(value)).array());
+      commaSeparatedProperties.append(key)
+              .append(AbfsHttpConstants.EQUAL)
+              .append(encodedPropertyValue);
+
+      commaSeparatedProperties.append(AbfsHttpConstants.COMMA);
+    }
+
+    if (commaSeparatedProperties.length() != 0) {
+      commaSeparatedProperties.deleteCharAt(commaSeparatedProperties.length() - 1);
+    }
+
+    return commaSeparatedProperties.toString();
+  }
+
+  private Hashtable<String, String> parseCommaSeparatedXmsProperties(String xMsProperties) throws
+      InvalidFileSystemPropertyException, InvalidAbfsRestOperationException {
+    Hashtable<String, String> properties = new Hashtable<>();
+
+    final CharsetDecoder decoder = Charset.forName(XMS_PROPERTIES_ENCODING).newDecoder();
+
+    if (xMsProperties != null && !xMsProperties.isEmpty()) {
+      String[] userProperties = xMsProperties.split(AbfsHttpConstants.COMMA);
+
+      if (userProperties.length == 0) {
+        return properties;
+      }
+
+      for (String property : userProperties) {
+        if (property.isEmpty()) {
+          throw new InvalidFileSystemPropertyException(xMsProperties);
+        }
+
+        String[] nameValue = property.split(AbfsHttpConstants.EQUAL, 2);
+        if (nameValue.length != 2) {
+          throw new InvalidFileSystemPropertyException(xMsProperties);
+        }
+
+        byte[] decodedValue = DatatypeConverter.parseBase64Binary(nameValue[1]);
+
+        final String value;
+        try {
+          value = decoder.decode(ByteBuffer.wrap(decodedValue)).toString();
+        } catch (CharacterCodingException ex) {
+          throw new InvalidAbfsRestOperationException(ex);
+        }
+        properties.put(nameValue[0], value);
+      }
+    }
+
+    return properties;
+  }
+
+  private boolean isKeyForDirectorySet(String key, Set<String> dirSet) {
+    for (String dir : dirSet) {
+      if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) {
+        return true;
+      }
+
+      try {
+        URI uri = new URI(dir);
+        if (null == uri.getAuthority()) {
+          if (key.startsWith(dir + "/")){
+            return true;
+          }
+        }
+      } catch (URISyntaxException e) {
+        this.LOG.info("URI syntax error creating URI for {}", dir);
+      }
+    }
+
+    return false;
+  }
+
+  private static class VersionedFileStatus extends FileStatus {
+    private final String version;
+
+    VersionedFileStatus(
+        final String owner, final String group,
+        final long length, final boolean isdir, final int blockReplication,
+        final long blocksize, final long modificationTime, final Path path,
+        String version) {
+      super(length, isdir, blockReplication, blocksize, modificationTime, 0,
+          new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL),
+          owner,
+          group,
+          path);
+
+      this.version = version;
+    }
+
+    /** Compare if this object is equal to another object.
+     * @param   obj the object to be compared.
+     * @return  true if two file status has the same path name; false if not.
+     */
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      }
+
+      if (obj == null) {
+        return false;
+      }
+
+      if (this.getClass() == obj.getClass()) {
+        VersionedFileStatus other = (VersionedFileStatus) obj;
+        return this.getPath().equals(other.getPath()) && this.version.equals(other.version);
+      }
+
+      return false;
+    }
+
+    /**
+     * Returns a hash code value for the object, which is defined as
+     * the hash code of the path name.
+     *
+     * @return  a hash code value for the path name and version
+     */
+    @Override
+    public int hashCode() {
+      int hash = getPath().hashCode();
+      hash = 89 * hash + (this.version != null ? this.version.hashCode() : 0);
+      return hash;
+    }
+
+    /**
+     * Returns the version of this FileStatus
+     *
+     * @return  a string value for the FileStatus version
+     */
+    public String getVersion() {
+      return this.version;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdf5f4c3/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
new file mode 100644
index 0000000..6554380
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
@@ -0,0 +1,382 @@
+/**
+ * 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.azurebfs.services;
+
+import java.io.EOFException;
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+
+/**
+ * The AbfsInputStream for AbfsClient
+ */
+public class AbfsInputStream extends FSInputStream {
+  private final AbfsClient client;
+  private final Statistics statistics;
+  private final String path;
+  private final long contentLength;
+  private final int bufferSize; // default buffer size
+  private final int readAheadQueueDepth;         // initialized in constructor
+  private final String eTag;                  // eTag of the path when InputStream are created
+  private final boolean tolerateOobAppends; // whether tolerate Oob Appends
+  private final boolean readAheadEnabled; // whether enable readAhead;
+
+  private byte[] buffer = null;            // will be initialized on first use
+
+  private long fCursor = 0;  // cursor of buffer within file - offset of next byte to read from remote server
+  private long fCursorAfterLastRead = -1;
+  private int bCursor = 0;   // cursor of read within buffer - offset of next byte to be returned from buffer
+  private int limit = 0;     // offset of next byte to be read into buffer from service (i.e., upper marker+1
+  //                                                      of valid bytes in buffer)
+  private boolean closed = false;
+
+  public AbfsInputStream(
+      final AbfsClient client,
+      final Statistics statistics,
+      final String path,
+      final long contentLength,
+      final int bufferSize,
+      final int readAheadQueueDepth,
+      final String eTag) {
+    super();
+    this.client = client;
+    this.statistics = statistics;
+    this.path = path;
+    this.contentLength = contentLength;
+    this.bufferSize = bufferSize;
+    this.readAheadQueueDepth = (readAheadQueueDepth >= 0) ? readAheadQueueDepth : 2 * Runtime.getRuntime().availableProcessors();
+    this.eTag = eTag;
+    this.tolerateOobAppends = false;
+    this.readAheadEnabled = true;
+  }
+
+  public String getPath() {
+    return path;
+  }
+
+  @Override
+  public int read() throws IOException {
+    byte[] b = new byte[1];
+    int numberOfBytesRead = read(b, 0, 1);
+    if (numberOfBytesRead < 0) {
+      return -1;
+    } else {
+      return (b[0] & 0xFF);
+    }
+  }
+
+  @Override
+  public synchronized int read(final byte[] b, final int off, final int len) throws IOException {
+    int currentOff = off;
+    int currentLen = len;
+    int lastReadBytes;
+    int totalReadBytes = 0;
+    do {
+      lastReadBytes = readOneBlock(b, currentOff, currentLen);
+      if (lastReadBytes > 0) {
+        currentOff += lastReadBytes;
+        currentLen -= lastReadBytes;
+        totalReadBytes += lastReadBytes;
+      }
+      if (currentLen <= 0 || currentLen > b.length - currentOff) {
+        break;
+      }
+    } while (lastReadBytes > 0);
+    return totalReadBytes > 0 ? totalReadBytes : lastReadBytes;
+  }
+
+  private int readOneBlock(final byte[] b, final int off, final int len) throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+
+    Preconditions.checkNotNull(b);
+
+    if (len == 0) {
+      return 0;
+    }
+
+    if (this.available() == 0) {
+      return -1;
+    }
+
+    if (off < 0 || len < 0 || len > b.length - off) {
+      throw new IndexOutOfBoundsException();
+    }
+
+    //If buffer is empty, then fill the buffer.
+    if (bCursor == limit) {
+      //If EOF, then return -1
+      if (fCursor >= contentLength) {
+        return -1;
+      }
+
+      long bytesRead = 0;
+      //reset buffer to initial state - i.e., throw away existing data
+      bCursor = 0;
+      limit = 0;
+      if (buffer == null) {
+        buffer = new byte[bufferSize];
+      }
+
+      // Enable readAhead when reading sequentially
+      if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) {
+        bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false);
+      } else {
+        bytesRead = readInternal(fCursor, buffer, 0, b.length, true);
+      }
+
+      if (bytesRead == -1) {
+        return -1;
+      }
+
+      limit += bytesRead;
+      fCursor += bytesRead;
+      fCursorAfterLastRead = fCursor;
+    }
+
+    //If there is anything in the buffer, then return lesser of (requested bytes) and (bytes in buffer)
+    //(bytes returned may be less than requested)
+    int bytesRemaining = limit - bCursor;
+    int bytesToRead = Math.min(len, bytesRemaining);
+    System.arraycopy(buffer, bCursor, b, off, bytesToRead);
+    bCursor += bytesToRead;
+    if (statistics != null) {
+      statistics.incrementBytesRead(bytesToRead);
+    }
+    return bytesToRead;
+  }
+
+
+  private int readInternal(final long position, final byte[] b, final int offset, final int length,
+                           final boolean bypassReadAhead) throws IOException {
+    if (readAheadEnabled && !bypassReadAhead) {
+      // try reading from read-ahead
+      if (offset != 0) {
+        throw new IllegalArgumentException("readahead buffers cannot have non-zero buffer offsets");
+      }
+      int receivedBytes;
+
+      // queue read-aheads
+      int numReadAheads = this.readAheadQueueDepth;
+      long nextSize;
+      long nextOffset = position;
+      while (numReadAheads > 0 && nextOffset < contentLength) {
+        nextSize = Math.min((long) bufferSize, contentLength - nextOffset);
+        ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize);
+        nextOffset = nextOffset + nextSize;
+        numReadAheads--;
+      }
+
+      // try reading from buffers first
+      receivedBytes = ReadBufferManager.getBufferManager().getBlock(this, position, length, b);
+      if (receivedBytes > 0) {
+        return receivedBytes;
+      }
+
+      // got nothing from read-ahead, do our own read now
+      receivedBytes = readRemote(position, b, offset, length);
+      return receivedBytes;
+    } else {
+      return readRemote(position, b, offset, length);
+    }
+  }
+
+  int readRemote(long position, byte[] b, int offset, int length) throws IOException {
+    if (position < 0) {
+      throw new IllegalArgumentException("attempting to read from negative offset");
+    }
+    if (position >= contentLength) {
+      return -1;  // Hadoop prefers -1 to EOFException
+    }
+    if (b == null) {
+      throw new IllegalArgumentException("null byte array passed in to read() method");
+    }
+    if (offset >= b.length) {
+      throw new IllegalArgumentException("offset greater than length of array");
+    }
+    if (length < 0) {
+      throw new IllegalArgumentException("requested read length is less than zero");
+    }
+    if (length > (b.length - offset)) {
+      throw new IllegalArgumentException("requested read length is more than will fit after requested offset in buffer");
+    }
+    final AbfsRestOperation op;
+    try {
+      op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag);
+    } catch (AzureBlobFileSystemException ex) {
+      throw new IOException(ex);
+    }
+    long bytesRead = op.getResult().getBytesReceived();
+    if (bytesRead > Integer.MAX_VALUE) {
+      throw new IOException("Unexpected Content-Length");
+    }
+    return (int) bytesRead;
+  }
+
+  /**
+   * Seek to given position in stream.
+   * @param n position to seek to
+   * @throws IOException if there is an error
+   * @throws EOFException if attempting to seek past end of file
+   */
+  @Override
+  public synchronized void seek(long n) throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    if (n < 0) {
+      throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
+    }
+    if (n > contentLength) {
+      throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+    }
+
+    if (n>=fCursor-limit && n<=fCursor) { // within buffer
+      bCursor = (int) (n-(fCursor-limit));
+      return;
+    }
+
+    // next read will read from here
+    fCursor = n;
+
+    //invalidate buffer
+    limit = 0;
+    bCursor = 0;
+  }
+
+  @Override
+  public synchronized long skip(long n) throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    long currentPos = getPos();
+    if (currentPos == contentLength) {
+      if (n > 0) {
+        throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+      }
+    }
+    long newPos = currentPos + n;
+    if (newPos < 0) {
+      newPos = 0;
+      n = newPos - currentPos;
+    }
+    if (newPos > contentLength) {
+      newPos = contentLength;
+      n = newPos - currentPos;
+    }
+    seek(newPos);
+    return n;
+  }
+
+  /**
+   * Return the size of the remaining available bytes
+   * if the size is less than or equal to {@link Integer#MAX_VALUE},
+   * otherwise, return {@link Integer#MAX_VALUE}.
+   *
+   * This is to match the behavior of DFSInputStream.available(),
+   * which some clients may rely on (HBase write-ahead log reading in
+   * particular).
+   */
+  @Override
+  public synchronized int available() throws IOException {
+    if (closed) {
+      throw new IOException(
+          FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    final long remaining = this.contentLength - this.getPos();
+    return remaining <= Integer.MAX_VALUE
+        ? (int) remaining : Integer.MAX_VALUE;
+  }
+
+  /**
+   * Returns the length of the file that this stream refers to. Note that the length returned is the length
+   * as of the time the Stream was opened. Specifically, if there have been subsequent appends to the file,
+   * they wont be reflected in the returned length.
+   *
+   * @return length of the file.
+   * @throws IOException if the stream is closed
+   */
+  public long length() throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    return contentLength;
+  }
+
+  /**
+   * Return the current offset from the start of the file
+   * @throws IOException throws {@link IOException} if there is an error
+   */
+  @Override
+  public synchronized long getPos() throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    return fCursor - limit + bCursor;
+  }
+
+  /**
+   * Seeks a different copy of the data.  Returns true if
+   * found a new source, false otherwise.
+   * @throws IOException throws {@link IOException} if there is an error
+   */
+  @Override
+  public boolean seekToNewSource(long l) throws IOException {
+    return false;
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    closed = true;
+    buffer = null; // de-reference the buffer so it can be GC'ed sooner
+  }
+
+  /**
+   * Not supported by this stream. Throws {@link UnsupportedOperationException}
+   * @param readlimit ignored
+   */
+  @Override
+  public synchronized void mark(int readlimit) {
+    throw new UnsupportedOperationException("mark()/reset() not supported on this stream");
+  }
+
+  /**
+   * Not supported by this stream. Throws {@link UnsupportedOperationException}
+   */
+  @Override
+  public synchronized void reset() throws IOException {
+    throw new UnsupportedOperationException("mark()/reset() not supported on this stream");
+  }
+
+  /**
+   * gets whether mark and reset are supported by {@code ADLFileInputStream}. Always returns false.
+   *
+   * @return always {@code false}
+   */
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+}
\ 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


[13/50] [abbrv] hadoop git commit: HADOOP-15583. Stabilize S3A Assumed Role support. Contributed by Steve Loughran.

Posted by tm...@apache.org.
HADOOP-15583. Stabilize S3A Assumed Role support.
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/da9a39ee
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/da9a39ee
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/da9a39ee

Branch: refs/heads/HADOOP-15407
Commit: da9a39eed138210de29b59b90c449b28da1c04f9
Parents: d81cd36
Author: Steve Loughran <st...@apache.org>
Authored: Wed Aug 8 22:57:10 2018 -0700
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Aug 8 22:57:24 2018 -0700

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |  18 +-
 .../fs/s3a/AWSCredentialProviderList.java       | 101 ++++++--
 .../org/apache/hadoop/fs/s3a/Constants.java     |  19 +-
 .../hadoop/fs/s3a/DefaultS3ClientFactory.java   | 190 ++++----------
 .../fs/s3a/InconsistentAmazonS3Client.java      |  10 +
 .../fs/s3a/InconsistentS3ClientFactory.java     |  11 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  35 ++-
 .../apache/hadoop/fs/s3a/S3ARetryPolicy.java    |   4 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java | 245 +++++++++++++++++--
 .../apache/hadoop/fs/s3a/S3ClientFactory.java   |   7 +-
 .../s3a/auth/AssumedRoleCredentialProvider.java |  78 +++++-
 .../fs/s3a/auth/NoAuthWithAWSException.java     |  37 +++
 .../apache/hadoop/fs/s3a/auth/RoleModel.java    |   8 +
 .../apache/hadoop/fs/s3a/auth/RolePolicies.java | 143 +++++++++--
 .../hadoop/fs/s3a/auth/STSClientFactory.java    |  78 ++++++
 .../fs/s3a/s3guard/DynamoDBClientFactory.java   |  18 +-
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java   |  62 ++++-
 .../markdown/tools/hadoop-aws/assumed_roles.md  | 191 +++++++++++----
 .../src/site/markdown/tools/hadoop-aws/index.md |   6 +-
 .../hadoop/fs/s3a/ITestS3AConfiguration.java    | 117 ++++-----
 .../fs/s3a/ITestS3ATemporaryCredentials.java    |  71 +++---
 .../fs/s3a/ITestS3GuardListConsistency.java     |  68 +++--
 .../hadoop/fs/s3a/ITestS3GuardWriteBack.java    |  57 +++--
 .../hadoop/fs/s3a/MockS3ClientFactory.java      |   6 +-
 .../fs/s3a/TestS3AAWSCredentialsProvider.java   |  76 +++++-
 .../hadoop/fs/s3a/auth/ITestAssumeRole.java     | 151 ++++++++++--
 .../auth/ITestAssumedRoleCommitOperations.java  |   5 +-
 .../hadoop/fs/s3a/auth/RoleTestUtils.java       |  24 +-
 .../s3guard/AbstractS3GuardToolTestBase.java    |   7 +-
 .../s3a/s3guard/ITestS3GuardConcurrentOps.java  | 147 ++++++-----
 30 files changed, 1461 insertions(+), 529 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 75acf48..29c2bc2 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1033,7 +1033,19 @@
   <name>fs.s3a.assumed.role.sts.endpoint</name>
   <value/>
   <description>
-    AWS Simple Token Service Endpoint. If unset, uses the default endpoint.
+    AWS Security Token Service Endpoint.
+    If unset, uses the default endpoint.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.sts.endpoint.region</name>
+  <value>us-west-1</value>
+  <description>
+    AWS Security Token Service Endpoint's region;
+    Needed if fs.s3a.assumed.role.sts.endpoint points to an endpoint
+    other than the default one and the v4 signature is used.
     Only used if AssumedRoleCredentialProvider is the AWS credential provider.
   </description>
 </property>
@@ -1058,7 +1070,9 @@
 <property>
   <name>fs.s3a.connection.ssl.enabled</name>
   <value>true</value>
-  <description>Enables or disables SSL connections to S3.</description>
+  <description>Enables or disables SSL connections to AWS services.
+    Also sets the default port to use for the s3a proxy settings,
+    when not explicitly set in fs.s3a.proxy.port.</description>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java
index 10201f0..f9052fa 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java
@@ -18,25 +18,29 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.AnonymousAWSCredentials;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
 import org.apache.hadoop.io.IOUtils;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.stream.Collectors;
-
 /**
  * A list of providers.
  *
@@ -62,10 +66,18 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
   public static final String NO_AWS_CREDENTIAL_PROVIDERS
       = "No AWS Credential Providers";
 
+  static final String
+      CREDENTIALS_REQUESTED_WHEN_CLOSED
+      = "Credentials requested after provider list was closed";
+
   private final List<AWSCredentialsProvider> providers = new ArrayList<>(1);
   private boolean reuseLastProvider = true;
   private AWSCredentialsProvider lastProvider;
 
+  private final AtomicInteger refCount = new AtomicInteger(1);
+
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
   /**
    * Empty instance. This is not ready to be used.
    */
@@ -94,6 +106,9 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
    */
   @Override
   public void refresh() {
+    if (isClosed()) {
+      return;
+    }
     for (AWSCredentialsProvider provider : providers) {
       provider.refresh();
     }
@@ -106,6 +121,11 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
    */
   @Override
   public AWSCredentials getCredentials() {
+    if (isClosed()) {
+      LOG.warn(CREDENTIALS_REQUESTED_WHEN_CLOSED);
+      throw new NoAuthWithAWSException(
+          CREDENTIALS_REQUESTED_WHEN_CLOSED);
+    }
     checkNotEmpty();
     if (reuseLastProvider && lastProvider != null) {
       return lastProvider.getCredentials();
@@ -136,8 +156,7 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
     if (lastException != null) {
       message += ": " + lastException;
     }
-    throw new AmazonClientException(message, lastException);
-
+    throw new NoAuthWithAWSException(message, lastException);
   }
 
   /**
@@ -156,7 +175,7 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
    */
   public void checkNotEmpty() {
     if (providers.isEmpty()) {
-      throw new AmazonClientException(NO_AWS_CREDENTIAL_PROVIDERS);
+      throw new NoAuthWithAWSException(NO_AWS_CREDENTIAL_PROVIDERS);
     }
   }
 
@@ -178,8 +197,38 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
    */
   @Override
   public String toString() {
-    return "AWSCredentialProviderList: " +
-        StringUtils.join(providers, " ");
+    return "AWSCredentialProviderList[" +
+        "refcount= " + refCount.get() + ": [" +
+        StringUtils.join(providers, ", ") + ']';
+  }
+
+  /**
+   * Get a reference to this object with an updated reference count.
+   *
+   * @return a reference to this
+   */
+  public synchronized AWSCredentialProviderList share() {
+    Preconditions.checkState(!closed.get(), "Provider list is closed");
+    refCount.incrementAndGet();
+    return this;
+  }
+
+  /**
+   * Get the current reference count.
+   * @return the current ref count
+   */
+  @VisibleForTesting
+  public int getRefCount() {
+    return refCount.get();
+  }
+
+  /**
+   * Get the closed flag.
+   * @return true iff the list is closed.
+   */
+  @VisibleForTesting
+  public boolean isClosed() {
+    return closed.get();
   }
 
   /**
@@ -190,9 +239,29 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
    */
   @Override
   public void close() {
-    for(AWSCredentialsProvider p: providers) {
+    synchronized (this) {
+      if (closed.get()) {
+        // already closed: no-op
+        return;
+      }
+      int remainder = refCount.decrementAndGet();
+      if (remainder != 0) {
+        // still actively used, or somehow things are
+        // now negative
+        LOG.debug("Not closing {}", this);
+        return;
+      }
+      // at this point, the closing is going to happen
+      LOG.debug("Closing {}", this);
+      closed.set(true);
+    }
+
+    // do this outside the synchronized block.
+    for (AWSCredentialsProvider p : providers) {
       if (p instanceof Closeable) {
-        IOUtils.closeStream((Closeable)p);
+        IOUtils.closeStream((Closeable) p);
+      } else if (p instanceof AutoCloseable) {
+        S3AUtils.closeAutocloseables(LOG, (AutoCloseable)p);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index c521936..a8da6ec 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -84,10 +84,27 @@ public final class Constants {
   public static final String ASSUMED_ROLE_SESSION_DURATION =
       "fs.s3a.assumed.role.session.duration";
 
-  /** Simple Token Service Endpoint. If unset, uses the default endpoint. */
+  /** Security Token Service Endpoint. If unset, uses the default endpoint. */
   public static final String ASSUMED_ROLE_STS_ENDPOINT =
       "fs.s3a.assumed.role.sts.endpoint";
 
+  /**
+   * Region for the STS endpoint; only relevant if the endpoint
+   * is set.
+   */
+  public static final String ASSUMED_ROLE_STS_ENDPOINT_REGION =
+      "fs.s3a.assumed.role.sts.endpoint.region";
+
+  /**
+   * Default value for the STS endpoint region; needed for
+   * v4 signing.
+   */
+  public static final String ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT =
+      "us-west-1";
+
+  /**
+   * Default duration of an assumed role.
+   */
   public static final String ASSUMED_ROLE_SESSION_DURATION_DEFAULT = "30m";
 
   /** list of providers to authenticate for the assumed role. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
index f33b25e..ade317f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
@@ -18,59 +18,45 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import java.io.IOException;
+import java.net.URI;
+
 import com.amazonaws.ClientConfiguration;
-import com.amazonaws.Protocol;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.AmazonS3Client;
 import com.amazonaws.services.s3.S3ClientOptions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.util.VersionInfo;
 import org.slf4j.Logger;
 
-import java.io.IOException;
-import java.net.URI;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
 
-import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
-import static org.apache.hadoop.fs.s3a.S3AUtils.intOption;
+import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
+import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
 
 /**
- * The default factory implementation, which calls the AWS SDK to configure
- * and create an {@link AmazonS3Client} that communicates with the S3 service.
+ * The default {@link S3ClientFactory} implementation.
+ * This which calls the AWS SDK to configure and create an
+ * {@link AmazonS3Client} that communicates with the S3 service.
  */
-public class DefaultS3ClientFactory extends Configured implements
-    S3ClientFactory {
+public class DefaultS3ClientFactory extends Configured
+    implements S3ClientFactory {
 
   protected static final Logger LOG = S3AFileSystem.LOG;
 
   @Override
-  public AmazonS3 createS3Client(URI name) throws IOException {
+  public AmazonS3 createS3Client(URI name,
+      final String bucket,
+      final AWSCredentialsProvider credentials) throws IOException {
     Configuration conf = getConf();
-    AWSCredentialsProvider credentials =
-        createAWSCredentialProviderSet(name, conf);
-    final ClientConfiguration awsConf = createAwsConf(getConf());
-    AmazonS3 s3 = newAmazonS3Client(credentials, awsConf);
-    return createAmazonS3Client(s3, conf, credentials, awsConf);
+    final ClientConfiguration awsConf = S3AUtils.createAwsConf(getConf(), bucket);
+    return configureAmazonS3Client(
+        newAmazonS3Client(credentials, awsConf), conf);
   }
 
   /**
-   * Create a new {@link ClientConfiguration}.
-   * @param conf The Hadoop configuration
-   * @return new AWS client configuration
-   */
-  public static ClientConfiguration createAwsConf(Configuration conf) {
-    final ClientConfiguration awsConf = new ClientConfiguration();
-    initConnectionSettings(conf, awsConf);
-    initProxySupport(conf, awsConf);
-    initUserAgent(conf, awsConf);
-    return awsConf;
-  }
-
-  /**
-   * Wrapper around constructor for {@link AmazonS3} client.  Override this to
-   * provide an extended version of the client
+   * Wrapper around constructor for {@link AmazonS3} client.
+   * Override this to provide an extended version of the client
    * @param credentials credentials to use
    * @param awsConf  AWS configuration
    * @return  new AmazonS3 client
@@ -81,120 +67,17 @@ public class DefaultS3ClientFactory extends Configured implements
   }
 
   /**
-   * Initializes all AWS SDK settings related to connection management.
-   *
-   * @param conf Hadoop configuration
-   * @param awsConf AWS SDK configuration
-   */
-  private static void initConnectionSettings(Configuration conf,
-      ClientConfiguration awsConf) {
-    awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
-        DEFAULT_MAXIMUM_CONNECTIONS, 1));
-    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
-        DEFAULT_SECURE_CONNECTIONS);
-    awsConf.setProtocol(secureConnections ?  Protocol.HTTPS : Protocol.HTTP);
-    awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
-        DEFAULT_MAX_ERROR_RETRIES, 0));
-    awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
-        DEFAULT_ESTABLISH_TIMEOUT, 0));
-    awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
-        DEFAULT_SOCKET_TIMEOUT, 0));
-    int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER,
-        DEFAULT_SOCKET_SEND_BUFFER, 2048);
-    int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER,
-        DEFAULT_SOCKET_RECV_BUFFER, 2048);
-    awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer);
-    String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
-    if (!signerOverride.isEmpty()) {
-      LOG.debug("Signer override = {}", signerOverride);
-      awsConf.setSignerOverride(signerOverride);
-    }
-  }
-
-  /**
-   * Initializes AWS SDK proxy support if configured.
-   *
-   * @param conf Hadoop configuration
-   * @param awsConf AWS SDK configuration
-   * @throws IllegalArgumentException if misconfigured
-   */
-  private static void initProxySupport(Configuration conf,
-      ClientConfiguration awsConf) throws IllegalArgumentException {
-    String proxyHost = conf.getTrimmed(PROXY_HOST, "");
-    int proxyPort = conf.getInt(PROXY_PORT, -1);
-    if (!proxyHost.isEmpty()) {
-      awsConf.setProxyHost(proxyHost);
-      if (proxyPort >= 0) {
-        awsConf.setProxyPort(proxyPort);
-      } else {
-        if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
-          LOG.warn("Proxy host set without port. Using HTTPS default 443");
-          awsConf.setProxyPort(443);
-        } else {
-          LOG.warn("Proxy host set without port. Using HTTP default 80");
-          awsConf.setProxyPort(80);
-        }
-      }
-      String proxyUsername = conf.getTrimmed(PROXY_USERNAME);
-      String proxyPassword = conf.getTrimmed(PROXY_PASSWORD);
-      if ((proxyUsername == null) != (proxyPassword == null)) {
-        String msg = "Proxy error: " + PROXY_USERNAME + " or " +
-            PROXY_PASSWORD + " set without the other.";
-        LOG.error(msg);
-        throw new IllegalArgumentException(msg);
-      }
-      awsConf.setProxyUsername(proxyUsername);
-      awsConf.setProxyPassword(proxyPassword);
-      awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN));
-      awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Using proxy server {}:{} as user {} with password {} on " +
-                "domain {} as workstation {}", awsConf.getProxyHost(),
-            awsConf.getProxyPort(),
-            String.valueOf(awsConf.getProxyUsername()),
-            awsConf.getProxyPassword(), awsConf.getProxyDomain(),
-            awsConf.getProxyWorkstation());
-      }
-    } else if (proxyPort >= 0) {
-      String msg =
-          "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
-      LOG.error(msg);
-      throw new IllegalArgumentException(msg);
-    }
-  }
-
-  /**
-   * Initializes the User-Agent header to send in HTTP requests to the S3
-   * back-end.  We always include the Hadoop version number.  The user also
-   * may set an optional custom prefix to put in front of the Hadoop version
-   * number.  The AWS SDK interally appends its own information, which seems
-   * to include the AWS SDK version, OS and JVM version.
+   * Configure S3 client from the Hadoop configuration.
    *
-   * @param conf Hadoop configuration
-   * @param awsConf AWS SDK configuration
-   */
-  private static void initUserAgent(Configuration conf,
-      ClientConfiguration awsConf) {
-    String userAgent = "Hadoop " + VersionInfo.getVersion();
-    String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, "");
-    if (!userAgentPrefix.isEmpty()) {
-      userAgent = userAgentPrefix + ", " + userAgent;
-    }
-    LOG.debug("Using User-Agent: {}", userAgent);
-    awsConf.setUserAgentPrefix(userAgent);
-  }
-
-  /**
-   * Creates an {@link AmazonS3Client} from the established configuration.
+   * This includes: endpoint, Path Access and possibly other
+   * options.
    *
    * @param conf Hadoop configuration
-   * @param credentials AWS credentials
-   * @param awsConf AWS SDK configuration
    * @return S3 client
    * @throws IllegalArgumentException if misconfigured
    */
-  private static AmazonS3 createAmazonS3Client(AmazonS3 s3, Configuration conf,
-      AWSCredentialsProvider credentials, ClientConfiguration awsConf)
+  private static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
+      Configuration conf)
       throws IllegalArgumentException {
     String endPoint = conf.getTrimmed(ENDPOINT, "");
     if (!endPoint.isEmpty()) {
@@ -206,21 +89,29 @@ public class DefaultS3ClientFactory extends Configured implements
         throw new IllegalArgumentException(msg, e);
       }
     }
-    enablePathStyleAccessIfRequired(s3, conf);
-    return s3;
+    return applyS3ClientOptions(s3, conf);
   }
 
   /**
-   * Enables path-style access to S3 buckets if configured.  By default, the
+   * Perform any tuning of the {@code S3ClientOptions} settings based on
+   * the Hadoop configuration.
+   * This is different from the general AWS configuration creation as
+   * it is unique to S3 connections.
+   *
+   * The {@link Constants#PATH_STYLE_ACCESS} option enables path-style access
+   * to S3 buckets if configured.  By default, the
    * behavior is to use virtual hosted-style access with URIs of the form
-   * http://bucketname.s3.amazonaws.com.  Enabling path-style access and a
+   * {@code http://bucketname.s3.amazonaws.com}
+   * Enabling path-style access and a
    * region-specific endpoint switches the behavior to use URIs of the form
-   * http://s3-eu-west-1.amazonaws.com/bucketname.
-   *
+   * {@code http://s3-eu-west-1.amazonaws.com/bucketname}.
+   * It is common to use this when connecting to private S3 servers, as it
+   * avoids the need to play with DNS entries.
    * @param s3 S3 client
    * @param conf Hadoop configuration
+   * @return the S3 client
    */
-  private static void enablePathStyleAccessIfRequired(AmazonS3 s3,
+  private static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
       Configuration conf) {
     final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
     if (pathStyleAccess) {
@@ -229,5 +120,6 @@ public class DefaultS3ClientFactory extends Configured implements
           .setPathStyleAccess(true)
           .build());
     }
+    return s3;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
index 99ed87d..2cd1aae 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
@@ -114,6 +114,16 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
   /** Map of key to delay -> time it was created. */
   private Map<String, Long> delayedPutKeys = new HashMap<>();
 
+  /**
+   * Instantiate.
+   * This subclasses a deprecated constructor of the parent
+   * {@code AmazonS3Client} class; we can't use the builder API because,
+   * that only creates the consistent client.
+   * @param credentials credentials to auth.
+   * @param clientConfiguration connection settings
+   * @param conf hadoop configuration.
+   */
+  @SuppressWarnings("deprecation")
   public InconsistentAmazonS3Client(AWSCredentialsProvider credentials,
       ClientConfiguration clientConfiguration, Configuration conf) {
     super(credentials, clientConfiguration);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
index 17d268b..932c472 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
@@ -21,16 +21,27 @@ package org.apache.hadoop.fs.s3a;
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.services.s3.AmazonS3;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
  * S3 Client factory used for testing with eventual consistency fault injection.
+ * This client is for testing <i>only</i>; it is in the production
+ * {@code hadoop-aws} module to enable integration tests to use this
+ * just by editing the Hadoop configuration used to bring up the client.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class InconsistentS3ClientFactory extends DefaultS3ClientFactory {
 
+  /**
+   * Create the inconsistent client.
+   * Logs a warning that this is being done.
+   * @param credentials credentials to use
+   * @param awsConf  AWS configuration
+   * @return an inconsistent client.
+   */
   @Override
   protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials,
       ClientConfiguration awsConf) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 737d7da..72a5fde 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -77,8 +77,9 @@ import com.amazonaws.event.ProgressListener;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.ListeningExecutorService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -124,9 +125,6 @@ import static org.apache.hadoop.fs.s3a.Statistic.*;
 import static org.apache.commons.lang3.StringUtils.isNotBlank;
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * The core S3A Filesystem implementation.
  *
@@ -205,6 +203,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   private boolean useListV1;
   private MagicCommitIntegration committerIntegration;
 
+  private AWSCredentialProviderList credentials;
+
   /** Add any deprecated keys. */
   @SuppressWarnings("deprecation")
   private static void addDeprecatedKeys() {
@@ -252,8 +252,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       Class<? extends S3ClientFactory> s3ClientFactoryClass = conf.getClass(
           S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
           S3ClientFactory.class);
+
+      credentials = createAWSCredentialProviderSet(name, conf);
       s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
-          .createS3Client(name);
+          .createS3Client(name, bucket, credentials);
       invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
       s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()),
           onRetry);
@@ -2470,12 +2472,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
         transfers.shutdownNow(true);
         transfers = null;
       }
-      if (metadataStore != null) {
-        metadataStore.close();
-        metadataStore = null;
-      }
-      IOUtils.closeQuietly(instrumentation);
+      S3AUtils.closeAll(LOG, metadataStore, instrumentation);
+      metadataStore = null;
       instrumentation = null;
+      closeAutocloseables(LOG, credentials);
+      credentials = null;
     }
   }
 
@@ -2885,6 +2886,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     }
     sb.append(", boundedExecutor=").append(boundedThreadPool);
     sb.append(", unboundedExecutor=").append(unboundedThreadPool);
+    sb.append(", credentials=").append(credentials);
     sb.append(", statistics {")
         .append(statistics)
         .append("}");
@@ -3319,4 +3321,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       return false;
     }
   }
+
+  /**
+   * Get a shared copy of the AWS credentials, with its reference
+   * counter updated.
+   * Caller is required to call {@code close()} on this after
+   * they have finished using it.
+   * @param purpose what is this for? This is initially for logging
+   * @return a reference to shared credentials.
+   */
+  public AWSCredentialProviderList shareCredentials(final String purpose) {
+    LOG.debug("Sharing credentials for: {}", purpose);
+    return credentials.share();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
index 2b361fd..e6e7895 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
@@ -37,6 +37,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.net.ConnectTimeoutException;
@@ -154,8 +155,9 @@ public class S3ARetryPolicy implements RetryPolicy {
     policyMap.put(InterruptedException.class, fail);
     // note this does not pick up subclasses (like socket timeout)
     policyMap.put(InterruptedIOException.class, fail);
-    // interesting question: should this be retried ever?
+    // Access denial and auth exceptions are not retried
     policyMap.put(AccessDeniedException.class, fail);
+    policyMap.put(NoAuthWithAWSException.class, fail);
     policyMap.put(FileNotFoundException.class, fail);
     policyMap.put(InvalidRequestException.class, fail);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/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 a5f7d75..9908fd1 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
@@ -21,6 +21,8 @@ package org.apache.hadoop.fs.s3a;
 import com.amazonaws.AbortedException;
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.Protocol;
 import com.amazonaws.SdkBaseException;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
@@ -44,15 +46,18 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.util.VersionInfo;
 
 import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
+import java.io.Closeable;
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -174,11 +179,17 @@ public final class S3AUtils {
         // call considered an sign of connectivity failure
         return (EOFException)new EOFException(message).initCause(exception);
       }
+      if (exception instanceof NoAuthWithAWSException) {
+        // the exception raised by AWSCredentialProvider list if the
+        // credentials were not accepted.
+        return (AccessDeniedException)new AccessDeniedException(path, null,
+            exception.toString()).initCause(exception);
+      }
       return new AWSClientIOException(message, exception);
     } else {
       if (exception instanceof AmazonDynamoDBException) {
         // special handling for dynamo DB exceptions
-        return translateDynamoDBException(message,
+        return translateDynamoDBException(path, message,
             (AmazonDynamoDBException)exception);
       }
       IOException ioe;
@@ -373,20 +384,45 @@ public final class S3AUtils {
 
   /**
    * Translate a DynamoDB exception into an IOException.
+   *
+   * @param path path in the DDB
    * @param message preformatted message for the exception
-   * @param ex exception
+   * @param ddbException exception
    * @return an exception to throw.
    */
-  public static IOException translateDynamoDBException(String message,
-      AmazonDynamoDBException ex) {
-    if (isThrottleException(ex)) {
-      return new AWSServiceThrottledException(message, ex);
+  public static IOException translateDynamoDBException(final String path,
+      final String message,
+      final AmazonDynamoDBException ddbException) {
+    if (isThrottleException(ddbException)) {
+      return new AWSServiceThrottledException(message, ddbException);
     }
-    if (ex instanceof ResourceNotFoundException) {
+    if (ddbException instanceof ResourceNotFoundException) {
       return (FileNotFoundException) new FileNotFoundException(message)
-          .initCause(ex);
+          .initCause(ddbException);
+    }
+    final int statusCode = ddbException.getStatusCode();
+    final String errorCode = ddbException.getErrorCode();
+    IOException result = null;
+    // 400 gets used a lot by DDB
+    if (statusCode == 400) {
+      switch (errorCode) {
+      case "AccessDeniedException":
+        result = (IOException) new AccessDeniedException(
+            path,
+            null,
+            ddbException.toString())
+            .initCause(ddbException);
+        break;
+
+      default:
+        result = new AWSBadRequestException(message, ddbException);
+      }
+
     }
-    return new AWSServiceIOException(message, ex);
+    if (result ==  null) {
+      result = new AWSServiceIOException(message, ddbException);
+    }
+    return result;
   }
 
   /**
@@ -738,6 +774,29 @@ public final class S3AUtils {
       String baseKey,
       String overrideVal)
       throws IOException {
+    return lookupPassword(bucket, conf, baseKey, overrideVal, "");
+  }
+
+  /**
+   * Get a password from a configuration, including JCEKS files, handling both
+   * the absolute key and bucket override.
+   * @param bucket bucket or "" if none known
+   * @param conf configuration
+   * @param baseKey base key to look up, e.g "fs.s3a.secret.key"
+   * @param overrideVal override value: if non empty this is used instead of
+   * querying the configuration.
+   * @param defVal value to return if there is no password
+   * @return a password or the value of defVal.
+   * @throws IOException on any IO problem
+   * @throws IllegalArgumentException bad arguments
+   */
+  public static String lookupPassword(
+      String bucket,
+      Configuration conf,
+      String baseKey,
+      String overrideVal,
+      String defVal)
+      throws IOException {
     String initialVal;
     Preconditions.checkArgument(baseKey.startsWith(FS_S3A_PREFIX),
         "%s does not start with $%s", baseKey, FS_S3A_PREFIX);
@@ -757,7 +816,7 @@ public final class S3AUtils {
       // no bucket, make the initial value the override value
       initialVal = overrideVal;
     }
-    return getPassword(conf, baseKey, initialVal);
+    return getPassword(conf, baseKey, initialVal, defVal);
   }
 
   /**
@@ -1059,6 +1118,134 @@ public final class S3AUtils {
     }
   }
 
+  /**
+   * Create a new AWS {@code ClientConfiguration}.
+   * All clients to AWS services <i>MUST</i> use this for consistent setup
+   * of connectivity, UA, proxy settings.
+   * @param conf The Hadoop configuration
+   * @param bucket Optional bucket to use to look up per-bucket proxy secrets
+   * @return new AWS client configuration
+   */
+  public static ClientConfiguration createAwsConf(Configuration conf,
+      String bucket)
+      throws IOException {
+    final ClientConfiguration awsConf = new ClientConfiguration();
+    initConnectionSettings(conf, awsConf);
+    initProxySupport(conf, bucket, awsConf);
+    initUserAgent(conf, awsConf);
+    return awsConf;
+  }
+
+  /**
+   * Initializes all AWS SDK settings related to connection management.
+   *
+   * @param conf Hadoop configuration
+   * @param awsConf AWS SDK configuration
+   */
+  public static void initConnectionSettings(Configuration conf,
+      ClientConfiguration awsConf) {
+    awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
+        DEFAULT_MAXIMUM_CONNECTIONS, 1));
+    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
+        DEFAULT_SECURE_CONNECTIONS);
+    awsConf.setProtocol(secureConnections ?  Protocol.HTTPS : Protocol.HTTP);
+    awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
+        DEFAULT_MAX_ERROR_RETRIES, 0));
+    awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
+        DEFAULT_ESTABLISH_TIMEOUT, 0));
+    awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
+        DEFAULT_SOCKET_TIMEOUT, 0));
+    int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER,
+        DEFAULT_SOCKET_SEND_BUFFER, 2048);
+    int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER,
+        DEFAULT_SOCKET_RECV_BUFFER, 2048);
+    awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer);
+    String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
+    if (!signerOverride.isEmpty()) {
+     LOG.debug("Signer override = {}", signerOverride);
+      awsConf.setSignerOverride(signerOverride);
+    }
+  }
+
+  /**
+   * Initializes AWS SDK proxy support in the AWS client configuration
+   * if the S3A settings enable it.
+   *
+   * @param conf Hadoop configuration
+   * @param bucket Optional bucket to use to look up per-bucket proxy secrets
+   * @param awsConf AWS SDK configuration to update
+   * @throws IllegalArgumentException if misconfigured
+   * @throws IOException problem getting username/secret from password source.
+   */
+  public static void initProxySupport(Configuration conf,
+      String bucket,
+      ClientConfiguration awsConf) throws IllegalArgumentException,
+      IOException {
+    String proxyHost = conf.getTrimmed(PROXY_HOST, "");
+    int proxyPort = conf.getInt(PROXY_PORT, -1);
+    if (!proxyHost.isEmpty()) {
+      awsConf.setProxyHost(proxyHost);
+      if (proxyPort >= 0) {
+        awsConf.setProxyPort(proxyPort);
+      } else {
+        if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
+          LOG.warn("Proxy host set without port. Using HTTPS default 443");
+          awsConf.setProxyPort(443);
+        } else {
+          LOG.warn("Proxy host set without port. Using HTTP default 80");
+          awsConf.setProxyPort(80);
+        }
+      }
+      final String proxyUsername = lookupPassword(bucket, conf, PROXY_USERNAME,
+          null, null);
+      final String proxyPassword = lookupPassword(bucket, conf, PROXY_PASSWORD,
+          null, null);
+      if ((proxyUsername == null) != (proxyPassword == null)) {
+        String msg = "Proxy error: " + PROXY_USERNAME + " or " +
+            PROXY_PASSWORD + " set without the other.";
+        LOG.error(msg);
+        throw new IllegalArgumentException(msg);
+      }
+      awsConf.setProxyUsername(proxyUsername);
+      awsConf.setProxyPassword(proxyPassword);
+      awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN));
+      awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Using proxy server {}:{} as user {} with password {} on " +
+                "domain {} as workstation {}", awsConf.getProxyHost(),
+            awsConf.getProxyPort(),
+            String.valueOf(awsConf.getProxyUsername()),
+            awsConf.getProxyPassword(), awsConf.getProxyDomain(),
+            awsConf.getProxyWorkstation());
+      }
+    } else if (proxyPort >= 0) {
+      String msg =
+          "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
+      LOG.error(msg);
+      throw new IllegalArgumentException(msg);
+    }
+  }
+
+  /**
+   * Initializes the User-Agent header to send in HTTP requests to AWS
+   * services.  We always include the Hadoop version number.  The user also
+   * may set an optional custom prefix to put in front of the Hadoop version
+   * number.  The AWS SDK internally appends its own information, which seems
+   * to include the AWS SDK version, OS and JVM version.
+   *
+   * @param conf Hadoop configuration
+   * @param awsConf AWS SDK configuration to update
+   */
+  private static void initUserAgent(Configuration conf,
+      ClientConfiguration awsConf) {
+    String userAgent = "Hadoop " + VersionInfo.getVersion();
+    String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, "");
+    if (!userAgentPrefix.isEmpty()) {
+      userAgent = userAgentPrefix + ", " + userAgent;
+    }
+    LOG.debug("Using User-Agent: {}", userAgent);
+    awsConf.setUserAgentPrefix(userAgent);
+  }
 
   /**
    * An interface for use in lambda-expressions working with
@@ -1289,18 +1476,40 @@ public final class S3AUtils {
    * @param closeables the objects to close
    */
   public static void closeAll(Logger log,
-      java.io.Closeable... closeables) {
-    for (java.io.Closeable c : closeables) {
+      Closeable... closeables) {
+    if (log == null) {
+      log = LOG;
+    }
+    for (Closeable c : closeables) {
       if (c != null) {
         try {
-          if (log != null) {
-            log.debug("Closing {}", c);
-          }
+          log.debug("Closing {}", c);
           c.close();
         } catch (Exception e) {
-          if (log != null && log.isDebugEnabled()) {
-            log.debug("Exception in closing {}", c, e);
-          }
+          log.debug("Exception in closing {}", c, e);
+        }
+      }
+    }
+  }
+  /**
+   * Close the Closeable objects and <b>ignore</b> any Exception or
+   * null pointers.
+   * (This is the SLF4J equivalent of that in {@code IOUtils}).
+   * @param log the log to log at debug level. Can be null.
+   * @param closeables the objects to close
+   */
+  public static void closeAutocloseables(Logger log,
+      AutoCloseable... closeables) {
+    if (log == null) {
+      log = LOG;
+    }
+    for (AutoCloseable c : closeables) {
+      if (c != null) {
+        try {
+          log.debug("Closing {}", c);
+          c.close();
+        } catch (Exception e) {
+          log.debug("Exception in closing {}", c, e);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
index 9abb362..b237e85 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a;
 import java.io.IOException;
 import java.net.URI;
 
+import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.services.s3.AmazonS3;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -37,9 +38,13 @@ public interface S3ClientFactory {
    * Creates a new {@link AmazonS3} client.
    *
    * @param name raw input S3A file system URI
+   * @param bucket Optional bucket to use to look up per-bucket proxy secrets
+   * @param credentialSet credentials to use
    * @return S3 client
    * @throws IOException IO problem
    */
-  AmazonS3 createS3Client(URI name) throws IOException;
+  AmazonS3 createS3Client(URI name,
+      final String bucket,
+      final AWSCredentialsProvider credentialSet) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
index fdaf9bd..e5a3639 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
@@ -24,9 +24,11 @@ import java.net.URI;
 import java.util.Locale;
 import java.util.concurrent.TimeUnit;
 
+import com.amazonaws.AmazonClientException;
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
 import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
 import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
@@ -37,6 +39,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
+import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
 import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -77,17 +82,21 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
 
   private final String arn;
 
+  private final AWSCredentialProviderList credentialsToSTS;
+
+  private final Invoker invoker;
+
   /**
    * Instantiate.
    * This calls {@link #getCredentials()} to fail fast on the inner
    * role credential retrieval.
-   * @param uri URI of endpoint.
+   * @param fsUri URI of the filesystem.
    * @param conf configuration
    * @throws IOException on IO problems and some parameter checking
    * @throws IllegalArgumentException invalid parameters
    * @throws AWSSecurityTokenServiceException problems getting credentials
    */
-  public AssumedRoleCredentialProvider(URI uri, Configuration conf)
+  public AssumedRoleCredentialProvider(URI fsUri, Configuration conf)
       throws IOException {
 
     arn = conf.getTrimmed(ASSUMED_ROLE_ARN, "");
@@ -99,13 +108,14 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
     Class<?>[] awsClasses = loadAWSProviderClasses(conf,
         ASSUMED_ROLE_CREDENTIALS_PROVIDER,
         SimpleAWSCredentialsProvider.class);
-    AWSCredentialProviderList credentials = new AWSCredentialProviderList();
+    credentialsToSTS = new AWSCredentialProviderList();
     for (Class<?> aClass : awsClasses) {
       if (this.getClass().equals(aClass)) {
         throw new IOException(E_FORBIDDEN_PROVIDER);
       }
-      credentials.add(createAWSCredentialProvider(conf, aClass, uri));
+      credentialsToSTS.add(createAWSCredentialProvider(conf, aClass, fsUri));
     }
+    LOG.debug("Credentials to obtain role credentials: {}", credentialsToSTS);
 
     // then the STS binding
     sessionName = conf.getTrimmed(ASSUMED_ROLE_SESSION_NAME,
@@ -122,14 +132,27 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
       LOG.debug("Scope down policy {}", policy);
       builder.withScopeDownPolicy(policy);
     }
-    String epr = conf.get(ASSUMED_ROLE_STS_ENDPOINT, "");
-    if (StringUtils.isNotEmpty(epr)) {
-      LOG.debug("STS Endpoint: {}", epr);
-      builder.withServiceEndpoint(epr);
-    }
-    LOG.debug("Credentials to obtain role credentials: {}", credentials);
-    builder.withLongLivedCredentialsProvider(credentials);
+    String endpoint = conf.get(ASSUMED_ROLE_STS_ENDPOINT, "");
+    String region = conf.get(ASSUMED_ROLE_STS_ENDPOINT_REGION,
+        ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT);
+    AWSSecurityTokenServiceClientBuilder stsbuilder =
+        STSClientFactory.builder(
+          conf,
+          fsUri.getHost(),
+          credentialsToSTS,
+          endpoint,
+          region);
+    // the STS client is not tracked for a shutdown in close(), because it
+    // (currently) throws an UnsupportedOperationException in shutdown().
+    builder.withStsClient(stsbuilder.build());
+
+    //now build the provider
     stsProvider = builder.build();
+
+    // to handle STS throttling by the AWS account, we
+    // need to retry
+    invoker = new Invoker(new S3ARetryPolicy(conf), this::operationRetried);
+
     // and force in a fail-fast check just to keep the stack traces less
     // convoluted
     getCredentials();
@@ -143,7 +166,17 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
   @Override
   public AWSCredentials getCredentials() {
     try {
-      return stsProvider.getCredentials();
+      return invoker.retryUntranslated("getCredentials",
+          true,
+          stsProvider::getCredentials);
+    } catch (IOException e) {
+      // this is in the signature of retryUntranslated;
+      // its hard to see how this could be raised, but for
+      // completeness, it is wrapped as an Amazon Client Exception
+      // and rethrown.
+      throw new AmazonClientException(
+          "getCredentials failed: " + e,
+          e);
     } catch (AWSSecurityTokenServiceException e) {
       LOG.error("Failed to get credentials for role {}",
           arn, e);
@@ -161,7 +194,7 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
    */
   @Override
   public void close() {
-    stsProvider.close();
+    S3AUtils.closeAutocloseables(LOG, stsProvider, credentialsToSTS);
   }
 
   @Override
@@ -205,4 +238,23 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
     return r.toString();
   }
 
+  /**
+   * Callback from {@link Invoker} when an operation is retried.
+   * @param text text of the operation
+   * @param ex exception
+   * @param retries number of retries
+   * @param idempotent is the method idempotent
+   */
+  public void operationRetried(
+      String text,
+      Exception ex,
+      int retries,
+      boolean idempotent) {
+    if (retries == 0) {
+      // log on the first retry attempt of the credential access.
+      // At worst, this means one log entry every intermittent renewal
+      // time.
+      LOG.info("Retried {}", text);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java
new file mode 100644
index 0000000..f48e17a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.s3a.auth;
+
+import com.amazonaws.AmazonClientException;
+
+/**
+ * A specific subclass of {@code AmazonClientException} which can
+ * be used in the retry logic to fail fast when there is any
+ * authentication problem.
+ */
+public class NoAuthWithAWSException extends AmazonClientException {
+
+  public NoAuthWithAWSException(final String message, final Throwable t) {
+    super(message, t);
+  }
+
+  public NoAuthWithAWSException(final String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RoleModel.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RoleModel.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RoleModel.java
index ca2c993..d4568b0 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RoleModel.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RoleModel.java
@@ -205,6 +205,14 @@ public class RoleModel {
     return new Policy(statements);
   }
 
+  /**
+   * From a set of statements, create a policy.
+   * @param statements statements
+   * @return the policy
+   */
+  public static Policy policy(final List<RoleModel.Statement> statements) {
+    return new Policy(statements);
+  }
 
   /**
    * Effect options.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java
index 6711eee..34ed295 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java
@@ -29,6 +29,55 @@ public final class RolePolicies {
   private RolePolicies() {
   }
 
+  /** All KMS operations: {@value}.*/
+  public static final String KMS_ALL_OPERATIONS = "kms:*";
+
+  /** KMS encryption. This is <i>Not</i> used by SSE-KMS: {@value}. */
+  public static final String KMS_ENCRYPT = "kms:Encrypt";
+
+  /**
+   * Decrypt data encrypted with SSE-KMS: {@value}.
+   */
+  public static final String KMS_DECRYPT = "kms:Decrypt";
+
+  /**
+   * Arn for all KMS keys: {@value}.
+   */
+  public static final String KMS_ALL_KEYS = "arn:aws:kms:*";
+
+  /**
+   * This is used by S3 to generate a per-object encryption key and
+   * the encrypted value of this, the latter being what it tags
+   * the object with for later decryption: {@value}.
+   */
+  public static final String KMS_GENERATE_DATA_KEY = "kms:GenerateDataKey";
+
+  /**
+   * Actions needed to read and write SSE-KMS data.
+   */
+  private static final String[] KMS_KEY_RW =
+      new String[]{KMS_DECRYPT, KMS_GENERATE_DATA_KEY};
+
+  /**
+   * Actions needed to read SSE-KMS data.
+   */
+  private static final String[] KMS_KEY_READ =
+      new String[] {KMS_DECRYPT};
+
+  /**
+   * Statement to allow KMS R/W access access, so full use of
+   * SSE-KMS.
+   */
+  public static final Statement STATEMENT_ALLOW_SSE_KMS_RW =
+      statement(true, KMS_ALL_KEYS, KMS_KEY_RW);
+
+  /**
+   * Statement to allow read access to KMS keys, so the ability
+   * to read SSE-KMS data,, but not decrypt it.
+   */
+  public static final Statement STATEMENT_ALLOW_SSE_KMS_READ =
+      statement(true, KMS_ALL_KEYS, KMS_KEY_READ);
+
   /**
    * All S3 operations: {@value}.
    */
@@ -52,7 +101,6 @@ public final class RolePolicies {
   public static final String S3_LIST_BUCKET_MULTPART_UPLOADS =
       "s3:ListBucketMultipartUploads";
 
-
   /**
    * List multipart upload is needed for the S3A Commit protocols.
    */
@@ -97,6 +145,8 @@ public final class RolePolicies {
 
   public static final String S3_GET_OBJECT_VERSION = "s3:GetObjectVersion";
 
+  public static final String S3_GET_BUCKET_LOCATION = "s3:GetBucketLocation";
+
   public static final String S3_GET_OBJECT_VERSION_ACL
       = "s3:GetObjectVersionAcl";
 
@@ -128,7 +178,8 @@ public final class RolePolicies {
   public static final String S3_RESTORE_OBJECT = "s3:RestoreObject";
 
   /**
-   * Actions needed to read data from S3 through S3A.
+   * Actions needed to read a file in S3 through S3A, excluding
+   * S3Guard and SSE-KMS.
    */
   public static final String[] S3_PATH_READ_OPERATIONS =
       new String[]{
@@ -136,18 +187,20 @@ public final class RolePolicies {
       };
 
   /**
-   * Actions needed to read data from S3 through S3A.
+   * Base actions needed to read data from S3 through S3A,
+   * excluding SSE-KMS data and S3Guard-ed buckets.
    */
   public static final String[] S3_ROOT_READ_OPERATIONS =
       new String[]{
           S3_LIST_BUCKET,
           S3_LIST_BUCKET_MULTPART_UPLOADS,
-          S3_GET_OBJECT,
+          S3_ALL_GET,
       };
 
   /**
    * Actions needed to write data to an S3A Path.
-   * This includes the appropriate read operations.
+   * This includes the appropriate read operations, but
+   * not SSE-KMS or S3Guard support.
    */
   public static final String[] S3_PATH_RW_OPERATIONS =
       new String[]{
@@ -163,6 +216,7 @@ public final class RolePolicies {
    * This is purely the extra operations needed for writing atop
    * of the read operation set.
    * Deny these and a path is still readable, but not writeable.
+   * Excludes: SSE-KMS and S3Guard permissions.
    */
   public static final String[] S3_PATH_WRITE_OPERATIONS =
       new String[]{
@@ -173,6 +227,7 @@ public final class RolePolicies {
 
   /**
    * Actions needed for R/W IO from the root of a bucket.
+   * Excludes: SSE-KMS and S3Guard permissions.
    */
   public static final String[] S3_ROOT_RW_OPERATIONS =
       new String[]{
@@ -190,26 +245,57 @@ public final class RolePolicies {
    */
   public static final String DDB_ALL_OPERATIONS = "dynamodb:*";
 
-  public static final String DDB_ADMIN = "dynamodb:*";
+  /**
+   * Operations needed for DDB/S3Guard Admin.
+   * For now: make this {@link #DDB_ALL_OPERATIONS}.
+   */
+  public static final String DDB_ADMIN = DDB_ALL_OPERATIONS;
 
+  /**
+   * Permission for DDB describeTable() operation: {@value}.
+   * This is used during initialization.
+   */
+  public static final String DDB_DESCRIBE_TABLE = "dynamodb:DescribeTable";
 
-  public static final String DDB_BATCH_WRITE = "dynamodb:BatchWriteItem";
+  /**
+   * Permission to query the DDB table: {@value}.
+   */
+  public static final String DDB_QUERY = "dynamodb:Query";
 
   /**
-   * All DynamoDB tables: {@value}.
+   * Permission for DDB operation to get a record: {@value}.
    */
-  public static final String ALL_DDB_TABLES = "arn:aws:dynamodb:::*";
+  public static final String DDB_GET_ITEM = "dynamodb:GetItem";
 
+  /**
+   * Permission for DDB write record operation: {@value}.
+   */
+  public static final String DDB_PUT_ITEM = "dynamodb:PutItem";
 
+  /**
+   * Permission for DDB update single item operation: {@value}.
+   */
+  public static final String DDB_UPDATE_ITEM = "dynamodb:UpdateItem";
 
-  public static final String WILDCARD = "*";
+  /**
+   * Permission for DDB delete operation: {@value}.
+   */
+  public static final String DDB_DELETE_ITEM = "dynamodb:DeleteItem";
 
   /**
-   * Allow all S3 Operations.
+   * Permission for DDB operation: {@value}.
    */
-  public static final Statement STATEMENT_ALL_S3 = statement(true,
-      S3_ALL_BUCKETS,
-      S3_ALL_OPERATIONS);
+  public static final String DDB_BATCH_GET_ITEM = "dynamodb:BatchGetItem";
+
+  /**
+   * Batch write permission for DDB: {@value}.
+   */
+  public static final String DDB_BATCH_WRITE_ITEM = "dynamodb:BatchWriteItem";
+
+  /**
+   * All DynamoDB tables: {@value}.
+   */
+  public static final String ALL_DDB_TABLES = "arn:aws:dynamodb:*";
 
   /**
    * Statement to allow all DDB access.
@@ -218,11 +304,36 @@ public final class RolePolicies {
       ALL_DDB_TABLES, DDB_ALL_OPERATIONS);
 
   /**
-   * Allow all S3 and S3Guard operations.
+   * Statement to allow all client operations needed for S3Guard,
+   * but none of the admin operations.
+   */
+  public static final Statement STATEMENT_S3GUARD_CLIENT = statement(true,
+      ALL_DDB_TABLES,
+      DDB_BATCH_GET_ITEM,
+      DDB_BATCH_WRITE_ITEM,
+      DDB_DELETE_ITEM,
+      DDB_DESCRIBE_TABLE,
+      DDB_GET_ITEM,
+      DDB_PUT_ITEM,
+      DDB_QUERY,
+      DDB_UPDATE_ITEM
+      );
+
+  /**
+   * Allow all S3 Operations.
+   * This does not cover DDB or S3-KMS
+   */
+  public static final Statement STATEMENT_ALL_S3 = statement(true,
+      S3_ALL_BUCKETS,
+      S3_ALL_OPERATIONS);
+
+  /**
+   * Policy for all S3 and S3Guard operations, and SSE-KMS.
    */
   public static final Policy ALLOW_S3_AND_SGUARD = policy(
       STATEMENT_ALL_S3,
-      STATEMENT_ALL_DDB
+      STATEMENT_ALL_DDB,
+      STATEMENT_ALLOW_SSE_KMS_RW
   );
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
new file mode 100644
index 0000000..10bf88c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
@@ -0,0 +1,78 @@
+/*
+ * 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.s3a.auth;
+
+import java.io.IOException;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.S3AUtils;
+
+/**
+ * Factory for creating STS Clients.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class STSClientFactory {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(STSClientFactory.class);
+
+  /**
+   * Create the builder ready for any final configuration options.
+   * Picks up connection settings from the Hadoop configuration, including
+   * proxy secrets.
+   * @param conf Configuration to act as source of options.
+   * @param bucket Optional bucket to use to look up per-bucket proxy secrets
+   * @param credentials AWS credential chain to use
+   * @param stsEndpoint optional endpoint "https://sns.us-west-1.amazonaws.com"
+   * @param stsRegion the region, e.g "us-west-1"
+   * @return the builder to call {@code build()}
+   * @throws IOException problem reading proxy secrets
+   */
+  public static AWSSecurityTokenServiceClientBuilder builder(
+      final Configuration conf,
+      final String bucket,
+      final AWSCredentialsProvider credentials, final String stsEndpoint,
+      final String stsRegion) throws IOException {
+    Preconditions.checkArgument(credentials != null, "No credentials");
+    final AWSSecurityTokenServiceClientBuilder builder
+        = AWSSecurityTokenServiceClientBuilder.standard();
+    final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
+    builder.withClientConfiguration(awsConf);
+    builder.withCredentials(credentials);
+    if (StringUtils.isNotEmpty(stsEndpoint)) {
+      LOG.debug("STS Endpoint ={}", stsEndpoint);
+      builder.withEndpointConfiguration(
+          new AwsClientBuilder.EndpointConfiguration(stsEndpoint, stsRegion));
+    }
+    return builder;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
index 91e64cd..9e1d2f4 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
@@ -34,10 +34,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.s3a.DefaultS3ClientFactory;
+import org.apache.hadoop.fs.s3a.S3AUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY;
-import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
 
 /**
  * Interface to create a DynamoDB client.
@@ -58,10 +57,14 @@ public interface DynamoDBClientFactory extends Configurable {
    * it will indicate an error.
    *
    * @param defaultRegion the default region of the AmazonDynamoDB client
+   * @param bucket Optional bucket to use to look up per-bucket proxy secrets
+   * @param credentials credentials to use for authentication.
    * @return a new DynamoDB client
    * @throws IOException if any IO error happens
    */
-  AmazonDynamoDB createDynamoDBClient(String defaultRegion) throws IOException;
+  AmazonDynamoDB createDynamoDBClient(final String defaultRegion,
+      final String bucket,
+      final AWSCredentialsProvider credentials) throws IOException;
 
   /**
    * The default implementation for creating an AmazonDynamoDB.
@@ -69,16 +72,15 @@ public interface DynamoDBClientFactory extends Configurable {
   class DefaultDynamoDBClientFactory extends Configured
       implements DynamoDBClientFactory {
     @Override
-    public AmazonDynamoDB createDynamoDBClient(String defaultRegion)
+    public AmazonDynamoDB createDynamoDBClient(String defaultRegion,
+        final String bucket,
+        final AWSCredentialsProvider credentials)
         throws IOException {
       Preconditions.checkNotNull(getConf(),
           "Should have been configured before usage");
 
       final Configuration conf = getConf();
-      final AWSCredentialsProvider credentials =
-          createAWSCredentialProviderSet(null, conf);
-      final ClientConfiguration awsConf =
-          DefaultS3ClientFactory.createAwsConf(conf);
+      final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
 
       final String region = getRegion(conf, defaultRegion);
       LOG.debug("Creating DynamoDB client in region {}", region);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da9a39ee/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
index 43849b1..ba80b88 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
@@ -22,6 +22,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.URI;
+import java.nio.file.AccessDeniedException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -34,6 +35,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.amazonaws.AmazonClientException;
+import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
 import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
 import com.amazonaws.services.dynamodbv2.document.DynamoDB;
@@ -67,6 +69,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
 import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.Invoker;
 import org.apache.hadoop.fs.s3a.Retries;
@@ -75,13 +78,14 @@ import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
 import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.fs.s3a.auth.RolePolicies;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
 import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*;
 
@@ -207,6 +211,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
       new ValueMap().withBoolean(":false", false);
 
   private DynamoDB dynamoDB;
+  private AWSCredentialProviderList credentials;
   private String region;
   private Table table;
   private String tableName;
@@ -242,10 +247,16 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * A utility function to create DynamoDB instance.
    * @param conf the file system configuration
    * @param s3Region region of the associated S3 bucket (if any).
+   * @param bucket Optional bucket to use to look up per-bucket proxy secrets
+   * @param credentials credentials.
    * @return DynamoDB instance.
    * @throws IOException I/O error.
    */
-  private static DynamoDB createDynamoDB(Configuration conf, String s3Region)
+  private static DynamoDB createDynamoDB(
+      final Configuration conf,
+      final String s3Region,
+      final String bucket,
+      final AWSCredentialsProvider credentials)
       throws IOException {
     Preconditions.checkNotNull(conf);
     final Class<? extends DynamoDBClientFactory> cls = conf.getClass(
@@ -254,10 +265,18 @@ public class DynamoDBMetadataStore implements MetadataStore {
         DynamoDBClientFactory.class);
     LOG.debug("Creating DynamoDB client {} with S3 region {}", cls, s3Region);
     final AmazonDynamoDB dynamoDBClient = ReflectionUtils.newInstance(cls, conf)
-        .createDynamoDBClient(s3Region);
+        .createDynamoDBClient(s3Region, bucket, credentials);
     return new DynamoDB(dynamoDBClient);
   }
 
+  /**
+   * {@inheritDoc}.
+   * The credentials for authenticating with S3 are requested from the
+   * FS via {@link S3AFileSystem#shareCredentials(String)}; this will
+   * increment the reference counter of these credentials.
+   * @param fs {@code S3AFileSystem} associated with the MetadataStore
+   * @throws IOException on a failure
+   */
   @Override
   @Retries.OnceRaw
   public void initialize(FileSystem fs) throws IOException {
@@ -274,11 +293,23 @@ public class DynamoDBMetadataStore implements MetadataStore {
       LOG.debug("Overriding S3 region with configured DynamoDB region: {}",
           region);
     } else {
-      region = owner.getBucketLocation();
+      try {
+        region = owner.getBucketLocation();
+      } catch (AccessDeniedException e) {
+        // access denied here == can't call getBucket. Report meaningfully
+        URI uri = owner.getUri();
+        LOG.error("Failed to get bucket location from S3 bucket {}",
+            uri);
+        throw (IOException)new AccessDeniedException(
+            "S3 client role lacks permission "
+                + RolePolicies.S3_GET_BUCKET_LOCATION + " for " + uri)
+            .initCause(e);
+      }
       LOG.debug("Inferring DynamoDB region from S3 bucket: {}", region);
     }
     username = owner.getUsername();
-    dynamoDB = createDynamoDB(conf, region);
+    credentials = owner.shareCredentials("s3guard");
+    dynamoDB = createDynamoDB(conf, region, bucket, credentials);
 
     // use the bucket as the DynamoDB table name if not specified in config
     tableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY, bucket);
@@ -311,6 +342,9 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * must declare the table name and region in the
    * {@link Constants#S3GUARD_DDB_TABLE_NAME_KEY} and
    * {@link Constants#S3GUARD_DDB_REGION_KEY} respectively.
+   * It also creates a new credential provider list from the configuration,
+   * using the base fs.s3a.* options, as there is no bucket to infer per-bucket
+   * settings from.
    *
    * @see #initialize(FileSystem)
    * @throws IOException if there is an error
@@ -327,7 +361,8 @@ public class DynamoDBMetadataStore implements MetadataStore {
     region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
     Preconditions.checkArgument(!StringUtils.isEmpty(region),
         "No DynamoDB region configured");
-    dynamoDB = createDynamoDB(conf, region);
+    credentials = createAWSCredentialProviderSet(null, conf);
+    dynamoDB = createDynamoDB(conf, region, null, credentials);
 
     username = UserGroupInformation.getCurrentUser().getShortUserName();
     initDataAccessRetries(conf);
@@ -778,12 +813,17 @@ public class DynamoDBMetadataStore implements MetadataStore {
     if (instrumentation != null) {
       instrumentation.storeClosed();
     }
-    if (dynamoDB != null) {
-      LOG.debug("Shutting down {}", this);
-      dynamoDB.shutdown();
-      dynamoDB = null;
+    try {
+      if (dynamoDB != null) {
+        LOG.debug("Shutting down {}", this);
+        dynamoDB.shutdown();
+        dynamoDB = null;
+      }
+    } finally {
+      closeAutocloseables(LOG, credentials);
+      credentials = null;
     }
-  }
+}
 
   @Override
   @Retries.OnceTranslated


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


[10/50] [abbrv] hadoop git commit: YARN-8568. Replace the deprecated zk-address property in the HA config example in ResourceManagerHA.md (bsteinbach via rkanter)

Posted by tm...@apache.org.
YARN-8568. Replace the deprecated zk-address property in the HA config example in ResourceManagerHA.md (bsteinbach via rkanter)


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

Branch: refs/heads/HADOOP-15407
Commit: 8478732bb28e9e71061d6b4a043a3a1b5c688902
Parents: 3214cd7
Author: Robert Kanter <rk...@apache.org>
Authored: Wed Aug 8 15:08:55 2018 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Wed Aug 8 15:08:55 2018 -0700

----------------------------------------------------------------------
 .../hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md        | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8478732b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
index da9f5a0..ff97328 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
@@ -111,7 +111,7 @@ Here is the sample of minimal setup for RM failover.
   <value>master2:8088</value>
 </property>
 <property>
-  <name>yarn.resourcemanager.zk-address</name>
+  <name>hadoop.zk.address</name>
   <value>zk1:2181,zk2:2181,zk3:2181</value>
 </property>
 ```


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


[23/50] [abbrv] hadoop git commit: YARN-8588. Logging improvements for better debuggability. (Suma Shivaprasad via wangda)

Posted by tm...@apache.org.
YARN-8588. Logging improvements for better debuggability. (Suma Shivaprasad via wangda)

Change-Id: I66aa4b0ec031ae5ce0fae558e2f8cbcbbfebc442


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

Branch: refs/heads/HADOOP-15407
Commit: 344c335a920e6f32a35ebace0a118a9dc4a22fb7
Parents: 5326a79
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Aug 9 11:03:00 2018 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Aug 9 11:04:02 2018 -0700

----------------------------------------------------------------------
 .../capacity/AutoCreatedLeafQueueConfig.java    |  5 ++
 .../capacity/QueueManagementChange.java         |  2 +-
 .../QueueManagementDynamicEditPolicy.java       | 36 ++++++--------
 .../GuaranteedOrZeroCapacityOverTimePolicy.java | 50 ++++++++++++--------
 4 files changed, 52 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/344c335a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedLeafQueueConfig.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/AutoCreatedLeafQueueConfig.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/AutoCreatedLeafQueueConfig.java
index 5952250..87ef1c0 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/AutoCreatedLeafQueueConfig.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/AutoCreatedLeafQueueConfig.java
@@ -63,4 +63,9 @@ public class AutoCreatedLeafQueueConfig {
   public CapacitySchedulerConfiguration getLeafQueueConfigs() {
     return leafQueueConfigs;
   }
+
+  @Override public String toString() {
+    return "AutoCreatedLeafQueueConfig{" + "queueCapacities=" + queueCapacities
+        + ", leafQueueConfigs=" + leafQueueConfigs + '}';
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/344c335a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementChange.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/QueueManagementChange.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/QueueManagementChange.java
index 74d9b23..64ba578 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/QueueManagementChange.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/QueueManagementChange.java
@@ -124,7 +124,7 @@ public abstract class QueueManagementChange {
 
   @Override
   public String toString() {
-    return "QueueManagementChange{" + "queue=" + queue
+    return "QueueManagementChange{" + "queue=" + queue.getQueueName()
         + ", updatedEntitlementsByPartition=" + queueTemplateUpdate
         + ", queueAction=" + queueAction + ", transitionToQueueState="
         + transitionToQueueState + '}';

http://git-wip-us.apache.org/repos/asf/hadoop/blob/344c335a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementDynamicEditPolicy.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/QueueManagementDynamicEditPolicy.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/QueueManagementDynamicEditPolicy.java
index 9b0cf7b..ea43ac8 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/QueueManagementDynamicEditPolicy.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/QueueManagementDynamicEditPolicy.java
@@ -19,8 +19,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -50,8 +51,8 @@ import java.util.Set;
  */
 public class QueueManagementDynamicEditPolicy implements SchedulingEditPolicy {
 
-  private static final Log LOG =
-      LogFactory.getLog(QueueManagementDynamicEditPolicy.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(QueueManagementDynamicEditPolicy.class);
 
   private Clock clock;
 
@@ -90,7 +91,7 @@ public class QueueManagementDynamicEditPolicy implements SchedulingEditPolicy {
   @Override
   public void init(final Configuration config, final RMContext context,
       final ResourceScheduler sched) {
-    LOG.info("Queue Management Policy monitor:" + this.
+    LOG.info("Queue Management Policy monitor: {}" + this.
         getClass().getCanonicalName());
     assert null == scheduler : "Unexpected duplicate call to init";
     if (!(sched instanceof CapacityScheduler)) {
@@ -189,13 +190,7 @@ public class QueueManagementDynamicEditPolicy implements SchedulingEditPolicy {
           parentQueue.getAutoCreatedQueueManagementPolicy();
       long startTime = 0;
       try {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(MessageFormat
-              .format("Trying to use {0} to compute preemption "
-                      + "candidates",
-                  policyClazz.getClass().getName()));
-          startTime = clock.getTime();
-        }
+        startTime = clock.getTime();
 
         queueManagementChanges = policyClazz.computeQueueManagementChanges();
 
@@ -209,15 +204,14 @@ public class QueueManagementDynamicEditPolicy implements SchedulingEditPolicy {
         }
 
         if (LOG.isDebugEnabled()) {
-          LOG.debug(MessageFormat.format("{0} uses {1} millisecond"
-                  + " to run",
-              policyClazz.getClass().getName(), clock.getTime()
-                  - startTime));
+          LOG.debug("{} uses {} millisecond" + " to run",
+              policyClazz.getClass().getName(), clock.getTime() - startTime);
           if (queueManagementChanges.size() > 0) {
-            LOG.debug(" Updated queue management updates for parent queue"
-                + " ["
-                + parentQueue.getQueueName() + ": [\n" + queueManagementChanges
-                .toString() + "\n]");
+            LOG.debug(" Updated queue management changes for parent queue" + " "
+                    + "{}: [{}]", parentQueue.getQueueName(),
+                queueManagementChanges.size() < 25 ?
+                    queueManagementChanges.toString() :
+                    queueManagementChanges.size());
           }
         }
       } catch (YarnException e) {
@@ -232,7 +226,7 @@ public class QueueManagementDynamicEditPolicy implements SchedulingEditPolicy {
             "Skipping queue management updates for parent queue "
                 + parentQueue
                 .getQueuePath() + " "
-                + "since configuration for  auto creating queue's beyond "
+                + "since configuration for auto creating queues beyond "
                 + "parent's "
                 + "guaranteed capacity is disabled");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/344c335a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.java
index b2301fd..faa6e6f 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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.java
@@ -19,8 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
     .queuemanagement;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .QueueManagementDynamicEditPolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler
     .SchedulerDynamicEditException;
@@ -81,8 +83,8 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
   private CapacitySchedulerContext scheduler;
   private ManagedParentQueue managedParentQueue;
 
-  private static final Log LOG = LogFactory.getLog(
-      GuaranteedOrZeroCapacityOverTimePolicy.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(GuaranteedOrZeroCapacityOverTimePolicy.class);
 
   private ReentrantReadWriteLock.WriteLock writeLock;
 
@@ -380,6 +382,17 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
             deactivateLeafQueuesIfInActive(managedParentQueue, nodeLabel,
                 leafQueueEntitlements);
 
+        if (LOG.isDebugEnabled()) {
+          if ( deactivatedLeafQueues.size() > 0) {
+              LOG.debug("Parent queue = {},  " +
+                   ", nodeLabel = {}, deactivated leaf queues = [{}] ",
+                  managedParentQueue.getQueueName(), nodeLabel,
+                  deactivatedLeafQueues.size() > 25 ? deactivatedLeafQueues
+                      .size() : deactivatedLeafQueues);
+
+          }
+        }
+
         float deactivatedCapacity = getTotalDeactivatedCapacity(
             deactivatedLeafQueues, nodeLabel);
 
@@ -392,7 +405,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
                 + deactivatedCapacity + EPSILON;
 
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Parent queue : " + managedParentQueue.getQueueName()
+          LOG.debug("Parent queue = " + managedParentQueue.getQueueName()
               + ", nodeLabel = " + nodeLabel + ", absCapacity = "
               + parentAbsoluteCapacity + ", leafQueueAbsoluteCapacity = "
               + leafQueueTemplateAbsoluteCapacity + ", deactivatedCapacity = "
@@ -409,7 +422,8 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
                 pendingApps.size());
 
             if (LOG.isDebugEnabled()) {
-              LOG.debug("Found " + maxLeafQueuesTobeActivated + " leaf queues"
+              LOG.debug("Parent queue = " + managedParentQueue.getQueueName()
+                  +  " : Found " + maxLeafQueuesTobeActivated + " leaf queues"
                   + " to be activated with " + pendingApps.size() + " apps ");
             }
 
@@ -424,8 +438,9 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
 
             if (LOG.isDebugEnabled()) {
               if (leafQueuesToBeActivated.size() > 0) {
-                LOG.debug("Activated leaf queues : [" + leafQueuesToBeActivated
-                    + "]");
+                LOG.debug("Activated leaf queues : [{}]",
+                    leafQueuesToBeActivated.size() < 25 ?
+                    leafQueuesToBeActivated : leafQueuesToBeActivated.size());
               }
             }
           }
@@ -492,8 +507,9 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
         String partition = e.getKey();
         if (!newPartitions.contains(partition)) {
           itr.remove();
-          LOG.info(
-              "Removed partition " + partition + " from leaf queue " + "state");
+          LOG.info(managedParentQueue.getQueueName()  +
+              " : Removed partition " + partition + " from leaf queue " +
+              "state");
         } else{
           Map<String, LeafQueueStatePerPartition> queues = e.getValue();
           for (
@@ -502,7 +518,9 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
             String queue = queueItr.next().getKey();
             if (!newQueues.contains(queue)) {
               queueItr.remove();
-              LOG.info("Removed queue " + queue + " from leaf queue "
+              LOG.info(managedParentQueue.getQueueName() + " : Removed queue"
+                  + queue + " from "
+                  + "leaf queue "
                   + "state from partition " + partition);
             }
           }
@@ -582,12 +600,6 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
           updateToZeroCapacity(capacities, nodeLabel);
           deactivatedQueues.put(leafQueue.getQueueName(),
               leafQueueTemplateCapacities);
-        } else{
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(" Leaf queue has pending applications or is " + "inactive"
-                + " : " + leafQueue.getNumApplications()
-                + ".Skipping deactivation for " + leafQueue);
-          }
         }
       } else{
         LOG.warn("Could not find queue in scheduler while trying" + " to "
@@ -659,7 +671,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
             if (isActive(leafQueue, nodeLabel)) {
               if (LOG.isDebugEnabled()) {
                 LOG.debug("Queue is already active." + " Skipping activation : "
-                    + queue.getQueuePath());
+                    + leafQueue.getQueueName());
               }
             } else{
               activate(leafQueue, nodeLabel);
@@ -668,7 +680,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
             if (!isActive(leafQueue, nodeLabel)) {
               if (LOG.isDebugEnabled()) {
                 LOG.debug("Queue is already de-activated. Skipping "
-                    + "de-activation : " + leafQueue.getQueuePath());
+                    + "de-activation : " + leafQueue.getQueueName());
               }
             } else{
               deactivate(leafQueue, nodeLabel);


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


[32/50] [abbrv] hadoop git commit: YARN-8520. Document best practice for user management. Contributed by Eric Yang

Posted by tm...@apache.org.
YARN-8520. Document best practice for user management. Contributed by Eric Yang


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

Branch: refs/heads/HADOOP-15407
Commit: e7951c69cbc85604f72cdd3559122d4e2c1ea127
Parents: a28624d
Author: Shane Kumpf <sk...@apache.org>
Authored: Fri Aug 10 14:32:03 2018 -0600
Committer: Shane Kumpf <sk...@apache.org>
Committed: Fri Aug 10 14:32:03 2018 -0600

----------------------------------------------------------------------
 .../src/site/markdown/DockerContainers.md       | 194 ++++++++++++++++++-
 1 file changed, 193 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7951c69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
index d435495..447155c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
@@ -296,7 +296,8 @@ owner as the container user.  If the application owner is not a valid user
 in the Docker image, the application will fail. The container user is specified
 by the user's UID. If the user's UID is different between the NodeManager host
 and the Docker image, the container may be launched as the wrong user or may
-fail to launch because the UID does not exist.
+fail to launch because the UID does not exist.  See
+[User Management in Docker Container](#user-management) section for more details.
 
 Second, the Docker image must have whatever is expected by the application
 in order to execute.  In the case of Hadoop (MapReduce or Spark), the Docker
@@ -412,6 +413,197 @@ the environment variable would be set to "/sys/fs/cgroup:/sys/fs/cgroup:ro".
 The destination path is not restricted, "/sys/fs/cgroup:/cgroup:ro" would also
 be valid given the example admin whitelist.
 
+<a href="#user-management"></a>User Management in Docker Container
+-----------------------------------
+
+YARN's Docker container support launches container processes using the uid:gid
+identity of the user, as defined on the NodeManager host. User and group name
+mismatches between the NodeManager host and container can lead to permission
+issues, failed container launches, or even security holes. Centralizing user and
+group management for both hosts and containers greatly reduces these risks. When
+running containerized applications on YARN, it is necessary to understand which
+uid:gid pair will be used to launch the container's process.
+
+As an example of what is meant by uid:gid pair, consider the following. By
+default, in non-secure mode, YARN will launch processes as the user `nobody`
+(see the table at the bottom of
+[Using CGroups with YARN](./NodeManagerCgroups.html) for how the run as user is
+determined in non-secure mode). On CentOS based systems, the `nobody` user's uid
+is `99` and the `nobody` group is `99`. As a result, YARN will call `docker run`
+with `--user 99:99`. If the `nobody` user does not have the uid `99` in the
+container, the launch may fail or have unexpected results.
+
+One exception to this rule is the use of Privileged Docker containers.
+Privileged containers will not set the uid:gid pair when launching the container
+and will honor the USER or GROUP entries in the Dockerfile. This allows running
+privileged containers as any user which has security implications. Please
+understand these implications before enabling Privileged Docker containers.
+
+There are many ways to address user and group management. Docker, by default,
+will authenticate users against `/etc/passwd` (and `/etc/shadow`) within the
+container. Using the default `/etc/passwd` supplied in the Docker image is
+unlikely to contain the appropriate user entries and will result in launch
+failures. It is highly recommended to centralize user and group management.
+Several approaches to user and group management are outlined below.
+
+### Static user management
+
+The most basic approach to managing user and groups is to modify the user and
+group within the Docker image. This approach is only viable in non-secure mode
+where all container processes will be launched as a single known user, for
+instance `nobody`. In this case, the only requirement is that the uid:gid pair
+of the nobody user and group must match between the host and container. On a
+CentOS based system, this means that the nobody user in the container needs the
+UID `99` and the nobody group in the container needs GID `99`.
+
+One approach to change the UID and GID is by leveraging `usermod` and
+`groupmod`. The following sets the correct UID and GID for the nobody
+user/group.
+```
+usermod -u 99 nobody
+groupmod -g 99 nobody
+```
+
+This approach is not recommended beyond testing given the inflexibility to add
+users.
+
+### Bind mounting
+
+When organizations already have automation in place to create local users on
+each system, it may be appropriate to bind mount /etc/passwd and /etc/group
+into the container as an alternative to modifying the container image directly.
+To enable the ability to bind mount /etc/passwd and /etc/group, update
+`docker.allowed.ro-mounts` in `container-executor.cfg` to include those paths.
+When submitting the application, `YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS` will
+need to include `/etc/passwd:/etc/passwd:ro` and `/etc/group:/etc/group:ro`.
+
+There are several challenges with this bind mount approach that need to be
+considered.
+
+1. Any users and groups defined in the image will be overwritten by the host's users and groups
+2. No users and groups can be added once the container is started, as /etc/passwd and /etc/group are immutible in the container. Do not mount these read-write as it can render the host inoperable.
+
+This approach is not recommended beyond testing given the inflexibility to
+modify running containers.
+
+### SSSD
+
+An alternative approach that allows for centrally managing users and groups is
+SSSD. System Security Services Daemon (SSSD) provides access to different
+identity and authentication providers, such as LDAP or Active Directory.
+
+The traditional schema for Linux authentication is as follows:
+```
+application -> libpam -> pam_authenticate -> pam_unix.so -> /etc/passwd
+```
+
+If we use SSSD for user lookup, it becomes:
+```
+application -> libpam -> pam_authenticate -> pam_sss.so -> SSSD -> pam_unix.so -> /etc/passwd
+```
+
+We can bind-mount the UNIX sockets SSSD communicates over into the container.
+This will allow the SSSD client side libraries to authenticate against the SSSD
+running on the host. As a result, user information does not need to exist in
+/etc/passwd of the docker image and will instead be serviced by SSSD.
+
+Step by step configuration for host and container:
+
+1. Host config
+
+   - Install packages
+     ```
+     # yum -y install sssd-common sssd-proxy
+     ```
+   - create a PAM service for the container.
+     ```
+     # cat /etc/pam.d/sss_proxy
+     auth required pam_unix.so
+     account required pam_unix.so
+     password required pam_unix.so
+     session required pam_unix.so
+     ```
+   - create SSSD config file, /etc/sssd/sssd.conf
+     Please note that the permissions must be 0600 and the file must be owned by root:root.
+     ```
+     # cat /etc/sssd/sssd/conf
+     [sssd]
+     services = nss,pam
+     config_file_version = 2
+     domains = proxy
+     [nss]
+     [pam]
+     [domain/proxy]
+     id_provider = proxy
+     proxy_lib_name = files
+     proxy_pam_target = sss_proxy
+     ```
+   - start sssd
+     ```
+     # systemctl start sssd
+     ```
+   - verify a user can be retrieved with sssd
+     ```
+     # getent passwd -s sss localuser
+     ```
+
+2. Container setup
+
+   It's important to bind-mount the /var/lib/sss/pipes directory from the host to the container since SSSD UNIX sockets are located there.
+   ```
+   -v /var/lib/sss/pipes:/var/lib/sss/pipes:rw
+   ```
+
+3. Container config
+
+   All the steps below should be executed on the container itself.
+
+   - Install only the sss client libraries
+     ```
+     # yum -y install sssd-client
+     ```
+
+   - make sure sss is configured for passwd and group databases in
+     ```
+     /etc/nsswitch.conf
+     ```
+
+   - configure the PAM service that the application uses to call into SSSD
+     ```
+     # cat /etc/pam.d/system-auth
+     #%PAM-1.0
+     # This file is auto-generated.
+     # User changes will be destroyed the next time authconfig is run.
+     auth        required      pam_env.so
+     auth        sufficient    pam_unix.so try_first_pass nullok
+     auth        sufficient    pam_sss.so forward_pass
+     auth        required      pam_deny.so
+
+     account     required      pam_unix.so
+     account     [default=bad success=ok user_unknown=ignore] pam_sss.so
+     account     required      pam_permit.so
+
+     password    requisite     pam_pwquality.so try_first_pass local_users_only retry=3 authtok_type=
+     password    sufficient    pam_unix.so try_first_pass use_authtok nullok sha512 shadow
+     password    sufficient    pam_sss.so use_authtok
+     password    required      pam_deny.so
+
+     session     optional      pam_keyinit.so revoke
+     session     required      pam_limits.so
+     -session     optional      pam_systemd.so
+     session     [success=1 default=ignore] pam_succeed_if.so service in crond quiet use_uid
+     session     required      pam_unix.so
+     session     optional      pam_sss.so
+     ```
+
+   - Save the docker image and use the docker image as base image for your applications.
+
+   - test the docker image launched in YARN environment.
+     ```
+     $ id
+     uid=5000(localuser) gid=5000(localuser) groups=5000(localuser),1337(hadoop)
+     ```
+
 Privileged Container Security Consideration
 -------------------------------------------
 


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