Pārlūkot izejas kodu

Merge branch 'trunk' into HDFS-7240

Anu Engineer 7 gadi atpakaļ
vecāks
revīzija
e319be93b9
49 mainītis faili ar 2215 papildinājumiem un 754 dzēšanām
  1. 115 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  2. 37 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java
  3. 41 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java
  4. 30 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java
  5. 39 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java
  6. 14 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
  7. 111 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  8. 3 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
  9. 0 34
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
  10. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  11. 20 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
  12. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
  13. 54 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  14. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  15. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  16. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  17. 98 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
  18. 1 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
  19. 67 6
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
  20. 801 500
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
  21. 62 15
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
  22. 3 2
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
  23. 6 4
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
  24. 8 2
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java
  25. 2 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
  26. 6 6
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
  27. 11 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java
  28. 34 0
      hadoop-tools/hadoop-azure/src/site/markdown/index.md
  29. 2 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
  30. 4 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
  31. 266 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java
  32. 1 0
      hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
  33. 15 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  34. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
  35. 36 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java
  36. 69 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
  37. 29 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
  38. 56 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
  39. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
  40. 1 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
  41. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
  42. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
  43. 52 53
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
  44. 16 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
  45. 33 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
  46. 11 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
  47. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
  48. 8 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
  49. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java

+ 115 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -259,7 +259,18 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    */
   private static final WeakHashMap<Configuration,Object> REGISTRY = 
     new WeakHashMap<Configuration,Object>();
-  
+
+  /**
+   * Map to register all classes holding property tag enums.
+   */
+  private static final Map<String, Class>
+      REGISTERED_TAG_CLASS = new HashMap<>();
+  /**
+   * Map to hold properties by there tag groupings.
+   */
+  private final Map<PropertyTag, Properties> propertyTagsMap =
+      new ConcurrentHashMap<>();
+
   /**
    * List of default Resources. Resources are loaded in the order of the list 
    * entries
@@ -738,6 +749,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   public Configuration(boolean loadDefaults) {
     this.loadDefaults = loadDefaults;
     updatingResource = new ConcurrentHashMap<String, String[]>();
+
+    // Register all classes holding property tags with
+    REGISTERED_TAG_CLASS.put("core", CorePropertyTag.class);
+    REGISTERED_TAG_CLASS.put("hdfs", HDFSPropertyTag.class);
+    REGISTERED_TAG_CLASS.put("yarn", YarnPropertyTag.class);
+
     synchronized(Configuration.class) {
       REGISTRY.put(this, null);
     }
@@ -765,6 +782,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
      this.finalParameters = Collections.newSetFromMap(
          new ConcurrentHashMap<String, Boolean>());
      this.finalParameters.addAll(other.finalParameters);
+     this.REGISTERED_TAG_CLASS.putAll(other.REGISTERED_TAG_CLASS);
+     this.propertyTagsMap.putAll(other.propertyTagsMap);
    }
    
     synchronized(Configuration.class) {
@@ -2823,6 +2842,10 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
               } else if ("source".equals(propertyAttr)) {
                 confSource.add(StringInterner.weakIntern(
                     reader.getAttributeValue(i)));
+              } else if ("tag".equals(propertyAttr)) {
+                //Read tags and put them in propertyTagsMap
+                readTagFromConfig(reader.getAttributeValue(i), confName,
+                    confValue, confSource);
               }
             }
             break;
@@ -2830,6 +2853,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           case "value":
           case "final":
           case "source":
+          case "tag":
             parseToken = true;
             token.setLength(0);
             break;
@@ -2911,6 +2935,13 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           case "source":
             confSource.add(StringInterner.weakIntern(token.toString()));
             break;
+          case "tag":
+            if (token.length() > 0) {
+              //Read tags and put them in propertyTagsMap
+              readTagFromConfig(token.toString(), confName,
+                  confValue, confSource);
+            }
+            break;
           case "include":
             if (fallbackAllowed && !fallbackEntered) {
               throw new IOException("Fetch fail on include for '"
@@ -2962,6 +2993,48 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
 
+  private void readTagFromConfig(String attributeValue, String confName, String
+      confValue, List<String> confSource) {
+    for (String tagStr : attributeValue.split(",")) {
+      tagStr = tagStr.trim();
+      try {
+        if (confSource.size() > 0) {
+          for (String source : confSource) {
+            PropertyTag tag1 = this.getPropertyTag(tagStr,
+                source.split("-")[0]);
+            if (propertyTagsMap.containsKey(tag1)) {
+              propertyTagsMap.get(tag1)
+                  .setProperty(confName, confValue);
+            } else {
+              Properties props = new Properties();
+              props.setProperty(confName, confValue);
+              propertyTagsMap.put(tag1, props);
+            }
+          }
+        } else {
+          //If no source is set try to find tag in CorePropertyTag
+          if (propertyTagsMap
+              .containsKey(CorePropertyTag.valueOf(tagStr)
+              )) {
+            propertyTagsMap.get(CorePropertyTag.valueOf(tagStr))
+                .setProperty(confName, confValue);
+          } else {
+            Properties props = new Properties();
+            props.setProperty(confName, confValue);
+            propertyTagsMap.put(CorePropertyTag.valueOf(tagStr),
+                props);
+          }
+        }
+      } catch (IllegalArgumentException iae) {
+        //Log the invalid tag and continue to parse rest of the
+        // properties.
+        LOG.info("Invalid tag '" + tagStr + "' found for "
+            + "property:" + confName, iae);
+      }
+
+    }
+  }
+
   private void overlay(Properties to, Properties from) {
     for (Entry<Object, Object> entry: from.entrySet()) {
       to.put(entry.getKey(), entry.getValue());
@@ -3438,4 +3511,45 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
     return false;
   }
+
+  /**
+   * Get all properties belonging to tag.
+   * @return Properties with matching properties
+   */
+  public Properties getAllPropertiesByTag(final PropertyTag tag) {
+    Properties props = new Properties();
+    if (propertyTagsMap.containsKey(tag)) {
+      props.putAll(propertyTagsMap.get(tag));
+    }
+    return props;
+  }
+
+  /**
+   * Get all properties belonging to list of input tags. Calls
+   * getAllPropertiesByTag internally.
+   *
+   * @return Properties with all matching properties
+   */
+  public Properties getAllPropertiesByTags(final List<PropertyTag> tagList) {
+    Properties prop = new Properties();
+    for (PropertyTag tag : tagList) {
+      prop.putAll(this.getAllPropertiesByTag(tag));
+    }
+    return prop;
+  }
+
+  /**
+   * Get Property tag Enum corresponding to given source.
+   *
+   * @param tagStr String representation of Enum
+   * @param group Group to which enum belongs.Ex hdfs,yarn
+   * @return Properties with all matching properties
+   */
+  private PropertyTag getPropertyTag(String tagStr, String group) {
+    PropertyTag tag = null;
+    if (REGISTERED_TAG_CLASS.containsKey(group)) {
+      tag = (PropertyTag) Enum.valueOf(REGISTERED_TAG_CLASS.get(group), tagStr);
+    }
+    return tag;
+  }
 }

+ 37 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***************************************************************************
+ * Enum for tagging hadoop core properties according to there usage.
+ * CorePropertyTag implements the
+ * {@link org.apache.hadoop.conf.PropertyTag} interface,
+ ***************************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum CorePropertyTag implements PropertyTag {
+  CORE,
+  REQUIRED,
+  PERFORMANCE,
+  CLIENT,
+  SERVER,
+  SECURITY,
+  DEBUG
+}

+ 41 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***************************************************************************
+ * Enum for tagging hdfs properties according to there usage or application.
+ * HDFSPropertyTag implements the
+ * {@link org.apache.hadoop.conf.PropertyTag} interface,
+ ***************************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum HDFSPropertyTag implements PropertyTag {
+  HDFS,
+  NAMENODE,
+  DATANODE,
+  REQUIRED,
+  SECURITY,
+  KERBEROS,
+  PERFORMANCE,
+  CLIENT,
+  SERVER,
+  DEBUG,
+  DEPRICATED
+}

+ 30 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***********************************************************
+ * PropertyTag is used for creating extendable property tag Enums.
+ * Property tags will group related properties together.
+ ***********************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface PropertyTag {
+
+}

+ 39 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***************************************************************************
+ * Enum for tagging yarn properties according to there usage or application.
+ * YarnPropertyTag implements the
+ * {@link org.apache.hadoop.conf.PropertyTag} interface,
+ ***************************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum YarnPropertyTag implements PropertyTag {
+  YARN,
+  RESOURCEMANAGER,
+  SECURITY,
+  KERBEROS,
+  PERFORMANCE,
+  CLIENT,
+  REQUIRED,
+  SERVER,
+  DEBUG
+}

+ 14 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java

@@ -25,11 +25,22 @@ import java.util.concurrent.TimeUnit;
  * A simplified StopWatch implementation which can measure times in nanoseconds.
  */
 public class StopWatch implements Closeable {
+  private final Timer timer;
   private boolean isStarted;
   private long startNanos;
   private long currentElapsedNanos;
 
   public StopWatch() {
+    this(new Timer());
+  }
+
+  /**
+   * Used for tests to be able to create a StopWatch which does not follow real
+   * time.
+   * @param timer The timer to base this StopWatch's timekeeping off of.
+   */
+  public StopWatch(Timer timer) {
+    this.timer = timer;
   }
 
   /**
@@ -49,7 +60,7 @@ public class StopWatch implements Closeable {
       throw new IllegalStateException("StopWatch is already running");
     }
     isStarted = true;
-    startNanos = System.nanoTime();
+    startNanos = timer.monotonicNowNanos();
     return this;
   }
 
@@ -61,7 +72,7 @@ public class StopWatch implements Closeable {
     if (!isStarted) {
       throw new IllegalStateException("StopWatch is already stopped");
     }
-    long now = System.nanoTime();
+    long now = timer.monotonicNowNanos();
     isStarted = false;
     currentElapsedNanos += now - startNanos;
     return this;
@@ -90,7 +101,7 @@ public class StopWatch implements Closeable {
    */
   public long now() {
     return isStarted ?
-        System.nanoTime() - startNanos + currentElapsedNanos :
+        timer.monotonicNowNanos() - startNanos + currentElapsedNanos :
         currentElapsedNanos;
   }
 

+ 111 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -38,6 +38,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 import java.util.regex.Pattern;
@@ -53,7 +54,6 @@ import static org.junit.Assert.assertArrayEquals;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
@@ -135,6 +135,7 @@ public class TestConfiguration {
 
   private void endConfig() throws IOException{
     out.write("</configuration>\n");
+    out.flush();
     out.close();
   }
 
@@ -577,6 +578,34 @@ public class TestConfiguration {
     out.write("</property>\n");
   }
 
+  private void appendPropertyByTag(String name, String val, String tags,
+      String... sources) throws IOException {
+    appendPropertyByTag(name, val, false, tags, sources);
+  }
+
+  private void appendPropertyByTag(String name, String val, boolean isFinal,
+      String tag, String... sources) throws IOException {
+    out.write("<property>");
+    out.write("<name>");
+    out.write(name);
+    out.write("</name>");
+    out.write("<value>");
+    out.write(val);
+    out.write("</value>");
+    if (isFinal) {
+      out.write("<final>true</final>");
+    }
+    for (String s : sources) {
+      out.write("<source>");
+      out.write(s);
+      out.write("</source>");
+    }
+    out.write("<tag>");
+    out.write(tag);
+    out.write("</tag>");
+    out.write("</property>\n");
+  }
+
   void appendCompactFormatProperty(String name, String val) throws IOException {
     appendCompactFormatProperty(name, val, false);
   }
@@ -2215,4 +2244,85 @@ public class TestConfiguration {
       TestConfiguration.class.getName()
     });
   }
+
+  @Test
+  public void testGetAllPropertiesByTags() throws Exception {
+
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG");
+    appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,REQUIRED");
+    appendPropertyByTag("dfs.namenode.logging.level", "INFO", "CLIENT,DEBUG");
+    endConfig();
+
+    Path fileResource = new Path(CONFIG);
+    conf.addResource(fileResource);
+    conf.getProps();
+
+    List<PropertyTag> tagList = new ArrayList<>();
+    tagList.add(CorePropertyTag.REQUIRED);
+    tagList.add(CorePropertyTag.PERFORMANCE);
+    tagList.add(CorePropertyTag.DEBUG);
+    tagList.add(CorePropertyTag.CLIENT);
+
+    Properties properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(3, properties.size());
+    assertEq(true, properties.containsKey("dfs.namenode.logging.level"));
+    assertEq(true, properties.containsKey("dfs.replication"));
+    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(false, properties.containsKey("namenode.host"));
+  }
+
+  @Test
+  public void testGetAllPropertiesWithSourceByTags() throws Exception {
+
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG",
+        "hdfs-default.xml", "core-site.xml");
+    appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,HDFS",
+        "hdfs-default.xml");
+    appendPropertyByTag("yarn.resourcemanager.work-preserving-recovery"
+        + ".enabled", "INFO", "CLIENT,DEBUG", "yarn-default.xml", "yarn-site"
+        + ".xml");
+    endConfig();
+
+    Path fileResource = new Path(CONFIG);
+    conf.addResource(fileResource);
+    conf.getProps();
+
+    List<PropertyTag> tagList = new ArrayList<>();
+    tagList.add(CorePropertyTag.REQUIRED);
+
+    Properties properties;
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertNotEquals(3, properties.size());
+
+    tagList.add(HDFSPropertyTag.DEBUG);
+    tagList.add(YarnPropertyTag.CLIENT);
+    tagList.add(HDFSPropertyTag.PERFORMANCE);
+    tagList.add(HDFSPropertyTag.HDFS);
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(3, properties.size());
+
+    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(true, properties.containsKey("dfs.replication"));
+    assertEq(true, properties
+        .containsKey("yarn.resourcemanager.work-preserving-recovery.enabled"));
+    assertEq(false, properties.containsKey("namenode.host"));
+
+    tagList.clear();
+    tagList.add(HDFSPropertyTag.DEBUG);
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(false, properties.containsKey("yarn.resourcemanager"
+        + ".work-preserving-recovery"));
+
+    tagList.clear();
+    tagList.add(YarnPropertyTag.DEBUG);
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(false, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(true, properties.containsKey("yarn.resourcemanager"
+        + ".work-preserving-recovery.enabled"));
+  }
 }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
+
 /**
  * A response of add an ErasureCoding policy.
  */
@@ -38,7 +40,7 @@ public class AddECPolicyResponse {
   }
 
   public AddECPolicyResponse(ErasureCodingPolicy policy,
-      IllegalECPolicyException e) {
+      HadoopIllegalArgumentException e) {
     this(policy, e.getMessage());
   }
 

+ 0 - 34
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java

@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * An Exception indicates the error when adding an ErasureCoding policy.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class IllegalECPolicyException extends Exception {
-  static final long serialVersionUID = 1L;
-
-  public IllegalECPolicyException(String msg) {
-    super(msg);
-  }
-}

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -641,6 +641,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY = "dfs.datanode.min.supported.namenode.version";
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "2.1.0-beta";
   public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY = "dfs.namenode.inode.attributes.provider.class";
+  public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY = "dfs.namenode.inode.attributes.provider.bypass.users";
+  public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_DEFAULT = "";
+
   public static final String  DFS_DATANODE_BP_READY_TIMEOUT_KEY = "dfs.datanode.bp-ready.timeout";
   public static final long    DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT = 20;
 
@@ -924,7 +927,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_JOURNALNODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY = "dfs.journalnode.kerberos.internal.spnego.principal";
   public static final String DFS_JOURNALNODE_ENABLE_SYNC_KEY =
       "dfs.journalnode.enable.sync";
-  public static final boolean DFS_JOURNALNODE_ENABLE_SYNC_DEFAULT = false;
+  public static final boolean DFS_JOURNALNODE_ENABLE_SYNC_DEFAULT = true;
   public static final String DFS_JOURNALNODE_SYNC_INTERVAL_KEY =
       "dfs.journalnode.sync.interval";
   public static final long DFS_JOURNALNODE_SYNC_INTERVAL_DEFAULT = 2*60*1000L;

+ 20 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java

@@ -19,11 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -144,7 +144,7 @@ public final class ErasureCodingPolicyManager {
             policyName,
             DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
             names);
-        throw new IllegalArgumentException(msg);
+        throw new HadoopIllegalArgumentException(msg);
       }
       enabledPoliciesByName.put(ecPolicy.getName(), ecPolicy);
     }
@@ -230,33 +230,34 @@ public final class ErasureCodingPolicyManager {
    * Add an erasure coding policy.
    * @return the added policy
    */
-  public synchronized ErasureCodingPolicy addPolicy(ErasureCodingPolicy policy)
-      throws IllegalECPolicyException {
+  public synchronized ErasureCodingPolicy addPolicy(
+      ErasureCodingPolicy policy) {
     // Set policy state into DISABLED when adding into Hadoop.
     policy.setState(ErasureCodingPolicyState.DISABLED);
 
     if (!CodecUtil.hasCodec(policy.getCodecName())) {
-      throw new IllegalECPolicyException("Codec name "
+      throw new HadoopIllegalArgumentException("Codec name "
           + policy.getCodecName() + " is not supported");
     }
 
     if (policy.getCellSize() > maxCellSize) {
-      throw new IllegalECPolicyException("Cell size " + policy.getCellSize()
-          + " should not exceed maximum " + maxCellSize + " byte");
+      throw new HadoopIllegalArgumentException("Cell size " +
+          policy.getCellSize() + " should not exceed maximum " +
+          maxCellSize + " bytes");
     }
 
     String assignedNewName = ErasureCodingPolicy.composePolicyName(
         policy.getSchema(), policy.getCellSize());
     for (ErasureCodingPolicy p : getPolicies()) {
       if (p.getName().equals(assignedNewName)) {
-        throw new IllegalECPolicyException("The policy name " + assignedNewName
-            + " already exists");
+        throw new HadoopIllegalArgumentException("The policy name " +
+            assignedNewName + " already exists");
       }
       if (p.getSchema().equals(policy.getSchema()) &&
           p.getCellSize() == policy.getCellSize()) {
-        throw new IllegalECPolicyException("A policy with same schema "
+        throw new HadoopIllegalArgumentException("A policy with same schema "
             + policy.getSchema().toString() + " and cell size "
-            + p.getCellSize() + " is already exists");
+            + p.getCellSize() + " already exists");
       }
     }
     policy.setName(assignedNewName);
@@ -281,12 +282,12 @@ public final class ErasureCodingPolicyManager {
   public synchronized void removePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     if (ecPolicy.isSystemPolicy()) {
-      throw new IllegalArgumentException("System erasure coding policy " +
+      throw new HadoopIllegalArgumentException("System erasure coding policy " +
           name + " cannot be removed");
     }
 
@@ -317,8 +318,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void disablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     if (enabledPoliciesByName.containsKey(name)) {
@@ -336,8 +337,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void enablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     enabledPoliciesByName.put(name, ecPolicy);
@@ -346,4 +347,4 @@ public final class ErasureCodingPolicyManager {
         enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
     LOG.info("Enable the erasure coding policy " + name);
   }
-}
+}

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

@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -212,7 +211,7 @@ final class FSDirErasureCodingOp {
   }
 
   static ErasureCodingPolicy addErasureCodePolicy(final FSNamesystem fsn,
-      ErasureCodingPolicy policy) throws IllegalECPolicyException {
+      ErasureCodingPolicy policy) {
     Preconditions.checkNotNull(policy);
     return fsn.getErasureCodingPolicyManager().addPolicy(policy);
   }

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

@@ -74,6 +74,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
@@ -202,6 +203,10 @@ public class FSDirectory implements Closeable {
 
   private INodeAttributeProvider attributeProvider;
 
+  // A HashSet of principals of users for whom the external attribute provider
+  // will be bypassed
+  private HashSet<String> usersToBypassExtAttrProvider = null;
+
   public void setINodeAttributeProvider(INodeAttributeProvider provider) {
     attributeProvider = provider;
   }
@@ -357,6 +362,49 @@ public class FSDirectory implements Closeable {
     this.quotaInitThreads = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_KEY,
         DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_DEFAULT);
+
+    initUsersToBypassExtProvider(conf);
+  }
+
+  private void initUsersToBypassExtProvider(Configuration conf) {
+    String[] bypassUsers = conf.getTrimmedStrings(
+        DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY,
+        DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_DEFAULT);
+    for(int i = 0; i < bypassUsers.length; i++) {
+      String tmp = bypassUsers[i].trim();
+      if (!tmp.isEmpty()) {
+        if (usersToBypassExtAttrProvider == null) {
+          usersToBypassExtAttrProvider = new HashSet<String>();
+        }
+        LOG.info("Add user " + tmp + " to the list that will bypass external"
+            + " attribute provider.");
+        usersToBypassExtAttrProvider.add(tmp);
+      }
+    }
+  }
+
+  /**
+   * Check if a given user is configured to bypass external attribute provider.
+   * @param user user principal
+   * @return true if the user is to bypass external attribute provider
+   */
+  private boolean isUserBypassingExtAttrProvider(final String user) {
+    return (usersToBypassExtAttrProvider != null) &&
+          usersToBypassExtAttrProvider.contains(user);
+  }
+
+  /**
+   * Return attributeProvider or null if ugi is to bypass attributeProvider.
+   * @param ugi
+   * @return configured attributeProvider or null
+   */
+  private INodeAttributeProvider getUserFilteredAttributeProvider(
+      UserGroupInformation ugi) {
+    if (attributeProvider == null ||
+        (ugi != null && isUserBypassingExtAttrProvider(ugi.getUserName()))) {
+      return null;
+    }
+    return attributeProvider;
   }
 
   /**
@@ -1711,7 +1759,7 @@ public class FSDirectory implements Closeable {
   FSPermissionChecker getPermissionChecker(String fsOwner, String superGroup,
       UserGroupInformation ugi) throws AccessControlException {
     return new FSPermissionChecker(
-        fsOwner, superGroup, ugi, attributeProvider);
+        fsOwner, superGroup, ugi, getUserFilteredAttributeProvider(ugi));
   }
 
   void checkOwner(FSPermissionChecker pc, INodesInPath iip)
@@ -1896,18 +1944,20 @@ public class FSDirectory implements Closeable {
   }
 
   INodeAttributes getAttributes(INodesInPath iip)
-      throws FileNotFoundException {
+      throws IOException {
     INode node = FSDirectory.resolveLastINode(iip);
     int snapshot = iip.getPathSnapshotId();
     INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
-    if (attributeProvider != null) {
+    UserGroupInformation ugi = NameNode.getRemoteUser();
+    INodeAttributeProvider ap = this.getUserFilteredAttributeProvider(ugi);
+    if (ap != null) {
       // permission checking sends the full components array including the
       // first empty component for the root.  however file status
       // related calls are expected to strip out the root component according
       // to TestINodeAttributeProvider.
       byte[][] components = iip.getPathComponents();
       components = Arrays.copyOfRange(components, 1, components.length);
-      nodeAttrs = attributeProvider.getAttributes(components, nodeAttrs);
+      nodeAttrs = ap.getAttributes(components, nodeAttrs);
     }
     return nodeAttrs;
   }

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

@@ -201,7 +201,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -7207,7 +7206,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
           addECPolicyName = newPolicy.getName();
           responses.add(new AddECPolicyResponse(newPolicy));
-        } catch (IllegalECPolicyException e) {
+        } catch (HadoopIllegalArgumentException e) {
           responses.add(new AddECPolicyResponse(policy, e));
         }
       }

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -3933,7 +3933,7 @@
 
 <property>
   <name>dfs.journalnode.enable.sync</name>
-  <value>false</value>
+  <value>true</value>
   <description>
     If true, the journal nodes wil sync with each other. The journal nodes
     will periodically gossip with other journal nodes to compare edit log
@@ -4159,6 +4159,18 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.authorization.provider.bypass.users</name>
+  <value></value>
+  <description>
+    A list of user principals (in secure cluster) or user names (in insecure
+    cluster) for whom the external attribute provider will be bypassed for all
+    operations. This means file attributes stored in HDFS instead of the
+    external provider will be used for permission checking and be returned when
+    requested.
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.max-num-blocks-to-log</name>
   <value>1000</value>

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

@@ -1617,7 +1617,7 @@ public class TestDistributedFileSystem {
         fs.enableErasureCodingPolicy("notExistECName");
         Assert.fail("enable the policy that doesn't exist should fail");
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
       }
 
@@ -1626,7 +1626,7 @@ public class TestDistributedFileSystem {
         fs.disableErasureCodingPolicy("notExistECName");
         Assert.fail("disable the policy that doesn't exist should fail");
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
       }
 

+ 98 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java

@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableList;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -33,19 +34,25 @@ import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
 
 public class TestINodeAttributeProvider {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestINodeAttributeProvider.class);
+
   private MiniDFSCluster miniDFS;
   private static final Set<String> CALLED = new HashSet<String>();
+  private static final short HDFS_PERMISSION = 0777;
+  private static final short PROVIDER_PERMISSION = 0770;
 
   public static class MyAuthorizationProvider extends INodeAttributeProvider {
 
@@ -112,7 +119,8 @@ public class TestINodeAttributeProvider {
 
         @Override
         public long getPermissionLong() {
-          return (useDefault) ? inode.getPermissionLong() : 0770;
+          return (useDefault) ? inode.getPermissionLong() :
+            (long)PROVIDER_PERMISSION;
         }
 
         @Override
@@ -177,6 +185,9 @@ public class TestINodeAttributeProvider {
     conf.set(DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY,
         MyAuthorizationProvider.class.getName());
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    conf.set(
+        DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY,
+        " u2,, ,u3, ");
     EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
     miniDFS = new MiniDFSCluster.Builder(conf).build();
   }
@@ -195,8 +206,11 @@ public class TestINodeAttributeProvider {
   public void testDelegationToProvider() throws Exception {
     Assert.assertTrue(CALLED.contains("start"));
     FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
-    fs.mkdirs(new Path("/tmp"));
-    fs.setPermission(new Path("/tmp"), new FsPermission((short) 0777));
+    final Path tmpPath = new Path("/tmp");
+    final Path fooPath = new Path("/tmp/foo");
+
+    fs.mkdirs(tmpPath);
+    fs.setPermission(tmpPath, new FsPermission(HDFS_PERMISSION));
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting("u1",
         new String[]{"g1"});
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
@@ -204,17 +218,19 @@ public class TestINodeAttributeProvider {
       public Void run() throws Exception {
         FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
         CALLED.clear();
-        fs.mkdirs(new Path("/tmp/foo"));
+        fs.mkdirs(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         Assert.assertTrue(CALLED.contains("checkPermission|WRITE|null|null"));
+
         CALLED.clear();
-        fs.listStatus(new Path("/tmp/foo"));
+        fs.listStatus(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(
             CALLED.contains("checkPermission|null|null|READ_EXECUTE"));
+
         CALLED.clear();
-        fs.getAclStatus(new Path("/tmp/foo"));
+        fs.getAclStatus(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         return null;
@@ -222,6 +238,81 @@ public class TestINodeAttributeProvider {
     });
   }
 
+  private class AssertHelper {
+    private boolean bypass = true;
+    AssertHelper(boolean bp) {
+      bypass = bp;
+    }
+    public void doAssert(boolean x) {
+      if (bypass) {
+        Assert.assertFalse(x);
+      } else {
+        Assert.assertTrue(x);
+      }
+    }
+  }
+
+  private void testBypassProviderHelper(final String[] users,
+      final short expectedPermission, final boolean bypass) throws Exception {
+    final AssertHelper asserter = new AssertHelper(bypass);
+
+    Assert.assertTrue(CALLED.contains("start"));
+
+    FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+    final Path userPath = new Path("/user");
+    final Path authz = new Path("/user/authz");
+    final Path authzChild = new Path("/user/authz/child2");
+
+    fs.mkdirs(userPath);
+    fs.setPermission(userPath, new FsPermission(HDFS_PERMISSION));
+    fs.mkdirs(authz);
+    fs.setPermission(authz, new FsPermission(HDFS_PERMISSION));
+    fs.mkdirs(authzChild);
+    fs.setPermission(authzChild, new FsPermission(HDFS_PERMISSION));
+    for(String user : users) {
+      UserGroupInformation ugiBypass =
+          UserGroupInformation.createUserForTesting(user,
+              new String[]{"g1"});
+      ugiBypass.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+          Assert.assertEquals(expectedPermission,
+              fs.getFileStatus(authzChild).getPermission().toShort());
+          asserter.doAssert(CALLED.contains("getAttributes"));
+          asserter.doAssert(CALLED.contains("checkPermission|null|null|null"));
+
+          CALLED.clear();
+          Assert.assertEquals(expectedPermission,
+              fs.listStatus(userPath)[0].getPermission().toShort());
+          asserter.doAssert(CALLED.contains("getAttributes"));
+          asserter.doAssert(
+              CALLED.contains("checkPermission|null|null|READ_EXECUTE"));
+
+          CALLED.clear();
+          fs.getAclStatus(authzChild);
+          asserter.doAssert(CALLED.contains("getAttributes"));
+          asserter.doAssert(CALLED.contains("checkPermission|null|null|null"));
+          return null;
+        }
+      });
+    }
+  }
+
+  @Test
+  public void testAuthzDelegationToProvider() throws Exception {
+    LOG.info("Test not bypassing provider");
+    String[] users = {"u1"};
+    testBypassProviderHelper(users, PROVIDER_PERMISSION, false);
+  }
+
+  @Test
+  public void testAuthzBypassingProvider() throws Exception {
+    LOG.info("Test bypassing provider");
+    String[] users = {"u2", "u3"};
+    testBypassProviderHelper(users, HDFS_PERMISSION, true);
+  }
+
   @Test
   public void testCustomProvider() throws Exception {
     FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java

@@ -152,7 +152,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     streamStatistics.streamOpened();
     try {
       GetObjectRequest request = new GetObjectRequest(bucket, key)
-          .withRange(targetPos, contentRangeFinish);
+          .withRange(targetPos, contentRangeFinish - 1);
       if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
           StringUtils.isNotBlank(serverSideEncryptionKey)){
         request.setSSECustomerKey(new SSECustomerKey(serverSideEncryptionKey));

+ 67 - 6
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java

@@ -202,6 +202,23 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    */
   private Set<String> pageBlobDirs;
 
+  /**
+   * Configuration key to indicate the set of directories in WASB where we
+   * should store files as block blobs with block compaction enabled.
+   *
+   * Entries can be directory paths relative to the container (e.g. "/path") or
+   * fully qualified wasb:// URIs (e.g.
+   * wasb://container@example.blob.core.windows.net/path)
+   */
+  public static final String KEY_BLOCK_BLOB_WITH_COMPACTION_DIRECTORIES =
+          "fs.azure.block.blob.with.compaction.dir";
+
+  /**
+   * The set of directories where we should store files as block blobs with
+   * block compaction enabled.
+   */
+  private Set<String> blockBlobWithCompationDirs;
+
   /**
    * Configuration key to indicate the set of directories in WASB where
    * we should do atomic folder rename synchronized with createNonRecursive.
@@ -527,6 +544,12 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     // User-agent
     userAgentId = conf.get(USER_AGENT_ID_KEY, USER_AGENT_ID_DEFAULT);
 
+    // Extract the directories that should contain block blobs with compaction
+    blockBlobWithCompationDirs = getDirectorySet(
+        KEY_BLOCK_BLOB_WITH_COMPACTION_DIRECTORIES);
+    LOG.debug("Block blobs with compaction directories:  {}",
+        setToString(blockBlobWithCompationDirs));
+
     // Extract directories that should have atomic rename applied.
     atomicRenameDirs = getDirectorySet(KEY_ATOMIC_RENAME_DIRECTORIES);
     String hbaseRoot;
@@ -1164,6 +1187,17 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     return isKeyForDirectorySet(key, pageBlobDirs);
   }
 
+  /**
+   * Checks if the given key in Azure Storage should be stored as a block blobs
+   * with compaction enabled instead of normal block blob.
+   *
+   * @param key blob name
+   * @return true, if the file is in directory with block compaction enabled.
+   */
+  public boolean isBlockBlobWithCompactionKey(String key) {
+    return isKeyForDirectorySet(key, blockBlobWithCompationDirs);
+  }
+
   /**
    * Checks if the given key in Azure storage should have synchronized
    * atomic folder rename createNonRecursive implemented.
@@ -1356,7 +1390,9 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   }
 
   @Override
-  public DataOutputStream storefile(String key, PermissionStatus permissionStatus)
+  public DataOutputStream storefile(String keyEncoded,
+                                    PermissionStatus permissionStatus,
+                                    String key)
       throws AzureException {
     try {
 
@@ -1417,12 +1453,26 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
       // Get the blob reference from the store's container and
       // return it.
-      CloudBlobWrapper blob = getBlobReference(key);
+      CloudBlobWrapper blob = getBlobReference(keyEncoded);
       storePermissionStatus(blob, permissionStatus);
 
       // Create the output stream for the Azure blob.
       //
-      OutputStream outputStream = openOutputStream(blob);
+      OutputStream outputStream;
+
+      if (isBlockBlobWithCompactionKey(key)) {
+        BlockBlobAppendStream blockBlobOutputStream = new BlockBlobAppendStream(
+            (CloudBlockBlobWrapper) blob,
+            keyEncoded,
+            this.uploadBlockSizeBytes,
+            true,
+            getInstrumentedContext());
+
+        outputStream = blockBlobOutputStream;
+      } else {
+        outputStream = openOutputStream(blob);
+      }
+
       DataOutputStream dataOutStream = new SyncableDataOutputStream(outputStream);
       return dataOutStream;
     } catch (Exception e) {
@@ -2869,10 +2919,21 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
       CloudBlobWrapper blob =  this.container.getBlockBlobReference(key);
 
-      BlockBlobAppendStream appendStream = new BlockBlobAppendStream((CloudBlockBlobWrapper) blob, key, bufferSize, getInstrumentedContext());
-      appendStream.initialize();
+      OutputStream outputStream;
+
+      BlockBlobAppendStream blockBlobOutputStream = new BlockBlobAppendStream(
+          (CloudBlockBlobWrapper) blob,
+          key,
+          bufferSize,
+          isBlockBlobWithCompactionKey(key),
+          getInstrumentedContext());
+
+      outputStream = blockBlobOutputStream;
+
+      DataOutputStream dataOutStream = new SyncableDataOutputStream(
+          outputStream);
 
-      return new DataOutputStream(appendStream);
+      return dataOutStream;
     } catch(Exception ex) {
       throw new AzureException(ex);
     }

+ 801 - 500
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java

@@ -22,122 +22,256 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.HashMap;
-import java.util.Locale;
+import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
 import java.util.Random;
-import java.util.TimeZone;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
 
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.azure.StorageInterface.CloudBlockBlobWrapper;
-import org.eclipse.jetty.util.log.Log;
+import org.apache.hadoop.io.ElasticByteBufferPool;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.microsoft.azure.storage.AccessCondition;
 import com.microsoft.azure.storage.OperationContext;
 import com.microsoft.azure.storage.StorageException;
+import com.microsoft.azure.storage.StorageErrorCodeStrings;
 import com.microsoft.azure.storage.blob.BlobRequestOptions;
 import com.microsoft.azure.storage.blob.BlockEntry;
 import com.microsoft.azure.storage.blob.BlockListingFilter;
+import com.microsoft.azure.storage.blob.BlockSearchMode;
+
+import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HFLUSH;
+import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HSYNC;
 
 /**
- * Stream object that implememnts append for Block Blobs in WASB.
+ * Stream object that implements append for Block Blobs in WASB.
+ *
+ * The stream object implements hflush/hsync and block compaction. Block
+ * compaction is the process of replacing a sequence of small blocks with one
+ * big block. Azure Block blobs supports up to 50000 blocks and every
+ * hflush/hsync generates one block. When the number of blocks is above 32000,
+ * the process of compaction decreases the total number of blocks, if possible.
+ * If compaction is disabled, hflush/hsync are empty functions.
+ *
+ * The stream object uses background threads for uploading the blocks and the
+ * block blob list. Blocks can be uploaded concurrently. However, when the block
+ * list is uploaded, block uploading should stop. If a block is uploaded before
+ * the block list and the block id is not in the list, the block will be lost.
+ * If the block is uploaded after the block list and the block id is in the
+ * list, the block list upload will fail. The exclusive access for the block
+ * list upload is managed by uploadingSemaphore.
  */
-public class BlockBlobAppendStream extends OutputStream {
 
+public class BlockBlobAppendStream extends OutputStream implements Syncable,
+    StreamCapabilities {
+
+  /**
+   * The name of the blob/file.
+   */
   private final String key;
-  private final int bufferSize;
-  private ByteArrayOutputStream outBuffer;
-  private final CloudBlockBlobWrapper blob;
-  private final OperationContext opContext;
 
   /**
-   * Variable to track if the stream has been closed.
+   * This variable tracks if this is new blob or existing one.
+   */
+  private boolean blobExist;
+
+  /**
+   * When the blob exist, to to prevent concurrent write we take a lease.
+   * Taking a lease is not necessary for new blobs.
    */
-  private boolean closed = false;
+  private SelfRenewingLease lease = null;
 
   /**
-   * Variable to track if the append lease is released.
+   * The support for process of compaction is optional.
    */
+  private final boolean compactionEnabled;
 
-  private volatile boolean leaseFreed;
+  /**
+   * The number of blocks above each block compaction is triggered.
+   */
+  private static final int DEFAULT_ACTIVATE_COMPACTION_BLOCK_COUNT = 32000;
 
   /**
-   * Variable to track if the append stream has been
-   * initialized.
+   * The number of blocks above each block compaction is triggered.
    */
+  private int activateCompactionBlockCount
+      = DEFAULT_ACTIVATE_COMPACTION_BLOCK_COUNT;
 
-  private boolean initialized = false;
+  /**
+   * The size of the output buffer. Writes store the data in outBuffer until
+   * either the size is above maxBlockSize or hflush/hsync is called.
+   */
+  private final AtomicInteger maxBlockSize;
 
   /**
-   *  Last IOException encountered
+   * The current buffer where writes are stored.
    */
-  private volatile IOException lastError = null;
+  private ByteBuffer outBuffer;
 
   /**
-   * List to keep track of the uncommitted azure storage
-   * block ids
+   * The size of the blob that has been successfully stored in the Azure Blob
+   * service.
    */
-  private final List<BlockEntry> uncommittedBlockEntries;
+  private final AtomicLong committedBlobLength = new AtomicLong(0);
 
-  private static final int UNSET_BLOCKS_COUNT = -1;
+  /**
+   * Position of last block in the blob.
+   */
+  private volatile long blobLength = 0;
 
   /**
-   * Variable to hold the next block id to be used for azure
-   * storage blocks.
+   * Minutes waiting before the close operation timed out.
    */
-  private long nextBlockCount = UNSET_BLOCKS_COUNT;
+  private static final int CLOSE_UPLOAD_DELAY = 10;
 
   /**
-   * Variable to hold the block id prefix to be used for azure
-   * storage blocks from azure-storage-java sdk version 4.2.0 onwards
+   * Keep alive time for the threadpool.
    */
-  private String blockIdPrefix = null;
+  private static final int THREADPOOL_KEEP_ALIVE = 30;
+  /**
+   * Azure Block Blob used for the stream.
+   */
+  private final CloudBlockBlobWrapper blob;
+
+  /**
+   * Azure Storage operation context.
+   */
+  private final OperationContext opContext;
+
+  /**
+   * Commands send from client calls to the background thread pool.
+   */
+  private abstract class UploadCommand {
+
+    // the blob offset for the command
+    private final long commandBlobOffset;
+
+    // command completion latch
+    private final CountDownLatch completed = new CountDownLatch(1);
+
+    UploadCommand(long offset) {
+      this.commandBlobOffset = offset;
+    }
+
+    long getCommandBlobOffset() {
+      return commandBlobOffset;
+    }
+
+    void await() throws InterruptedException {
+      completed.await();
+    }
+
+    void awaitAsDependent() throws InterruptedException {
+      await();
+    }
+
+    void setCompleted() {
+      completed.countDown();
+    }
 
-  private final Random sequenceGenerator = new Random();
+    void execute() throws InterruptedException, IOException {}
+
+    void dump() {}
+  }
+
+  /**
+   * The list of recent commands. Before block list is committed, all the block
+   * listed in the list must be uploaded. activeBlockCommands is used for
+   * enumerating the blocks and waiting on the latch until the block is
+   * uploaded.
+   */
+  private final ConcurrentLinkedQueue<UploadCommand> activeBlockCommands
+      = new ConcurrentLinkedQueue<>();
+
+  /**
+   * Variable to track if the stream has been closed.
+   */
+  private volatile boolean closed = false;
+
+  /**
+   *  First IOException encountered.
+   */
+  private final AtomicReference<IOException> firstError
+          = new AtomicReference<>();
+
+  /**
+   * Flag set when the first error has been thrown.
+   */
+  private boolean firstErrorThrown = false;
 
   /**
-   *  Time to wait to renew lease in milliseconds
+   * Semaphore for serializing block uploads with NativeAzureFileSystem.
+   *
+   * The semaphore starts with number of permits equal to the number of block
+   * upload threads. Each block upload thread needs one permit to start the
+   * upload. The put block list acquires all the permits before the block list
+   * is committed.
    */
-  private static final int LEASE_RENEWAL_PERIOD = 10000;
+  private final Semaphore uploadingSemaphore = new Semaphore(
+      MAX_NUMBER_THREADS_IN_THREAD_POOL,
+      true);
 
   /**
-   *  Number of times to retry for lease renewal
+   * Queue storing buffers with the size of the Azure block ready for
+   * reuse. The pool allows reusing the blocks instead of allocating new
+   * blocks. After the data is sent to the service, the buffer is returned
+   * back to the queue
    */
-  private static final int MAX_LEASE_RENEWAL_RETRY_COUNT = 3;
+  private final ElasticByteBufferPool poolReadyByteBuffers
+          = new ElasticByteBufferPool();
 
   /**
-   *  Time to wait before retrying to set the lease
+   * The blob's block list.
    */
-  private static final int LEASE_RENEWAL_RETRY_SLEEP_PERIOD = 500;
+  private final List<BlockEntry> blockEntries = new ArrayList<>(
+      DEFAULT_CAPACITY_BLOCK_ENTRIES);
+
+  private static final int DEFAULT_CAPACITY_BLOCK_ENTRIES = 1024;
 
   /**
-   *  Metadata key used on the blob to indicate append lease is active
+   * The uncommitted blob's block list.
    */
-  public static final String APPEND_LEASE = "append_lease";
+  private final ConcurrentLinkedDeque<BlockEntry> uncommittedBlockEntries
+      = new ConcurrentLinkedDeque<>();
+
+  /**
+   * Variable to hold the next block id to be used for azure storage blocks.
+   */
+  private static final int UNSET_BLOCKS_COUNT = -1;
+  private long nextBlockCount = UNSET_BLOCKS_COUNT;
 
   /**
-   * Timeout value for the append lease in millisecs. If the lease is not
-   * renewed within 30 seconds then another thread can acquire the append lease
-   * on the blob
+   * Variable to hold the block id prefix to be used for azure storage blocks.
    */
-  public static final int APPEND_LEASE_TIMEOUT = 30000;
+  private String blockIdPrefix = null;
 
   /**
-   *  Metdata key used on the blob to indicate last modified time of append lease
+   *  Maximum number of threads in block upload thread pool.
    */
-  public static final String APPEND_LEASE_LAST_MODIFIED = "append_lease_last_modified";
+  private static final int MAX_NUMBER_THREADS_IN_THREAD_POOL = 4;
 
   /**
    * Number of times block upload needs is retried.
@@ -145,16 +279,32 @@ public class BlockBlobAppendStream extends OutputStream {
   private static final int MAX_BLOCK_UPLOAD_RETRIES = 3;
 
   /**
-   * Wait time between block upload retries in millisecs.
+   * Wait time between block upload retries in milliseconds.
    */
   private static final int BLOCK_UPLOAD_RETRY_INTERVAL = 1000;
 
-  private static final Logger LOG = LoggerFactory.getLogger(BlockBlobAppendStream.class);
+  /**
+   * Logger.
+   */
+  private static final Logger LOG =
+          LoggerFactory.getLogger(BlockBlobAppendStream.class);
 
+  /**
+   * The absolute maximum of blocks for a blob. It includes committed and
+   * temporary blocks.
+   */
   private static final int MAX_BLOCK_COUNT = 100000;
 
+  /**
+   * The upload thread pool executor.
+   */
   private ThreadPoolExecutor ioThreadPool;
 
+  /**
+   * Azure Storage access conditions for the blob.
+   */
+  private final AccessCondition accessCondition = new AccessCondition();
+
   /**
    * Atomic integer to provide thread id for thread names for uploader threads.
    */
@@ -163,106 +313,123 @@ public class BlockBlobAppendStream extends OutputStream {
   /**
    * Prefix to be used for thread names for uploader threads.
    */
-  private static final String THREAD_ID_PREFIX = "BlockBlobAppendStream";
-
-  private static final String UTC_STR = "UTC";
+  private static final String THREAD_ID_PREFIX = "append-blockblob";
 
+  /**
+   * BlockBlobAppendStream constructor.
+   *
+   * @param blob
+   *          Azure Block Blob
+   * @param aKey
+   *          blob's name
+   * @param bufferSize
+   *          the maximum size of a blob block.
+   * @param compactionEnabled
+   *          is the compaction process enabled for this blob
+   * @param opContext
+   *          Azure Store operation context for the blob
+   * @throws IOException
+   *           if an I/O error occurs. In particular, an IOException may be
+   *           thrown if the output stream cannot be used for append operations
+   */
   public BlockBlobAppendStream(final CloudBlockBlobWrapper blob,
-      final String aKey, final int bufferSize, final OperationContext opContext)
+                               final String aKey,
+                               final int bufferSize,
+                               final boolean compactionEnabled,
+                               final OperationContext opContext)
           throws IOException {
 
-    if (null == aKey || 0 == aKey.length()) {
-      throw new IllegalArgumentException(
-          "Illegal argument: The key string is null or empty");
-    }
-
-    if (0 >= bufferSize) {
-      throw new IllegalArgumentException(
-          "Illegal argument bufferSize cannot be zero or negative");
-    }
-
+    Preconditions.checkArgument(StringUtils.isNotEmpty(aKey));
+    Preconditions.checkArgument(bufferSize >= 0);
 
     this.blob = blob;
     this.opContext = opContext;
     this.key = aKey;
-    this.bufferSize = bufferSize;
+    this.maxBlockSize = new AtomicInteger(bufferSize);
     this.threadSequenceNumber = new AtomicInteger(0);
     this.blockIdPrefix = null;
-    setBlocksCountAndBlockIdPrefix();
+    this.compactionEnabled = compactionEnabled;
+    this.blobExist = true;
+    this.outBuffer = poolReadyByteBuffers.getBuffer(false, maxBlockSize.get());
 
-    this.outBuffer = new ByteArrayOutputStream(bufferSize);
-    this.uncommittedBlockEntries = new ArrayList<BlockEntry>();
-
-    // Acquire append lease on the blob.
     try {
-      //Set the append lease if the value of the append lease is false
-      if (!updateBlobAppendMetadata(true, false)) {
-        LOG.error("Unable to set Append Lease on the Blob : {} "
-            + "Possibly because another client already has a create or append stream open on the Blob", key);
-        throw new IOException("Unable to set Append lease on the Blob. "
-            + "Possibly because another client already had an append stream open on the Blob.");
-      }
-    } catch (StorageException ex) {
-      LOG.error("Encountered Storage exception while acquiring append "
-          + "lease on blob : {}. Storage Exception : {} ErrorCode : {}",
-          key, ex, ex.getErrorCode());
+      // download the block list
+      blockEntries.addAll(
+          blob.downloadBlockList(
+              BlockListingFilter.COMMITTED,
+              new BlobRequestOptions(),
+              opContext));
+
+      blobLength = blob.getProperties().getLength();
+
+      committedBlobLength.set(blobLength);
 
-      throw new IOException(ex);
+      // Acquiring lease on the blob.
+      lease = new SelfRenewingLease(blob, true);
+      accessCondition.setLeaseID(lease.getLeaseID());
+
+    } catch (StorageException ex) {
+      if (ex.getErrorCode().equals(StorageErrorCodeStrings.BLOB_NOT_FOUND)) {
+        blobExist = false;
+      }
+      else if (ex.getErrorCode().equals(
+              StorageErrorCodeStrings.LEASE_ALREADY_PRESENT)) {
+        throw new AzureException(
+                "Unable to set Append lease on the Blob: " + ex, ex);
+      }
+      else {
+        LOG.debug(
+            "Encountered storage exception."
+                + " StorageException : {} ErrorCode : {}",
+            ex,
+            ex.getErrorCode());
+        throw new AzureException(ex);
+      }
     }
 
-    leaseFreed = false;
+    setBlocksCountAndBlockIdPrefix(blockEntries);
+
+    this.ioThreadPool = new ThreadPoolExecutor(
+        MAX_NUMBER_THREADS_IN_THREAD_POOL,
+        MAX_NUMBER_THREADS_IN_THREAD_POOL,
+        THREADPOOL_KEEP_ALIVE,
+        TimeUnit.SECONDS,
+        new LinkedBlockingQueue<>(),
+        new UploaderThreadFactory());
   }
 
   /**
-   * Helper method that starts an Append Lease renewer thread and the
-   * thread pool.
+   * Set payload size of the stream.
+   * It is intended to be used for unit testing purposes only.
    */
-  public synchronized void initialize() {
-
-    if (initialized) {
-      return;
-    }
-    /*
-     * Start the thread for  Append lease renewer.
-     */
-    Thread appendLeaseRenewer = new Thread(new AppendRenewer());
-    appendLeaseRenewer.setDaemon(true);
-    appendLeaseRenewer.setName(String.format("%s-AppendLeaseRenewer", key));
-    appendLeaseRenewer.start();
-
-    /*
-     * Parameters to ThreadPoolExecutor:
-     * corePoolSize : the number of threads to keep in the pool, even if they are idle,
-     *                unless allowCoreThreadTimeOut is set
-     * maximumPoolSize : the maximum number of threads to allow in the pool
-     * keepAliveTime - when the number of threads is greater than the core,
-     *                 this is the maximum time that excess idle threads will
-     *                 wait for new tasks before terminating.
-     * unit - the time unit for the keepAliveTime argument
-     * workQueue - the queue to use for holding tasks before they are executed
-     *  This queue will hold only the Runnable tasks submitted by the execute method.
-     */
-    this.ioThreadPool = new ThreadPoolExecutor(4, 4, 2, TimeUnit.SECONDS,
-        new LinkedBlockingQueue<Runnable>(), new UploaderThreadFactory());
+  @VisibleForTesting
+  synchronized void setMaxBlockSize(int size) {
+    maxBlockSize.set(size);
 
-    initialized = true;
+    // it is for testing only so we can abandon the previously allocated
+    // payload
+    this.outBuffer = ByteBuffer.allocate(maxBlockSize.get());
   }
 
   /**
-   * Get the blob name.
-   *
-   * @return String Blob name.
+   * Set compaction parameters.
+   * It is intended to be used for unit testing purposes only.
    */
-  public String getKey() {
-    return key;
+  @VisibleForTesting
+  void setCompactionBlockCount(int activationCount) {
+    activateCompactionBlockCount = activationCount;
   }
 
   /**
-   * Get the backing blob.
-   * @return buffer size of the stream.
+   * Get the list of block entries. It is used for testing purposes only.
+   * @return List of block entries.
    */
-  public int getBufferSize() {
-    return bufferSize;
+  @VisibleForTesting
+  List<BlockEntry> getBlockList() throws StorageException, IOException {
+    return blob.downloadBlockList(
+        BlockListingFilter.COMMITTED,
+        new BlobRequestOptions(),
+        opContext);
   }
 
   /**
@@ -282,21 +449,6 @@ public class BlockBlobAppendStream extends OutputStream {
     write(new byte[] { (byte) (byteVal & 0xFF) });
   }
 
-  /**
-   * Writes b.length bytes from the specified byte array to this output stream.
-   *
-   * @param data
-   *          the byte array 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 byte[] data) throws IOException {
-    write(data, 0, data.length);
-  }
-
   /**
    * Writes length bytes from the specified byte array starting at offset to
    * this output stream.
@@ -312,529 +464,678 @@ public class BlockBlobAppendStream extends OutputStream {
    *           thrown if the output stream has been closed.
    */
   @Override
-  public void write(final byte[] data, final int offset, final int length)
+  public synchronized void write(final byte[] data, int offset, int length)
       throws IOException {
+    Preconditions.checkArgument(data != null, "null data");
 
     if (offset < 0 || length < 0 || length > data.length - offset) {
-      throw new IndexOutOfBoundsException("write API in append stream called with invalid arguments");
-    }
-
-    writeInternal(data, offset, length);
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-
-    if (!initialized) {
-      throw new IOException("Trying to close an uninitialized Append stream");
+      throw new IndexOutOfBoundsException();
     }
 
     if (closed) {
-      return;
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
     }
 
-    if (leaseFreed) {
-      throw new IOException(String.format("Attempting to close an append stream on blob : %s "
-          + " that does not have lease on the Blob. Failing close", key));
-    }
+    while (outBuffer.remaining() < length) {
+
+      int remaining = outBuffer.remaining();
+      outBuffer.put(data, offset, remaining);
+
+      // upload payload to azure storage
+      addBlockUploadCommand();
 
-    if (outBuffer.size() > 0) {
-      uploadBlockToStorage(outBuffer.toByteArray());
+      offset += remaining;
+      length -= remaining;
     }
 
-    ioThreadPool.shutdown();
+    outBuffer.put(data, offset, length);
+  }
 
-    try {
-      if (!ioThreadPool.awaitTermination(10, TimeUnit.MINUTES)) {
-        LOG.error("Time out occurred while waiting for IO request to finish in append"
-            + " for blob : {}", key);
-        NativeAzureFileSystemHelper.logAllLiveStackTraces();
-        throw new IOException("Timed out waiting for IO requests to finish");
-      }
-    } catch(InterruptedException intrEx) {
 
-      // Restore the interrupted status
-      Thread.currentThread().interrupt();
-      LOG.error("Upload block operation in append interrupted for blob {}. Failing close", key);
-      throw new IOException("Append Commit interrupted.");
-    }
+  /**
+   * 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 {
 
-    // Calling commit after all blocks are succesfully uploaded.
-    if (lastError == null) {
-      commitAppendBlocks();
+    if (closed) {
+      // calling close() after the stream is closed starts with call to flush()
+      return;
     }
 
-    // Perform cleanup.
-    cleanup();
+    addBlockUploadCommand();
 
-    if (lastError != null) {
-      throw lastError;
+    if (committedBlobLength.get() < blobLength) {
+      try {
+        // wait until the block list is committed
+        addFlushCommand().await();
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      }
     }
   }
 
   /**
-   * Helper method that cleans up the append stream.
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete.
    */
-  private synchronized void cleanup() {
-
-    closed = true;
-
-    try {
-      // Set the value of append lease to false if the value is set to true.
-        updateBlobAppendMetadata(false, true);
-    } catch(StorageException ex) {
-      LOG.debug("Append metadata update on the Blob : {} encountered Storage Exception : {} "
-          + "Error Code : {}",
-          key, ex, ex.getErrorCode());
-      lastError = new IOException(ex);
+  @Override
+  public void hsync() throws IOException {
+    // when block compaction is disabled, hsync is empty function
+    if (compactionEnabled) {
+      flush();
     }
+  }
 
-    leaseFreed = true;
+  /**
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete.
+   */
+  @Override
+  public void hflush() throws IOException {
+    // when block compaction is disabled, hflush is empty function
+    if (compactionEnabled) {
+      flush();
+    }
   }
 
   /**
-   * Method to commit all the uncommited blocks to azure storage.
-   * If the commit fails then blocks are automatically cleaned up
-   * by Azure storage.
-   * @throws IOException
+   * The Synchronization capabilities of this stream depend upon the compaction
+   * policy.
+   * @param capability string to query the stream support for.
+   * @return true for hsync and hflush when compaction is enabled.
    */
-  private synchronized void commitAppendBlocks() throws IOException {
+  @Override
+  public boolean hasCapability(String capability) {
+    return compactionEnabled
+        && (capability.equalsIgnoreCase(HSYNC.getValue())
+        || capability.equalsIgnoreCase((HFLUSH.getValue())));
+  }
 
-    SelfRenewingLease lease = null;
+  /**
+   * 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 {
 
-    try {
-      if (uncommittedBlockEntries.size() > 0) {
+    LOG.debug("close {} ", key);
 
-        //Acquiring lease on the blob.
-        lease = new SelfRenewingLease(blob);
+    if (closed) {
+      return;
+    }
 
-        // Downloading existing blocks
-        List<BlockEntry> blockEntries =  blob.downloadBlockList(BlockListingFilter.COMMITTED,
-            new BlobRequestOptions(), opContext);
+    // Upload the last block regardless of compactionEnabled flag
+    flush();
 
-        // Adding uncommitted blocks.
-        blockEntries.addAll(uncommittedBlockEntries);
+    // Initiates an orderly shutdown in which previously submitted tasks are
+    // executed.
+    ioThreadPool.shutdown();
 
-        AccessCondition accessCondition = new AccessCondition();
-        accessCondition.setLeaseID(lease.getLeaseID());
-        blob.commitBlockList(blockEntries, accessCondition, new BlobRequestOptions(), opContext);
-        uncommittedBlockEntries.clear();
+    try {
+      // wait up to CLOSE_UPLOAD_DELAY minutes to upload all the blocks
+      if (!ioThreadPool.awaitTermination(CLOSE_UPLOAD_DELAY, TimeUnit.MINUTES)) {
+        LOG.error("Time out occurred while close() is waiting for IO request to"
+            + " finish in append"
+            + " for blob : {}",
+            key);
+        NativeAzureFileSystemHelper.logAllLiveStackTraces();
+        throw new AzureException("Timed out waiting for IO requests to finish");
       }
-    } catch(StorageException ex) {
-      LOG.error("Storage exception encountered during block commit phase of append for blob"
-          + " : {} Storage Exception : {} Error Code: {}", key, ex, ex.getErrorCode());
-      throw new IOException("Encountered Exception while committing append blocks", ex);
-    } finally {
-      if (lease != null) {
+    } catch(InterruptedException ex) {
+      Thread.currentThread().interrupt();
+    }
+
+    // release the lease
+    if (firstError.get() == null && blobExist) {
         try {
           lease.free();
-        } catch(StorageException ex) {
-          LOG.debug("Exception encountered while releasing lease for "
-              + "blob : {} StorageException : {} ErrorCode : {}", key, ex, ex.getErrorCode());
-          // Swallowing exception here as the lease is cleaned up by the SelfRenewingLease object.
+        } catch (StorageException ex) {
+          LOG.debug("Lease free update blob {} encountered Storage Exception:"
+              + " {} Error Code : {}",
+              key,
+              ex,
+              ex.getErrorCode());
+          maybeSetFirstError(new AzureException(ex));
         }
-      }
+    }
+
+    closed = true;
+
+    // finally, throw the first exception raised if it has not
+    // been thrown elsewhere.
+    if (firstError.get() != null && !firstErrorThrown) {
+      throw firstError.get();
     }
   }
 
   /**
-   * Helper method used to generate the blockIDs. The algorithm used is similar to the Azure
-   * storage SDK.
+   * Helper method used to generate the blockIDs. The algorithm used is similar
+   * to the Azure storage SDK.
    */
-  private void setBlocksCountAndBlockIdPrefix() throws IOException {
+  private void setBlocksCountAndBlockIdPrefix(List<BlockEntry> blockEntries) {
 
-    try {
+    if (nextBlockCount == UNSET_BLOCKS_COUNT && blockIdPrefix == null) {
 
-      if (nextBlockCount == UNSET_BLOCKS_COUNT && blockIdPrefix==null) {
+      Random sequenceGenerator = new Random();
 
-        List<BlockEntry> blockEntries =
-            blob.downloadBlockList(BlockListingFilter.COMMITTED, new BlobRequestOptions(), opContext);
+      String blockZeroBlockId = (!blockEntries.isEmpty())
+          ? blockEntries.get(0).getId()
+          : "";
+      String prefix = UUID.randomUUID().toString() + "-";
+      String sampleNewerVersionBlockId = generateNewerVersionBlockId(prefix,
+          0);
 
-        String blockZeroBlockId = (blockEntries.size() > 0) ? blockEntries.get(0).getId() : "";
-        String prefix = UUID.randomUUID().toString() + "-";
-        String sampleNewerVersionBlockId = generateNewerVersionBlockId(prefix, 0);
+      if (!blockEntries.isEmpty()
+          && blockZeroBlockId.length() < sampleNewerVersionBlockId.length()) {
 
-        if (blockEntries.size() > 0 && blockZeroBlockId.length() < sampleNewerVersionBlockId.length()) {
+        // If blob has already been created with 2.2.0, append subsequent blocks
+        // with older version (2.2.0) blockId compute nextBlockCount, the way it
+        // was done before; and don't use blockIdPrefix
+        this.blockIdPrefix = "";
+        nextBlockCount = (long) (sequenceGenerator.nextInt(Integer.MAX_VALUE))
+            + sequenceGenerator.nextInt(
+                Integer.MAX_VALUE - MAX_BLOCK_COUNT);
+        nextBlockCount += blockEntries.size();
 
-          // If blob has already been created with 2.2.0, append subsequent blocks with older version (2.2.0) blockId
-          // compute nextBlockCount, the way it was done before; and don't use blockIdPrefix
-          this.blockIdPrefix = "";
-          nextBlockCount = (long) (sequenceGenerator.nextInt(Integer.MAX_VALUE))
-              + sequenceGenerator.nextInt(Integer.MAX_VALUE - MAX_BLOCK_COUNT);
-          nextBlockCount += blockEntries.size();
-
-        } else {
-
-          // If there are no existing blocks, create the first block with newer version (4.2.0) blockId
-          // If blob has already been created with 4.2.0, append subsequent blocks with newer version (4.2.0) blockId
-          this.blockIdPrefix = prefix;
-          nextBlockCount = blockEntries.size();
-
-        }
+      } else {
 
+        // If there are no existing blocks, create the first block with newer
+        // version (4.2.0) blockId. If blob has already been created with 4.2.0,
+        // append subsequent blocks with newer version (4.2.0) blockId
+        this.blockIdPrefix = prefix;
+        nextBlockCount = blockEntries.size();
       }
-
-    } catch (StorageException ex) {
-      LOG.debug("Encountered storage exception during setting next Block Count and BlockId prefix."
-          + " StorageException : {} ErrorCode : {}", ex, ex.getErrorCode());
-      throw new IOException(ex);
     }
   }
 
   /**
-   * Helper method that generates the next block id for uploading a block to azure storage.
+   * Helper method that generates the next block id for uploading a block to
+   * azure storage.
    * @return String representing the block ID generated.
-   * @throws IOException
+   * @throws IOException if the stream is in invalid state
    */
   private String generateBlockId() throws IOException {
 
-    if (nextBlockCount == UNSET_BLOCKS_COUNT) {
-      throw new IOException("Append Stream in invalid state. nextBlockCount not set correctly");
-    }
-
-    if (this.blockIdPrefix == null) {
-      throw new IOException("Append Stream in invalid state. blockIdPrefix not set correctly");
-    }
-
-    if (!this.blockIdPrefix.equals("")) {
-
-      return generateNewerVersionBlockId(this.blockIdPrefix, nextBlockCount++);
-
-    } else {
-
-      return generateOlderVersionBlockId(nextBlockCount++);
-
+    if (nextBlockCount == UNSET_BLOCKS_COUNT || blockIdPrefix == null) {
+      throw new AzureException(
+            "Append Stream in invalid state. nextBlockCount not set correctly");
     }
 
+    return (!blockIdPrefix.isEmpty())
+        ? generateNewerVersionBlockId(blockIdPrefix, nextBlockCount++)
+        : generateOlderVersionBlockId(nextBlockCount++);
   }
 
   /**
-   * Helper method that generates an older (2.2.0) version blockId
+   * Helper method that generates an older (2.2.0) version blockId.
    * @return String representing the block ID generated.
    */
   private String generateOlderVersionBlockId(long id) {
 
-    byte[] blockIdInBytes = getBytesFromLong(id);
-    return new String(Base64.encodeBase64(blockIdInBytes), StandardCharsets.UTF_8);
+    byte[] blockIdInBytes = new byte[8];
+    for (int m = 0; m < 8; m++) {
+      blockIdInBytes[7 - m] = (byte) ((id >> (8 * m)) & 0xFF);
+    }
+
+    return new String(
+            Base64.encodeBase64(blockIdInBytes),
+            StandardCharsets.UTF_8);
   }
 
   /**
-   * Helper method that generates an newer (4.2.0) version blockId
+   * Helper method that generates an newer (4.2.0) version blockId.
    * @return String representing the block ID generated.
    */
   private String generateNewerVersionBlockId(String prefix, long id) {
 
     String blockIdSuffix  = String.format("%06d", id);
-    byte[] blockIdInBytes = (prefix + blockIdSuffix).getBytes(StandardCharsets.UTF_8);
+    byte[] blockIdInBytes =
+            (prefix + blockIdSuffix).getBytes(StandardCharsets.UTF_8);
     return new String(Base64.encodeBase64(blockIdInBytes), StandardCharsets.UTF_8);
   }
 
   /**
-   * Returns a byte array that represents the data of a <code>long</code> value. This
-   * utility method is copied from com.microsoft.azure.storage.core.Utility class.
-   * This class is marked as internal, hence we clone the method here and not express
-   * dependency on the Utility Class
-   *
-   * @param value
-   *            The value from which the byte array will be returned.
-   *
-   * @return A byte array that represents the data of the specified <code>long</code> value.
+   * This is shared between upload block Runnable and CommitBlockList. The
+   * method captures retry logic
+   * @param blockId block name
+   * @param dataPayload block content
    */
-  private static byte[] getBytesFromLong(final long value) {
+  private void writeBlockRequestInternal(String blockId,
+                                         ByteBuffer dataPayload,
+                                         boolean bufferPoolBuffer) {
+    IOException lastLocalException = null;
+
+    int uploadRetryAttempts = 0;
+    while (uploadRetryAttempts < MAX_BLOCK_UPLOAD_RETRIES) {
+      try {
+        long startTime = System.nanoTime();
+
+        blob.uploadBlock(blockId, accessCondition, new ByteArrayInputStream(
+            dataPayload.array()), dataPayload.position(),
+            new BlobRequestOptions(), opContext);
 
-    final byte[] tempArray = new byte[8];
+        LOG.debug("upload block finished for {} ms. block {} ",
+            TimeUnit.NANOSECONDS.toMillis(
+                System.nanoTime() - startTime), blockId);
+        break;
+
+      } catch(Exception ioe) {
+        LOG.debug("Encountered exception during uploading block for Blob {}"
+            + " Exception : {}", key, ioe);
+        uploadRetryAttempts++;
+        lastLocalException = new AzureException(
+            "Encountered Exception while uploading block: " + ioe, ioe);
+        try {
+          Thread.sleep(
+              BLOCK_UPLOAD_RETRY_INTERVAL * (uploadRetryAttempts + 1));
+        } catch(InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+    }
 
-    for (int m = 0; m < 8; m++) {
-      tempArray[7 - m] = (byte) ((value >> (8 * m)) & 0xFF);
+    if (bufferPoolBuffer) {
+      poolReadyByteBuffers.putBuffer(dataPayload);
     }
 
-    return tempArray;
+    if (uploadRetryAttempts == MAX_BLOCK_UPLOAD_RETRIES) {
+      maybeSetFirstError(lastLocalException);
+    }
   }
 
   /**
-   * Helper method that creates a thread to upload a block to azure storage.
-   * @param payload
-   * @throws IOException
+   * Set {@link #firstError} to the exception if it is not already set.
+   * @param exception exception to save
    */
-  private synchronized void uploadBlockToStorage(byte[] payload)
-      throws IOException {
-
-    // upload payload to azure storage
-    String blockId = generateBlockId();
-
-    // Since uploads of the Azure storage are done in parallel threads, we go ahead
-    // add the blockId in the uncommitted list. If the upload of the block fails
-    // we don't commit the blockIds.
-    BlockEntry blockEntry = new BlockEntry(blockId);
-    blockEntry.setSize(payload.length);
-    uncommittedBlockEntries.add(blockEntry);
-    ioThreadPool.execute(new WriteRequest(payload, blockId));
+  private void maybeSetFirstError(IOException exception) {
+    firstError.compareAndSet(null, exception);
   }
 
 
   /**
-   * Helper method to updated the Blob metadata during Append lease operations.
-   * Blob metadata is updated to holdLease value only if the current lease
-   * status is equal to testCondition and the last update on the blob metadata
-   * is less that 30 secs old.
-   * @param holdLease
-   * @param testCondition
-   * @return true if the updated lease operation was successful or false otherwise
-   * @throws StorageException
+   * Throw the first error caught if it has not been raised already
+   * @throws IOException if one is caught and needs to be thrown.
    */
-  private boolean updateBlobAppendMetadata(boolean holdLease, boolean testCondition)
-      throws StorageException {
-
-    SelfRenewingLease lease = null;
-    StorageException lastStorageException = null;
-    int leaseRenewalRetryCount = 0;
-
-    /*
-     * Updating the Blob metadata honours following algorithm based on
-     *  1) If the append lease metadata is present
-     *  2) Last updated time of the append lease
-     *  3) Previous value of the Append lease metadata.
-     *
-     * The algorithm:
-     *  1) If append lease metadata is not part of the Blob. In this case
-     *     this is the first client to Append so we update the metadata.
-     *  2) If append lease metadata is present and timeout has occurred.
-     *     In this case irrespective of what the value of the append lease is we update the metadata.
-     *  3) If append lease metadata is present and is equal to testCondition value (passed as parameter)
-     *     and timeout has not occurred, we update the metadata.
-     *  4) If append lease metadata is present and is not equal to testCondition value (passed as parameter)
-     *     and timeout has not occurred, we do not update metadata and return false.
-     *
-     */
-    while (leaseRenewalRetryCount < MAX_LEASE_RENEWAL_RETRY_COUNT) {
-
-      lastStorageException = null;
-
-      synchronized(this) {
-        try {
-
-          final Calendar currentCalendar = Calendar
-              .getInstance(Locale.US);
-          currentCalendar.setTimeZone(TimeZone.getTimeZone(UTC_STR));
-          long currentTime = currentCalendar.getTime().getTime();
+  private void maybeThrowFirstError() throws IOException {
+    if (firstError.get() != null) {
+      firstErrorThrown = true;
+      throw firstError.get();
+    }
+  }
 
-          // Acquire lease on the blob.
-          lease = new SelfRenewingLease(blob);
+  /**
+   * Write block list. The method captures retry logic
+   */
+  private void writeBlockListRequestInternal() {
 
-          blob.downloadAttributes(opContext);
-          HashMap<String, String> metadata = blob.getMetadata();
+    IOException lastLocalException = null;
 
-          if (metadata.containsKey(APPEND_LEASE)
-              && currentTime - Long.parseLong(
-                  metadata.get(APPEND_LEASE_LAST_MODIFIED)) <= BlockBlobAppendStream.APPEND_LEASE_TIMEOUT
-              && !metadata.get(APPEND_LEASE).equals(Boolean.toString(testCondition))) {
-            return false;
-          }
+    int uploadRetryAttempts = 0;
+    while (uploadRetryAttempts < MAX_BLOCK_UPLOAD_RETRIES) {
+      try {
 
-          metadata.put(APPEND_LEASE, Boolean.toString(holdLease));
-          metadata.put(APPEND_LEASE_LAST_MODIFIED, Long.toString(currentTime));
-          blob.setMetadata(metadata);
-          AccessCondition accessCondition = new AccessCondition();
-          accessCondition.setLeaseID(lease.getLeaseID());
-          blob.uploadMetadata(accessCondition, null, opContext);
-          return true;
+        long startTime = System.nanoTime();
 
-        } catch (StorageException ex) {
-
-          lastStorageException = ex;
-          LOG.debug("Lease renewal for Blob : {} encountered Storage Exception : {} "
-              + "Error Code : {}",
-              key, ex, ex.getErrorCode());
-          leaseRenewalRetryCount++;
-
-        } finally {
-
-          if (lease != null) {
-            try {
-              lease.free();
-            } catch(StorageException ex) {
-              LOG.debug("Encountered Storage exception while releasing lease for Blob {} "
-                  + "during Append  metadata operation. Storage Exception {} "
-                  + "Error Code : {} ", key, ex, ex.getErrorCode());
-            } finally {
-              lease = null;
-            }
-          }
-        }
-      }
+        blob.commitBlockList(blockEntries, accessCondition,
+            new BlobRequestOptions(), opContext);
 
-      if (leaseRenewalRetryCount == MAX_LEASE_RENEWAL_RETRY_COUNT) {
-        throw lastStorageException;
-      } else {
+        LOG.debug("Upload block list took {} ms for blob {} ",
+                TimeUnit.NANOSECONDS.toMillis(
+                    System.nanoTime() - startTime), key);
+        break;
+
+      } catch(Exception ioe) {
+        LOG.debug("Encountered exception during uploading block for Blob {}"
+            + " Exception : {}", key, ioe);
+        uploadRetryAttempts++;
+        lastLocalException = new AzureException(
+            "Encountered Exception while uploading block: " + ioe, ioe);
         try {
-          Thread.sleep(LEASE_RENEWAL_RETRY_SLEEP_PERIOD);
-        } catch(InterruptedException ex) {
-          LOG.debug("Blob append metadata updated method interrupted");
+          Thread.sleep(
+              BLOCK_UPLOAD_RETRY_INTERVAL * (uploadRetryAttempts + 1));
+        } catch(InterruptedException ie) {
           Thread.currentThread().interrupt();
+          break;
         }
       }
     }
 
-    // The code should not enter here because the while loop will
-    // always be executed and if the while loop is executed we
-    // would returning from the while loop.
-    return false;
+    if (uploadRetryAttempts == MAX_BLOCK_UPLOAD_RETRIES) {
+      maybeSetFirstError(lastLocalException);
+    }
   }
 
   /**
-   * This is the only method that should be writing to outBuffer to maintain consistency of the outBuffer.
-   * @param data
-   * @param offset
-   * @param length
-   * @throws IOException
+   * A ThreadFactory that creates uploader thread with
+   * meaningful names helpful for debugging purposes.
    */
-  private synchronized void writeInternal(final byte[] data, final int offset, final int length)
-      throws IOException {
+  class UploaderThreadFactory implements ThreadFactory {
 
-    if (!initialized) {
-      throw new IOException("Trying to write to an un-initialized Append stream");
+    @Override
+    public Thread newThread(Runnable r) {
+      Thread t = new Thread(r);
+      t.setName(String.format("%s-%d", THREAD_ID_PREFIX,
+          threadSequenceNumber.getAndIncrement()));
+      return t;
     }
+  }
 
-    if (closed) {
-      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
-    }
+  /**
+   * Upload block commands.
+   */
+  private class UploadBlockCommand extends UploadCommand {
 
-    if (leaseFreed) {
-      throw new IOException(String.format("Write called on a append stream not holding lease. Failing Write"));
-    }
+    // the block content for upload
+    private final ByteBuffer payload;
 
-    byte[] currentData = new byte[length];
-    System.arraycopy(data, offset, currentData, 0, length);
+    // description of the block
+    private final BlockEntry entry;
 
-    // check to see if the data to be appended exceeds the
-    // buffer size. If so we upload a block to azure storage.
-    while ((outBuffer.size() + currentData.length) > bufferSize) {
+    UploadBlockCommand(String blockId, ByteBuffer payload) {
 
-      byte[] payload = new byte[bufferSize];
+      super(blobLength);
 
-      // Add data from the existing buffer
-      System.arraycopy(outBuffer.toByteArray(), 0, payload, 0, outBuffer.size());
+      BlockEntry blockEntry = new BlockEntry(blockId);
+      blockEntry.setSize(payload.position());
+      blockEntry.setSearchMode(BlockSearchMode.LATEST);
 
-      // Updating the available size in the payload
-      int availableSpaceInPayload = bufferSize - outBuffer.size();
+      this.payload = payload;
+      this.entry = blockEntry;
 
-      // Adding data from the current call
-      System.arraycopy(currentData, 0, payload, outBuffer.size(), availableSpaceInPayload);
+      uncommittedBlockEntries.add(blockEntry);
+    }
 
-      uploadBlockToStorage(payload);
+    /**
+     * Execute command.
+     */
+    void execute() throws InterruptedException {
+
+      uploadingSemaphore.acquire(1);
+      writeBlockRequestInternal(entry.getId(), payload, true);
+      uploadingSemaphore.release(1);
 
-      // updating the currentData buffer
-      byte[] tempBuffer = new byte[currentData.length - availableSpaceInPayload];
-      System.arraycopy(currentData, availableSpaceInPayload,
-          tempBuffer, 0, currentData.length - availableSpaceInPayload);
-      currentData = tempBuffer;
-      outBuffer = new ByteArrayOutputStream(bufferSize);
     }
 
-    outBuffer.write(currentData);
+    void dump() {
+      LOG.debug("upload block {} size: {} for blob {}",
+          entry.getId(),
+          entry.getSize(),
+          key);
+    }
   }
 
   /**
-   * Runnable instance that uploads the block of data to azure storage.
-   *
-   *
+   * Upload blob block list commands.
    */
-  private class WriteRequest implements Runnable {
-    private final byte[] dataPayload;
-    private final String blockId;
+  private class UploadBlockListCommand extends UploadCommand {
+
+    private BlockEntry lastBlock = null;
+
+    UploadBlockListCommand() {
+      super(blobLength);
 
-    public WriteRequest(byte[] dataPayload, String blockId) {
-      this.dataPayload = dataPayload;
-      this.blockId = blockId;
+      if (!uncommittedBlockEntries.isEmpty()) {
+        lastBlock = uncommittedBlockEntries.getLast();
+      }
     }
 
-    @Override
-    public void run() {
+    void awaitAsDependent() throws InterruptedException {
+      // empty. later commit block does not need to wait previous commit block
+      // lists.
+    }
 
-      int uploadRetryAttempts = 0;
-      IOException lastLocalException = null;
-      while (uploadRetryAttempts < MAX_BLOCK_UPLOAD_RETRIES) {
-        try {
+    void dump() {
+      LOG.debug("commit block list with {} blocks for blob {}",
+          uncommittedBlockEntries.size(), key);
+    }
+
+    /**
+     * Execute command.
+     */
+    public void execute() throws InterruptedException, IOException {
+
+      if (committedBlobLength.get() >= getCommandBlobOffset()) {
+        LOG.debug("commit already applied for {}", key);
+        return;
+      }
+
+      if (lastBlock == null) {
+        LOG.debug("nothing to commit for {}", key);
+        return;
+      }
+
+      LOG.debug("active commands: {} for {}", activeBlockCommands.size(), key);
+
+      for (UploadCommand activeCommand : activeBlockCommands) {
+        if (activeCommand.getCommandBlobOffset() < getCommandBlobOffset()) {
+          activeCommand.dump();
+          activeCommand.awaitAsDependent();
+        } else {
+          break;
+        }
+      }
+
+      // stop all uploads until the block list is committed
+      uploadingSemaphore.acquire(MAX_NUMBER_THREADS_IN_THREAD_POOL);
+
+      BlockEntry uncommittedBlock;
+      do  {
+        uncommittedBlock = uncommittedBlockEntries.poll();
+        blockEntries.add(uncommittedBlock);
+      } while (uncommittedBlock != lastBlock);
+
+      if (blockEntries.size() > activateCompactionBlockCount) {
+        LOG.debug("Block compaction: activated with {} blocks for {}",
+            blockEntries.size(), key);
+
+        // Block compaction
+        long startCompaction = System.nanoTime();
+        blockCompaction();
+        LOG.debug("Block compaction finished for {} ms with {} blocks for {}",
+                TimeUnit.NANOSECONDS.toMillis(
+                    System.nanoTime() - startCompaction),
+                blockEntries.size(), key);
+      }
+
+      writeBlockListRequestInternal();
+
+      uploadingSemaphore.release(MAX_NUMBER_THREADS_IN_THREAD_POOL);
 
-          blob.uploadBlock(blockId, new ByteArrayInputStream(dataPayload),
-              dataPayload.length, new BlobRequestOptions(), opContext);
+      // remove blocks previous commands
+      for (Iterator<UploadCommand> it = activeBlockCommands.iterator();
+           it.hasNext();) {
+        UploadCommand activeCommand = it.next();
+        if (activeCommand.getCommandBlobOffset() <= getCommandBlobOffset()) {
+          it.remove();
+        } else {
           break;
-        } catch(Exception ioe) {
-          Log.getLog().debug("Encountered exception during uploading block for Blob : {} Exception : {}", key, ioe);
-          uploadRetryAttempts++;
-          lastLocalException = new IOException("Encountered Exception while uploading block", ioe);
-          try {
-            Thread.sleep(BLOCK_UPLOAD_RETRY_INTERVAL);
-          } catch(InterruptedException ie) {
-            Thread.currentThread().interrupt();
-            break;
+        }
+      }
+
+      committedBlobLength.set(getCommandBlobOffset());
+    }
+
+    /**
+     * Internal output stream with read access to the internal buffer.
+     */
+    private class ByteArrayOutputStreamInternal extends ByteArrayOutputStream {
+
+      ByteArrayOutputStreamInternal(int size) {
+        super(size);
+      }
+
+      byte[] getByteArray() {
+        return buf;
+      }
+    }
+
+    /**
+     * Block compaction process.
+     *
+     * Block compaction is only enabled when the number of blocks exceeds
+     * activateCompactionBlockCount. The algorithm searches for the longest
+     * segment [b..e) where (e-b) > 2 && |b| + |b+1| ... |e-1| < maxBlockSize
+     * such that size(b1) + size(b2) + ... + size(bn) < maximum-block-size.
+     * It then downloads the blocks in the sequence, concatenates the data to
+     * form a single block, uploads this new block, and updates the block
+     * list to replace the sequence of blocks with the new block.
+     */
+    private void blockCompaction() throws IOException {
+      //current segment [segmentBegin, segmentEnd) and file offset/size of the
+      // current segment
+      int segmentBegin = 0, segmentEnd = 0;
+      long segmentOffsetBegin = 0, segmentOffsetEnd = 0;
+
+      //longest segment [maxSegmentBegin, maxSegmentEnd) and file offset/size of
+      // the longest segment
+      int maxSegmentBegin = 0, maxSegmentEnd = 0;
+      long maxSegmentOffsetBegin = 0, maxSegmentOffsetEnd = 0;
+
+      for (BlockEntry block : blockEntries) {
+        segmentEnd++;
+        segmentOffsetEnd += block.getSize();
+        if (segmentOffsetEnd - segmentOffsetBegin > maxBlockSize.get()) {
+          if (segmentEnd - segmentBegin > 2) {
+            if (maxSegmentEnd - maxSegmentBegin < segmentEnd - segmentBegin) {
+              maxSegmentBegin = segmentBegin;
+              maxSegmentEnd = segmentEnd;
+              maxSegmentOffsetBegin = segmentOffsetBegin;
+              maxSegmentOffsetEnd = segmentOffsetEnd - block.getSize();
+            }
           }
+          segmentBegin = segmentEnd - 1;
+          segmentOffsetBegin = segmentOffsetEnd - block.getSize();
         }
       }
 
-      if (uploadRetryAttempts == MAX_BLOCK_UPLOAD_RETRIES) {
-        lastError = lastLocalException;
+      if (maxSegmentEnd - maxSegmentBegin > 1) {
+
+        LOG.debug("Block compaction: {} blocks for {}",
+            maxSegmentEnd - maxSegmentBegin, key);
+
+        // download synchronously all the blocks from the azure storage
+        ByteArrayOutputStreamInternal blockOutputStream
+            = new ByteArrayOutputStreamInternal(maxBlockSize.get());
+
+        try {
+          long length = maxSegmentOffsetEnd - maxSegmentOffsetBegin;
+          blob.downloadRange(maxSegmentOffsetBegin, length, blockOutputStream,
+              new BlobRequestOptions(), opContext);
+        } catch(StorageException ex) {
+          LOG.error(
+              "Storage exception encountered during block compaction phase"
+                  + " : {} Storage Exception : {} Error Code: {}",
+              key, ex, ex.getErrorCode());
+          throw new AzureException(
+              "Encountered Exception while committing append blocks " + ex, ex);
+        }
+
+        // upload synchronously new block to the azure storage
+        String blockId = generateBlockId();
+
+        ByteBuffer byteBuffer = ByteBuffer.wrap(
+            blockOutputStream.getByteArray());
+        byteBuffer.position(blockOutputStream.size());
+
+        writeBlockRequestInternal(blockId, byteBuffer, false);
+
+        // replace blocks from the longest segment with new block id
+        blockEntries.subList(maxSegmentBegin + 1, maxSegmentEnd - 1).clear();
+        BlockEntry newBlock = blockEntries.get(maxSegmentBegin);
+        newBlock.setId(blockId);
+        newBlock.setSearchMode(BlockSearchMode.LATEST);
+        newBlock.setSize(maxSegmentOffsetEnd - maxSegmentOffsetBegin);
       }
     }
   }
 
   /**
-   * A ThreadFactory that creates uploader thread with
-   * meaningful names helpful for debugging purposes.
+   * Prepare block upload command and queue the command in thread pool executor.
    */
-  class UploaderThreadFactory implements ThreadFactory {
+  private synchronized void addBlockUploadCommand() throws IOException {
+
+    maybeThrowFirstError();
+
+    if (blobExist && lease.isFreed()) {
+      throw new AzureException(String.format(
+          "Attempting to upload a block on blob : %s "
+              + " that does not have lease on the Blob. Failing upload", key));
+    }
+
+    int blockSize = outBuffer.position();
+    if (blockSize > 0) {
+      UploadCommand command = new UploadBlockCommand(generateBlockId(),
+          outBuffer);
+      activeBlockCommands.add(command);
+
+      blobLength += blockSize;
+      outBuffer = poolReadyByteBuffers.getBuffer(false, maxBlockSize.get());
+
+      ioThreadPool.execute(new WriteRequest(command));
 
-    @Override
-    public Thread newThread(Runnable r) {
-      Thread t = new Thread(r);
-      t.setName(String.format("%s-%s-%d", THREAD_ID_PREFIX, key,
-          threadSequenceNumber.getAndIncrement()));
-      return t;
     }
   }
 
   /**
-   * A deamon thread that renews the Append lease on the blob.
-   * The thread sleeps for LEASE_RENEWAL_PERIOD time before renewing
-   * the lease. If an error is encountered while renewing the lease
-   * then an lease is released by this thread, which fails all other
-   * operations.
+   * Prepare block list commit command and queue the command in thread pool
+   * executor.
    */
-  private class AppendRenewer implements Runnable {
+  private synchronized UploadCommand addFlushCommand() throws IOException {
 
-    @Override
-    public void run() {
+    maybeThrowFirstError();
 
-      while (!leaseFreed) {
+    if (blobExist && lease.isFreed()) {
+      throw new AzureException(
+          String.format("Attempting to upload block list on blob : %s"
+              + " that does not have lease on the Blob. Failing upload", key));
+    }
 
-        try {
-          Thread.sleep(LEASE_RENEWAL_PERIOD);
-        } catch (InterruptedException ie) {
-          LOG.debug("Appender Renewer thread interrupted");
-          Thread.currentThread().interrupt();
-        }
+    UploadCommand command = new UploadBlockListCommand();
+    activeBlockCommands.add(command);
 
-        Log.getLog().debug("Attempting to renew append lease on {}", key);
+    ioThreadPool.execute(new WriteRequest(command));
 
-        try {
-          if (!leaseFreed) {
-            // Update the blob metadata to renew the append lease
-            if (!updateBlobAppendMetadata(true, true)) {
-              LOG.error("Unable to re-acquire append lease on the Blob {} ", key);
-              leaseFreed = true;
-            }
-          }
-        } catch (StorageException ex) {
+    return command;
+  }
 
-          LOG.debug("Lease renewal for Blob : {} encountered "
-              + "Storage Exception : {} Error Code : {}", key, ex, ex.getErrorCode());
+  /**
+   * Runnable instance that uploads the block of data to azure storage.
+   */
+  private class WriteRequest implements Runnable {
+    private final UploadCommand command;
 
-          // We swallow the exception here because if the blob metadata is not updated for
-          // APPEND_LEASE_TIMEOUT period, another thread would be able to detect this and
-          // continue forward if it needs to append.
-          leaseFreed = true;
-        }
+    WriteRequest(UploadCommand command) {
+      this.command = command;
+    }
+
+    @Override
+    public void run() {
+
+      try {
+        command.dump();
+        long startTime = System.nanoTime();
+        command.execute();
+        command.setCompleted();
+        LOG.debug("command finished for {} ms",
+            TimeUnit.NANOSECONDS.toMillis(
+                    System.nanoTime() - startTime));
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      } catch (Exception ex) {
+        LOG.debug(
+                "Encountered exception during execution of command for Blob :"
+                        + " {} Exception : {}", key, ex);
+        firstError.compareAndSet(null, new AzureException(ex));
       }
     }
   }
-}
+}

+ 62 - 15
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java

@@ -62,6 +62,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemMetricsSystem;
 import org.apache.hadoop.fs.azure.security.Constants;
@@ -352,9 +354,9 @@ public class NativeAzureFileSystem extends FileSystem {
     }
 
     /**
-     * This is an exact copy of org.codehaus.jettison.json.JSONObject.quote 
+     * This is an exact copy of org.codehaus.jettison.json.JSONObject.quote
      * method.
-     * 
+     *
      * Produce a string in double quotes with backslash sequences in all the
      * right places. A backslash will be inserted within </, allowing JSON
      * text to be delivered in HTML. In JSON text, a string cannot contain a
@@ -947,11 +949,11 @@ public class NativeAzureFileSystem extends FileSystem {
     }
   }
 
-  private class NativeAzureFsOutputStream extends OutputStream {
-    // We should not override flush() to actually close current block and flush
-    // to DFS, this will break applications that assume flush() is a no-op.
-    // Applications are advised to use Syncable.hflush() for that purpose.
-    // NativeAzureFsOutputStream needs to implement Syncable if needed.
+  /**
+   * Azure output stream; wraps an inner stream of different types.
+   */
+  public class NativeAzureFsOutputStream extends OutputStream
+      implements Syncable, StreamCapabilities {
     private String key;
     private String keyEncoded;
     private OutputStream out;
@@ -983,6 +985,48 @@ public class NativeAzureFileSystem extends FileSystem {
       setEncodedKey(anEncodedKey);
     }
 
+    /**
+     * Get a reference to the wrapped output stream.
+     *
+     * @return the underlying output stream
+     */
+    @InterfaceAudience.LimitedPrivate({"HDFS"})
+    public OutputStream getOutStream() {
+      return out;
+    }
+
+    @Override  // Syncable
+    public void hflush() throws IOException {
+      if (out instanceof Syncable) {
+        ((Syncable) out).hflush();
+      } else {
+        flush();
+      }
+    }
+
+    @Override  // Syncable
+    public void hsync() throws IOException {
+      if (out instanceof Syncable) {
+        ((Syncable) out).hsync();
+      } else {
+        flush();
+      }
+    }
+
+    /**
+     * Propagate probe of stream capabilities to nested stream
+     * (if supported), else return false.
+     * @param capability string to query the stream support for.
+     * @return true if the nested stream supports the specific capability.
+     */
+    @Override // StreamCapability
+    public boolean hasCapability(String capability) {
+      if (out instanceof StreamCapabilities) {
+        return ((StreamCapabilities) out).hasCapability(capability);
+      }
+      return false;
+    }
+
     @Override
     public synchronized void close() throws IOException {
       if (out != null) {
@@ -990,8 +1034,11 @@ public class NativeAzureFileSystem extends FileSystem {
         // before returning to the caller.
         //
         out.close();
-        restoreKey();
-        out = null;
+        try {
+          restoreKey();
+        } finally {
+          out = null;
+        }
       }
     }
 
@@ -1045,10 +1092,10 @@ public class NativeAzureFileSystem extends FileSystem {
     /**
      * Writes <code>len</code> from the specified byte array starting at offset
      * <code>off</code> to the output stream. The general contract for write(b,
-     * off, len) is that some of the bytes in the array <code>
-     * b</code b> are written to the output stream in order; element
-     * <code>b[off]</code> is the first byte written and
-     * <code>b[off+len-1]</code> is the last byte written by this operation.
+     * off, len) is that some of the bytes in the array <code>b</code>
+     * are written to the output stream in order; element <code>b[off]</code>
+     * is the first byte written and <code>b[off+len-1]</code> is the last
+     * byte written by this operation.
      * 
      * @param b
      *          Byte array to be written.
@@ -1749,7 +1796,7 @@ public class NativeAzureFileSystem extends FileSystem {
     OutputStream bufOutStream;
     if (store.isPageBlobKey(key)) {
       // Store page blobs directly in-place without renames.
-      bufOutStream = store.storefile(key, permissionStatus);
+      bufOutStream = store.storefile(key, permissionStatus, key);
     } else {
       // This is a block blob, so open the output blob stream based on the
       // encoded key.
@@ -1777,7 +1824,7 @@ public class NativeAzureFileSystem extends FileSystem {
       // these
       // blocks.
       bufOutStream = new NativeAzureFsOutputStream(store.storefile(
-          keyEncoded, permissionStatus), key, keyEncoded);
+          keyEncoded, permissionStatus, key), key, keyEncoded);
     }
     // Construct the data output stream from the buffered output stream.
     FSDataOutputStream fsOut = new FSDataOutputStream(bufOutStream, statistics);

+ 3 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java

@@ -50,8 +50,9 @@ interface NativeFileSystemStore {
 
   InputStream retrieve(String key, long byteRangeStart) throws IOException;
 
-  DataOutputStream storefile(String key, PermissionStatus permissionStatus)
-      throws AzureException;
+  DataOutputStream storefile(String keyEncoded,
+      PermissionStatus permissionStatus,
+      String key) throws AzureException;
 
   boolean isPageBlobKey(String key);
 

+ 6 - 4
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java

@@ -519,7 +519,7 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
 
     @Override
     public SelfRenewingLease acquireLease() throws StorageException {
-      return new SelfRenewingLease(this);
+      return new SelfRenewingLease(this, false);
     }
   }
 
@@ -557,10 +557,12 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
     }
 
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition,
+        InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
-      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, null, options, opContext);
+      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length,
+          accessCondition, options, opContext);
     }
 
     @Override
@@ -593,4 +595,4 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
           null, options, opContext);
     }
   }
-}
+}

+ 8 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java

@@ -30,6 +30,8 @@ import com.microsoft.azure.storage.blob.CloudBlob;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static com.microsoft.azure.storage.StorageErrorCodeStrings.LEASE_ALREADY_PRESENT;
+
 /**
  * An Azure blob lease that automatically renews itself indefinitely
  * using a background thread. Use it to synchronize distributed processes,
@@ -66,7 +68,7 @@ public class SelfRenewingLease {
   @VisibleForTesting
   static final int LEASE_ACQUIRE_RETRY_INTERVAL = 2000;
 
-  public SelfRenewingLease(CloudBlobWrapper blobWrapper)
+  public SelfRenewingLease(CloudBlobWrapper blobWrapper, boolean throwIfPresent)
       throws StorageException {
 
     this.leaseFreed = false;
@@ -79,10 +81,14 @@ public class SelfRenewingLease {
         leaseID = blob.acquireLease(LEASE_TIMEOUT, null);
       } catch (StorageException e) {
 
+        if (throwIfPresent && e.getErrorCode().equals(LEASE_ALREADY_PRESENT)) {
+          throw e;
+        }
+
         // Throw again if we don't want to keep waiting.
         // We expect it to be that the lease is already present,
         // or in some cases that the blob does not exist.
-        if (!"LeaseAlreadyPresent".equals(e.getErrorCode())) {
+        if (!LEASE_ALREADY_PRESENT.equals(e.getErrorCode())) {
           LOG.info(
             "Caught exception when trying to get lease on blob "
             + blobWrapper.getUri().toString() + ". " + e.getMessage());

+ 2 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java

@@ -665,6 +665,7 @@ abstract class StorageInterface {
      *
      * @param blockId      A String that represents the Base-64 encoded block ID. Note for a given blob
      *                     the length of all Block IDs must be identical.
+     * @param accessCondition An {@link AccessCondition} object that represents the access conditions for the blob.
      * @param sourceStream An {@link InputStream} object that represents the input stream to write to the
      *                     block blob.
      * @param length       A long which represents the length, in bytes, of the stream data,
@@ -678,7 +679,7 @@ abstract class StorageInterface {
      * @throws IOException  If an I/O error occurred.
      * @throws StorageException If a storage service error occurred.
      */
-    void uploadBlock(String blockId, InputStream sourceStream,
+    void uploadBlock(String blockId, AccessCondition accessCondition, InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException;
 

+ 6 - 6
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java

@@ -277,7 +277,7 @@ class StorageInterfaceImpl extends StorageInterface {
 
       return new CloudBlockBlobWrapperImpl(container.getBlockBlobReference(relativePath));
     }
-    
+
     @Override
     public CloudBlobWrapper getPageBlobReference(String relativePath)
         throws URISyntaxException, StorageException {
@@ -286,7 +286,7 @@ class StorageInterfaceImpl extends StorageInterface {
     }
 
   }
-  
+
   abstract static class CloudBlobWrapperImpl implements CloudBlobWrapper {
     private final CloudBlob blob;
 
@@ -441,10 +441,10 @@ class StorageInterfaceImpl extends StorageInterface {
 
     @Override
     public SelfRenewingLease acquireLease() throws StorageException {
-      return new SelfRenewingLease(this);
+      return new SelfRenewingLease(this, false);
     }
   }
-  
+
 
   //
   // CloudBlockBlobWrapperImpl
@@ -479,10 +479,10 @@ class StorageInterfaceImpl extends StorageInterface {
     }
 
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition, InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
-      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, null, options, opContext);
+      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, accessCondition, options, opContext);
     }
 
     @Override

+ 11 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java

@@ -24,6 +24,7 @@ import java.io.OutputStream;
 
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
  * Support the Syncable interface on top of a DataOutputStream.
@@ -38,6 +39,16 @@ public class SyncableDataOutputStream extends DataOutputStream
     super(out);
   }
 
+  /**
+   * Get a reference to the wrapped output stream.
+   *
+   * @return the underlying output stream
+   */
+  @InterfaceAudience.LimitedPrivate({"HDFS"})
+  public OutputStream getOutStream() {
+    return out;
+  }
+
   @Override
   public boolean hasCapability(String capability) {
     if (out instanceof StreamCapabilities) {

+ 34 - 0
hadoop-tools/hadoop-azure/src/site/markdown/index.md

@@ -153,6 +153,40 @@ line argument:
 
 ```
 
+### Block Blob with Compaction Support and Configuration
+
+Block blobs are the default kind of blob and are good for most big-data use
+cases. However, block blobs have strict limit of 50,000 blocks per blob.
+To prevent reaching the limit WASB, by default, does not upload new block to
+the service after every `hflush()` or `hsync()`.
+
+For most of the cases, combining data from multiple `write()` calls in
+blocks of 4Mb is a good optimization. But, in others cases, like HBase log files,
+every call to `hflush()` or `hsync()` must upload the data to the service.
+
+Block blobs with compaction upload the data to the cloud service after every
+`hflush()`/`hsync()`. To mitigate the limit of 50000 blocks, `hflush()
+`/`hsync()` runs once compaction process, if number of blocks in the blob
+is above 32,000.
+
+Block compaction search and replaces a sequence of small blocks with one big
+block. That means there is associated cost with block compaction: reading
+small blocks back to the client and writing it again as one big block.
+
+In order to have the files you create be block blobs with block compaction
+enabled, the client must set the configuration variable
+`fs.azure.block.blob.with.compaction.dir` to a comma-separated list of
+folder names.
+
+For example:
+
+```xml
+<property>
+  <name>fs.azure.block.blob.with.compaction.dir</name>
+  <value>/hbase/WALs,/data/myblobfiles</value>
+</property>
+```
+
 ### Page Blob Support and Configuration
 
 The Azure Blob Storage interface for Hadoop supports two kinds of blobs,

+ 2 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java

@@ -551,7 +551,8 @@ public class MockStorageInterface extends StorageInterface {
       throw new UnsupportedOperationException("downloadBlockList not used in Mock Tests");
     }
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition,
+        InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
       throw new UnsupportedOperationException("uploadBlock not used in Mock Tests");

+ 4 - 2
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java

@@ -107,7 +107,8 @@ public class TestAzureConcurrentOutOfBandIo {
           //
           outputStream = writerStorageAccount.getStore().storefile(
               key,
-              new PermissionStatus("", "", FsPermission.getDefault()));
+              new PermissionStatus("", "", FsPermission.getDefault()),
+              key);
 
           Arrays.fill(dataBlockWrite, (byte) (i % 256));
           for (int j = 0; j < NUMBER_OF_BLOCKS; j++) {
@@ -141,7 +142,8 @@ public class TestAzureConcurrentOutOfBandIo {
    // reading.  This eliminates the race between the reader and writer threads.
    OutputStream outputStream = testAccount.getStore().storefile(
        "WASB_String.txt",
-       new PermissionStatus("", "", FsPermission.getDefault()));
+       new PermissionStatus("", "", FsPermission.getDefault()),
+           "WASB_String.txt");
    Arrays.fill(dataBlockWrite, (byte) 255);
    for (int i = 0; i < NUMBER_OF_BLOCKS; i++) {
      outputStream.write(dataBlockWrite);

+ 266 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java

@@ -0,0 +1,266 @@
+/*
+ * 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.azure;
+
+import com.microsoft.azure.storage.blob.BlockEntry;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.List;
+
+/**
+ * Test class that runs WASB block compaction process for block blobs.
+ */
+
+public class TestNativeAzureFileSystemBlockCompaction extends AbstractWasbTestBase {
+
+  private static final String TEST_FILE = "/user/active/test.dat";
+  private static final Path TEST_PATH = new Path(TEST_FILE);
+
+  private static final String TEST_FILE_NORMAL = "/user/normal/test.dat";
+  private static final Path TEST_PATH_NORMAL = new Path(TEST_FILE_NORMAL);
+
+  private AzureBlobStorageTestAccount testAccount = null;
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    testAccount = createTestAccount();
+    fs = testAccount.getFileSystem();
+    Configuration conf = fs.getConf();
+    conf.setBoolean(NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME, true);
+    conf.set(AzureNativeFileSystemStore.KEY_BLOCK_BLOB_WITH_COMPACTION_DIRECTORIES, "/user/active");
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+  }
+
+  /*
+   * Helper method that creates test data of size provided by the
+   * "size" parameter.
+   */
+  private static byte[] getTestData(int size) {
+    byte[] testData = new byte[size];
+    System.arraycopy(RandomStringUtils.randomAlphabetic(size).getBytes(), 0, testData, 0, size);
+    return testData;
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  private BlockBlobAppendStream getBlockBlobAppendStream(FSDataOutputStream appendStream) {
+    SyncableDataOutputStream dataOutputStream = null;
+
+    if (appendStream.getWrappedStream() instanceof NativeAzureFileSystem.NativeAzureFsOutputStream) {
+      NativeAzureFileSystem.NativeAzureFsOutputStream fsOutputStream =
+              (NativeAzureFileSystem.NativeAzureFsOutputStream) appendStream.getWrappedStream();
+
+      dataOutputStream = (SyncableDataOutputStream) fsOutputStream.getOutStream();
+    }
+
+    if (appendStream.getWrappedStream() instanceof SyncableDataOutputStream) {
+      dataOutputStream = (SyncableDataOutputStream) appendStream.getWrappedStream();
+    }
+
+    Assert.assertNotNull("Did not recognize " + dataOutputStream,
+        dataOutputStream);
+
+    return (BlockBlobAppendStream) dataOutputStream.getOutStream();
+  }
+
+  private void verifyBlockList(BlockBlobAppendStream blockBlobStream,
+                               int[] testData) throws Throwable {
+    List<BlockEntry> blockList = blockBlobStream.getBlockList();
+    Assert.assertEquals("Block list length", testData.length, blockList.size());
+
+    int i = 0;
+    for (BlockEntry block: blockList) {
+      Assert.assertTrue(block.getSize() == testData[i++]);
+    }
+  }
+
+  private void appendBlockList(FSDataOutputStream fsStream,
+                              ByteArrayOutputStream memStream,
+                              int[] testData) throws Throwable {
+
+    for (int d: testData) {
+      byte[] data = getTestData(d);
+      memStream.write(data);
+      fsStream.write(data);
+    }
+    fsStream.hflush();
+  }
+
+  @Test
+  public void testCompactionDisabled() throws Throwable {
+
+    try (FSDataOutputStream appendStream = fs.create(TEST_PATH_NORMAL)) {
+
+      // testing new file
+
+      SyncableDataOutputStream dataOutputStream = null;
+
+      OutputStream wrappedStream = appendStream.getWrappedStream();
+      if (wrappedStream instanceof NativeAzureFileSystem.NativeAzureFsOutputStream) {
+        NativeAzureFileSystem.NativeAzureFsOutputStream fsOutputStream =
+                (NativeAzureFileSystem.NativeAzureFsOutputStream) wrappedStream;
+
+        dataOutputStream = (SyncableDataOutputStream) fsOutputStream.getOutStream();
+      } else if (wrappedStream instanceof SyncableDataOutputStream) {
+        dataOutputStream = (SyncableDataOutputStream) wrappedStream;
+      } else {
+        Assert.fail("Unable to determine type of " + wrappedStream
+            + " class of " + wrappedStream.getClass());
+      }
+
+      Assert.assertFalse("Data output stream is a BlockBlobAppendStream: "
+          + dataOutputStream,
+          dataOutputStream.getOutStream() instanceof BlockBlobAppendStream);
+
+    }
+  }
+
+  @Test
+  public void testCompaction() throws Throwable {
+
+    final int n2 = 2;
+    final int n4 = 4;
+    final int n10 = 10;
+    final int n12 = 12;
+    final int n14 = 14;
+    final int n16 = 16;
+
+    final int maxBlockSize = 16;
+    final int compactionBlockCount = 4;
+
+    ByteArrayOutputStream memStream = new ByteArrayOutputStream();
+
+    try (FSDataOutputStream appendStream = fs.create(TEST_PATH)) {
+
+      // test new file
+
+      BlockBlobAppendStream blockBlobStream = getBlockBlobAppendStream(appendStream);
+      blockBlobStream.setMaxBlockSize(maxBlockSize);
+      blockBlobStream.setCompactionBlockCount(compactionBlockCount);
+
+      appendBlockList(appendStream, memStream, new int[]{n2});
+      verifyBlockList(blockBlobStream, new int[]{n2});
+
+      appendStream.hflush();
+      verifyBlockList(blockBlobStream, new int[]{n2});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n2, n4});
+
+      appendStream.hsync();
+      verifyBlockList(blockBlobStream, new int[]{n2, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n2, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n2, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n2, n4, n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n12, n10});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n12, n10, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4, n4, n4, n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4, n4, n4, n4, n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+
+      appendStream.close();
+
+      ContractTestUtils.verifyFileContents(fs, TEST_PATH, memStream.toByteArray());
+    }
+
+    try (FSDataOutputStream appendStream = fs.append(TEST_PATH)) {
+
+      // test existing file
+
+      BlockBlobAppendStream blockBlobStream = getBlockBlobAppendStream(appendStream);
+      blockBlobStream.setMaxBlockSize(maxBlockSize);
+      blockBlobStream.setCompactionBlockCount(compactionBlockCount);
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n4, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n16, n4});
+
+      appendStream.close();
+
+      ContractTestUtils.verifyFileContents(fs, TEST_PATH, memStream.toByteArray());
+    }
+  }
+}

+ 1 - 0
hadoop-tools/hadoop-azure/src/test/resources/log4j.properties

@@ -23,3 +23,4 @@ log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n
 
 log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG
+log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG

+ 15 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -1026,10 +1026,6 @@ public class ApplicationMaster {
       }
     }
 
-    @Override
-    public void onContainerResourceIncreased(
-        ContainerId containerId, Resource resource) {}
-
     @Override
     public void onStartContainerError(ContainerId containerId, Throwable t) {
       LOG.error("Failed to start Container " + containerId, t);
@@ -1050,10 +1046,25 @@ public class ApplicationMaster {
       containers.remove(containerId);
     }
 
+    @Deprecated
     @Override
     public void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t) {}
 
+    @Deprecated
+    @Override
+    public void onContainerResourceIncreased(
+        ContainerId containerId, Resource resource) {}
+
+    @Override
+    public void onUpdateContainerResourceError(
+        ContainerId containerId, Throwable t) {
+    }
+
+    @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+    }
   }
 
   /**

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java

@@ -104,9 +104,27 @@ public abstract class NMClient extends AbstractService {
    * @throws YarnException YarnException.
    * @throws IOException IOException.
    */
+  @Deprecated
   public abstract void increaseContainerResource(Container container)
       throws YarnException, IOException;
 
+  /**
+   * <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.
+   */
+  public abstract void updateContainerResource(Container container)
+      throws YarnException, IOException;
+
   /**
    * <p>Stop an started container.</p>
    *

+ 36 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java

@@ -177,8 +177,22 @@ public abstract class NMClientAsync extends AbstractService {
   public abstract void startContainerAsync(
       Container container, ContainerLaunchContext containerLaunchContext);
 
+  @Deprecated
   public abstract void increaseContainerResourceAsync(Container container);
 
+  /**
+   * <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.
+   */
+  public abstract void updateContainerResourceAsync(Container container);
+
   /**
    * <p>Re-Initialize the Container.</p>
    *
@@ -301,9 +315,20 @@ public abstract class NMClientAsync extends AbstractService {
      * @param containerId the Id of the container
      * @param resource the target resource of the container
      */
+    @Deprecated
     public abstract void onContainerResourceIncreased(
         ContainerId containerId, Resource resource);
 
+    /**
+     * The API is called when <code>NodeManager</code> responds to indicate
+     * the container resource has been successfully updated.
+     *
+     * @param containerId the Id of the container
+     * @param resource the target resource of the container
+     */
+    public abstract void onContainerResourceUpdated(
+        ContainerId containerId, Resource resource);
+
     /**
      * The API is called when an exception is raised in the process of
      * querying the status of a container.
@@ -321,9 +346,20 @@ public abstract class NMClientAsync extends AbstractService {
      * @param containerId the Id of the container
      * @param t the raised exception
      */
+    @Deprecated
     public abstract void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t);
 
+    /**
+     * The API is called when an exception is raised in the process of
+     * updating container resource.
+     *
+     * @param containerId the Id of the container
+     * @param t the raised exception
+     */
+    public abstract void onUpdateContainerResourceError(
+        ContainerId containerId, Throwable t);
+
     /**
      * The API is called when an exception is raised in the process of
      * stopping a container.

+ 69 - 23
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java

@@ -259,6 +259,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
     }
   }
 
+  @Deprecated
   public void increaseContainerResourceAsync(Container container) {
     if (!(callbackHandler instanceof AbstractCallbackHandler)) {
       LOG.error("Callback handler does not implement container resource "
@@ -274,7 +275,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
                   " is neither started nor scheduled to start"));
     }
     try {
-      events.put(new IncreaseContainerResourceEvent(container));
+      events.put(new UpdateContainerResourceEvent(container, true));
     } catch (InterruptedException e) {
       LOG.warn("Exception when scheduling the event of increasing resource of "
           + "Container " + container.getId());
@@ -282,6 +283,30 @@ public class NMClientAsyncImpl extends NMClientAsync {
     }
   }
 
+  @Override
+  public void updateContainerResourceAsync(Container container) {
+    if (!(callbackHandler instanceof AbstractCallbackHandler)) {
+      LOG.error("Callback handler does not implement container resource "
+          + "increase callback methods");
+      return;
+    }
+    AbstractCallbackHandler handler = (AbstractCallbackHandler) callbackHandler;
+    if (containers.get(container.getId()) == null) {
+      handler.onUpdateContainerResourceError(
+          container.getId(),
+          RPCUtil.getRemoteException(
+              "Container " + container.getId() +
+                  " is neither started nor scheduled to start"));
+    }
+    try {
+      events.put(new UpdateContainerResourceEvent(container, false));
+    } catch (InterruptedException e) {
+      LOG.warn("Exception when scheduling the event of increasing resource of "
+          + "Container " + container.getId());
+      handler.onUpdateContainerResourceError(container.getId(), e);
+    }
+  }
+
   @Override
   public void reInitializeContainerAsync(ContainerId containerId,
       ContainerLaunchContext containerLaunchContex, boolean autoCommit){
@@ -427,7 +452,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
     START_CONTAINER,
     STOP_CONTAINER,
     QUERY_CONTAINER,
-    INCREASE_CONTAINER_RESOURCE,
+    UPDATE_CONTAINER_RESOURCE,
     REINITIALIZE_CONTAINER,
     RESTART_CONTAINER,
     ROLLBACK_LAST_REINIT,
@@ -503,14 +528,20 @@ public class NMClientAsyncImpl extends NMClientAsync {
     }
   }
 
-  protected static class IncreaseContainerResourceEvent extends ContainerEvent {
+  protected static class UpdateContainerResourceEvent extends ContainerEvent {
     private Container container;
+    private boolean isIncreaseEvent;
 
-    public IncreaseContainerResourceEvent(Container container) {
+    // UpdateContainerResourceEvent constructor takes in a
+    // flag to support callback API's calling through the deprecated
+    // increaseContainerResource
+    public UpdateContainerResourceEvent(Container container,
+        boolean isIncreaseEvent) {
       super(container.getId(), container.getNodeId(),
           container.getContainerToken(),
-              ContainerEventType.INCREASE_CONTAINER_RESOURCE);
+          ContainerEventType.UPDATE_CONTAINER_RESOURCE);
       this.container = container;
+      this.isIncreaseEvent = isIncreaseEvent;
     }
 
     public Container getContainer() {
@@ -536,8 +567,8 @@ public class NMClientAsyncImpl extends NMClientAsync {
 
             // Transitions from RUNNING state
             .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
-                ContainerEventType.INCREASE_CONTAINER_RESOURCE,
-                new IncreaseContainerResourceTransition())
+                ContainerEventType.UPDATE_CONTAINER_RESOURCE,
+                new UpdateContainerResourceTransition())
 
             // Transitions for Container Upgrade
             .addTransition(ContainerState.RUNNING,
@@ -566,7 +597,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
             .addTransition(ContainerState.DONE, ContainerState.DONE,
                 EnumSet.of(ContainerEventType.START_CONTAINER,
                     ContainerEventType.STOP_CONTAINER,
-                    ContainerEventType.INCREASE_CONTAINER_RESOURCE))
+                    ContainerEventType.UPDATE_CONTAINER_RESOURCE))
 
             // Transition from FAILED state
             .addTransition(ContainerState.FAILED, ContainerState.FAILED,
@@ -576,7 +607,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
                     ContainerEventType.RESTART_CONTAINER,
                     ContainerEventType.COMMIT_LAST_REINT,
                     ContainerEventType.ROLLBACK_LAST_REINIT,
-                    ContainerEventType.INCREASE_CONTAINER_RESOURCE));
+                    ContainerEventType.UPDATE_CONTAINER_RESOURCE));
 
     protected static class StartContainerTransition implements
         MultipleArcTransition<StatefulContainer, ContainerEvent,
@@ -628,46 +659,61 @@ public class NMClientAsyncImpl extends NMClientAsync {
       }
     }
 
-    protected static class IncreaseContainerResourceTransition implements
+    protected static class UpdateContainerResourceTransition implements
         SingleArcTransition<StatefulContainer, ContainerEvent> {
+
+      @SuppressWarnings("deprecation")
       @Override
       public void transition(
           StatefulContainer container, ContainerEvent event) {
+        boolean isIncreaseEvent = false;
         if (!(container.nmClientAsync.getCallbackHandler()
             instanceof AbstractCallbackHandler)) {
           LOG.error("Callback handler does not implement container resource "
-              + "increase callback methods");
+              + "update callback methods");
           return;
         }
         AbstractCallbackHandler handler =
             (AbstractCallbackHandler) container.nmClientAsync
                 .getCallbackHandler();
         try {
-          if (!(event instanceof IncreaseContainerResourceEvent)) {
+          if (!(event instanceof UpdateContainerResourceEvent)) {
             throw new AssertionError("Unexpected event type. Expecting:"
-                + "IncreaseContainerResourceEvent. Got:" + event);
+                + "UpdateContainerResourceEvent. Got:" + event);
           }
-          IncreaseContainerResourceEvent increaseEvent =
-              (IncreaseContainerResourceEvent) event;
-          container.nmClientAsync.getClient().increaseContainerResource(
-              increaseEvent.getContainer());
+          UpdateContainerResourceEvent updateEvent =
+              (UpdateContainerResourceEvent) event;
+          container.nmClientAsync.getClient().updateContainerResource(
+              updateEvent.getContainer());
+          isIncreaseEvent = updateEvent.isIncreaseEvent;
           try {
-            handler.onContainerResourceIncreased(
-                increaseEvent.getContainerId(), increaseEvent.getContainer()
-                    .getResource());
+            //If isIncreaseEvent is set, set the appropriate callbacks
+            //for backward compatibility
+            if (isIncreaseEvent) {
+              handler.onContainerResourceIncreased(updateEvent.getContainerId(),
+                  updateEvent.getContainer().getResource());
+            } else {
+              handler.onContainerResourceUpdated(updateEvent.getContainerId(),
+                  updateEvent.getContainer().getResource());
+            }
           } catch (Throwable thr) {
             // Don't process user created unchecked exception
             LOG.info("Unchecked exception is thrown from "
-                + "onContainerResourceIncreased for Container "
+                + "onContainerResourceUpdated for Container "
                 + event.getContainerId(), thr);
           }
         } catch (Exception e) {
           try {
-            handler.onIncreaseContainerResourceError(event.getContainerId(), e);
+            if (isIncreaseEvent) {
+              handler
+                  .onIncreaseContainerResourceError(event.getContainerId(), e);
+            } else {
+              handler.onUpdateContainerResourceError(event.getContainerId(), e);
+            }
           } catch (Throwable thr) {
             // Don't process user created unchecked exception
             LOG.info("Unchecked exception is thrown from "
-                + "onIncreaseContainerResourceError for Container "
+                + "onUpdateContainerResourceError for Container "
                 + event.getContainerId(), thr);
           }
         }

+ 29 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java

@@ -230,6 +230,7 @@ public class NMClientImpl extends NMClient {
     }
   }
 
+  @Deprecated
   @Override
   public void increaseContainerResource(Container container)
       throws YarnException, IOException {
@@ -258,6 +259,34 @@ public class NMClientImpl extends NMClient {
     }
   }
 
+  @Override
+  public void updateContainerResource(Container container)
+      throws YarnException, IOException {
+    ContainerManagementProtocolProxyData proxy = null;
+    try {
+      proxy =
+          cmProxy.getProxy(container.getNodeId().toString(), container.getId());
+      List<Token> updateTokens = new ArrayList<>();
+      updateTokens.add(container.getContainerToken());
+
+      ContainerUpdateRequest request =
+          ContainerUpdateRequest.newInstance(updateTokens);
+      ContainerUpdateResponse response =
+          proxy.getContainerManagementProtocol().updateContainer(request);
+
+      if (response.getFailedRequests() != null && response.getFailedRequests()
+          .containsKey(container.getId())) {
+        Throwable t =
+            response.getFailedRequests().get(container.getId()).deSerialize();
+        parseAndThrowException(t);
+      }
+    } finally {
+      if (proxy != null) {
+        cmProxy.mayBeCloseProxy(proxy);
+      }
+    }
+  }
+
   @Override
   public void stopContainer(ContainerId containerId, NodeId nodeId)
       throws YarnException, IOException {

+ 56 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java

@@ -253,7 +253,7 @@ public class TestNMClientAsync {
         int t = containerId.getId() % 5;
         switch (t) {
         case 0:
-          asyncClient.increaseContainerResourceAsync(container);
+          asyncClient.updateContainerResourceAsync(container);
           break;
         case 1:
           asyncClient.reInitializeContainerAsync(containerId,
@@ -295,7 +295,7 @@ public class TestNMClientAsync {
       // containerId
       Container container = Container.newInstance(
           containerId, nodeId, null, null, null, containerToken);
-      asyncClient.increaseContainerResourceAsync(container);
+      asyncClient.updateContainerResourceAsync(container);
 
       // Shouldn't crash the test thread
       throw new RuntimeException("Ignorable Exception");
@@ -320,6 +320,25 @@ public class TestNMClientAsync {
       throw new RuntimeException("Ignorable Exception");
     }
 
+    @SuppressWarnings("deprecation")
+    @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+      if (containerId.getId() >= expectedSuccess) {
+        errorMsgs.add("Container " + containerId +
+            " should throw the exception onContainerResourceUpdated");
+        return;
+      }
+      TestData td = testMap.get(OpsToTest.INCR);
+      td.success.addAndGet(1);
+      td.successArray.set(containerId.getId(), 1);
+      // move on to the following success tests
+      asyncClient.reInitializeContainerAsync(containerId,
+          Records.newRecord(ContainerLaunchContext.class), true);
+      // throw a fake user exception, and shouldn't crash the test
+      throw new RuntimeException("Ignorable Exception");
+    }
+
     @SuppressWarnings("deprecation")
     @Override
     public void onContainerReInitialize(ContainerId containerId) {
@@ -450,6 +469,27 @@ public class TestNMClientAsync {
       throw new RuntimeException("Ignorable Exception");
     }
 
+    @SuppressWarnings("deprecation")
+    @Override
+    public void onUpdateContainerResourceError(ContainerId containerId,
+        Throwable t) {
+      if (containerId.getId() < expectedSuccess + expectedFailure) {
+        errorMsgs.add("Container " + containerId +
+            " shouldn't throw the exception onUpdatedContainerResourceError");
+        return;
+      }
+      TestData td = testMap.get(OpsToTest.INCR);
+      td.failure.addAndGet(1);
+      td.failureArray.set(
+          containerId.getId() - expectedSuccess - expectedFailure, 1);
+      // increase container resource error should NOT change the
+      // the container status to FAILED
+      // move on to the following failure tests
+      asyncClient.stopContainerAsync(containerId, nodeId);
+      // Shouldn't crash the test thread
+      throw new RuntimeException("Ignorable Exception");
+    }
+
     @SuppressWarnings("deprecation")
     @Override
     public void onContainerReInitializeError(ContainerId containerId,
@@ -673,7 +713,7 @@ public class TestNMClientAsync {
         when(client.getContainerStatus(any(ContainerId.class),
             any(NodeId.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
-        doNothing().when(client).increaseContainerResource(
+        doNothing().when(client).updateContainerResource(
             any(Container.class));
         doNothing().when(client).reInitializeContainer(
             any(ContainerId.class), any(ContainerLaunchContext.class),
@@ -703,7 +743,7 @@ public class TestNMClientAsync {
             any(NodeId.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
         doThrow(RPCUtil.getRemoteException("Increase Resource Exception"))
-            .when(client).increaseContainerResource(any(Container.class));
+            .when(client).updateContainerResource(any(Container.class));
         doThrow(RPCUtil.getRemoteException("ReInitialize Exception"))
             .when(client).reInitializeContainer(
             any(ContainerId.class), any(ContainerLaunchContext.class),
@@ -818,10 +858,16 @@ public class TestNMClientAsync {
         ContainerStatus containerStatus) {
     }
 
+    @Deprecated
     @Override
     public void onContainerResourceIncreased(
         ContainerId containerId, Resource resource) {}
 
+    @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+    }
+
     @Override
     public void onContainerStopped(ContainerId containerId) {
     }
@@ -847,10 +893,16 @@ public class TestNMClientAsync {
         Throwable t) {
     }
 
+    @Deprecated
     @Override
     public void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t) {}
 
+    @Override
+    public void onUpdateContainerResourceError(ContainerId containerId,
+        Throwable t) {
+    }
+
     @Override
     public void onStopContainerError(ContainerId containerId, Throwable t) {
     }

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java

@@ -1428,6 +1428,7 @@ public class TestAMRMClient {
     amClient.ask.clear();
   }
 
+  @SuppressWarnings("deprecation")
   private void updateContainerExecType(AllocateResponse allocResponse,
       ExecutionType expectedExecType, NMClientImpl nmClient)
       throws IOException, YarnException {

+ 1 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java

@@ -29,7 +29,6 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -93,6 +92,7 @@ public class TestAMRMClientOnRMRestart {
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
     conf.setLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
+    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, "0.0.0.0:0");
   }
 
   @AfterClass
@@ -496,8 +496,6 @@ public class TestAMRMClientOnRMRestart {
       .getMasterKey().getKeyId() == newToken.decodeIdentifier().getKeyId());
 
     // start 2nd RM
-    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, "0.0.0.0:"
-        + ServerSocketUtil.getPort(45020, 10));
     final MyResourceManager2 rm2 = new MyResourceManager2(conf, memStore);
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java

@@ -301,10 +301,10 @@ public class TestNMClient {
         assertTrue("The thrown exception is not expected",
             e.getMessage().contains("is not handled by this NodeManager"));
       }
-      // increaseContainerResource shouldn't be called before startContainer,
+      // upadateContainerResource shouldn't be called before startContainer,
       // otherwise, NodeManager cannot find the container
       try {
-        nmClient.increaseContainerResource(container);
+        nmClient.updateContainerResource(container);
         fail("Exception is expected");
       } catch (YarnException e) {
         assertTrue("The thrown exception is not expected",
@@ -469,6 +469,7 @@ public class TestNMClient {
     }
   }
 
+  @SuppressWarnings("deprecation")
   private void testIncreaseContainerResource(Container container)
     throws YarnException, IOException {
     try {

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java

@@ -285,7 +285,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testFetchFinishedApplictionLogs() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -646,7 +646,7 @@ public class TestLogsCLI {
         any(ContainerId.class));
 
     // create local logs
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     FileSystem fs = FileSystem.get(configuration);
     String rootLogDir = "target/LocalLogs";
     Path rootLogDirPath = new Path(rootLogDir);
@@ -790,7 +790,7 @@ public class TestLogsCLI {
     UserGroupInformation testUgi = UserGroupInformation
         .createRemoteUser(testUser);
 
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -921,7 +921,7 @@ public class TestLogsCLI {
   public void testLogsCLIWithInvalidArgs() throws Exception {
     String localDir = "target/SaveLogs";
     Path localPath = new Path(localDir);
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     FileSystem fs = FileSystem.get(configuration);
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     YarnClient mockYarnClient =
@@ -992,7 +992,7 @@ public class TestLogsCLI {
     String localDir = "target/SaveLogs";
     Path localPath = new Path(localDir);
 
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1083,7 +1083,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testPrintContainerLogMetadata() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1188,7 +1188,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testListNodeInfo() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1239,7 +1239,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testFetchApplictionLogsHar() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);

+ 52 - 53
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java

@@ -44,11 +44,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.regex.Pattern;
-import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.input.BoundedInputStream;
 import org.apache.commons.io.output.WriterOutputStream;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.commons.math3.util.Pair;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -61,6 +58,7 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.file.tfile.TFile;
@@ -71,7 +69,8 @@ import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Times;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
@@ -81,7 +80,8 @@ import com.google.common.collect.Sets;
 @Evolving
 public class AggregatedLogFormat {
 
-  private static final Log LOG = LogFactory.getLog(AggregatedLogFormat.class);
+  private final static Logger LOG = LoggerFactory.getLogger(
+      AggregatedLogFormat.class);
   private static final LogKey APPLICATION_ACL_KEY = new LogKey("APPLICATION_ACL");
   private static final LogKey APPLICATION_OWNER_KEY = new LogKey("APPLICATION_OWNER");
   private static final LogKey VERSION_KEY = new LogKey("VERSION");
@@ -247,7 +247,7 @@ public class AggregatedLogFormat {
           in = secureOpenFile(logFile);
         } catch (IOException e) {
           logErrorMessage(logFile, e);
-          IOUtils.closeQuietly(in);
+          IOUtils.cleanupWithLogger(LOG, in);
           continue;
         }
 
@@ -285,7 +285,7 @@ public class AggregatedLogFormat {
           String message = logErrorMessage(logFile, e);
           out.write(message.getBytes(Charset.forName("UTF-8")));
         } finally {
-          IOUtils.closeQuietly(in);
+          IOUtils.cleanupWithLogger(LOG, in);
         }
       }
     }
@@ -555,7 +555,7 @@ public class AggregatedLogFormat {
       } catch (Exception e) {
         LOG.warn("Exception closing writer", e);
       } finally {
-        IOUtils.closeQuietly(this.fsDataOStream);
+        IOUtils.cleanupWithLogger(LOG, this.fsDataOStream);
       }
     }
   }
@@ -583,9 +583,9 @@ public class AggregatedLogFormat {
 
     /**
      * Returns the owner of the application.
-     * 
+     *
      * @return the application owner.
-     * @throws IOException
+     * @throws IOException if we can not get the application owner.
      */
     public String getApplicationOwner() throws IOException {
       TFile.Reader.Scanner ownerScanner = null;
@@ -603,16 +603,16 @@ public class AggregatedLogFormat {
         }
         return null;
       } finally {
-        IOUtils.closeQuietly(ownerScanner);
+        IOUtils.cleanupWithLogger(LOG, ownerScanner);
       }
     }
 
     /**
      * Returns ACLs for the application. An empty map is returned if no ACLs are
      * found.
-     * 
+     *
      * @return a map of the Application ACLs.
-     * @throws IOException
+     * @throws IOException if we can not get the application acls.
      */
     public Map<ApplicationAccessType, String> getApplicationAcls()
         throws IOException {
@@ -649,16 +649,17 @@ public class AggregatedLogFormat {
         }
         return acls;
       } finally {
-        IOUtils.closeQuietly(aclScanner);
+        IOUtils.cleanupWithLogger(LOG, aclScanner);
       }
     }
 
     /**
      * Read the next key and return the value-stream.
      * 
-     * @param key
-     * @return the valueStream if there are more keys or null otherwise.
-     * @throws IOException
+     * @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
      */
     public DataInputStream next(LogKey key) throws IOException {
       if (!this.atBeginning) {
@@ -683,10 +684,10 @@ public class AggregatedLogFormat {
      * Get a ContainerLogsReader to read the logs for
      * the specified container.
      *
-     * @param containerId
+     * @param containerId the containerId
      * @return object to read the container's logs or null if the
      *         logs could not be found
-     * @throws IOException
+     * @throws IOException if we can not get the container log reader.
      */
     @Private
     public ContainerLogsReader getContainerLogsReader(
@@ -752,10 +753,10 @@ public class AggregatedLogFormat {
 
     /**
      * Writes all logs for a single container to the provided writer.
-     * @param valueStream
-     * @param writer
-     * @param logUploadedTime
-     * @throws IOException
+     * @param valueStream the valueStream
+     * @param writer the log writer
+     * @param logUploadedTime the time stamp
+     * @throws IOException if we can not read the container logs.
      */
     public static void readAcontainerLogs(DataInputStream valueStream,
         Writer writer, long logUploadedTime) throws IOException {
@@ -773,16 +774,15 @@ public class AggregatedLogFormat {
           }
         }
       } finally {
-        IOUtils.closeQuietly(ps);
-        IOUtils.closeQuietly(os);
+        IOUtils.cleanupWithLogger(LOG, ps, os);
       }
     }
 
     /**
      * Writes all logs for a single container to the provided writer.
-     * @param valueStream
-     * @param writer
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param writer the log writer
+     * @throws IOException if we can not read the container logs.
      */
     public static void readAcontainerLogs(DataInputStream valueStream,
         Writer writer) throws IOException {
@@ -849,10 +849,11 @@ public class AggregatedLogFormat {
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container.
      * 
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @throws IOException
+     * @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.
      */
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime)
@@ -864,11 +865,11 @@ public class AggregatedLogFormat {
      * 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
-     * @param out
-     * @param logUploadedTime
-     * @param bytes
-     * @throws IOException
+     * @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
      */
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
@@ -880,9 +881,9 @@ public class AggregatedLogFormat {
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container.
      * 
-     * @param valueStream
-     * @param out
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the output print stream
+     * @throws IOException if we can not read the container log
      */
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out)
@@ -893,11 +894,11 @@ public class AggregatedLogFormat {
     /**
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * the specific types for a single container.
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @param logType
-     * @throws IOException
+     * @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
      */
     public static int readContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
@@ -909,11 +910,11 @@ public class AggregatedLogFormat {
     /**
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * the specific types for a single container.
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @param logType
-     * @throws IOException
+     * @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
      */
     public static int readContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
@@ -999,9 +1000,7 @@ public class AggregatedLogFormat {
     }
 
     public void close() {
-      IOUtils.closeQuietly(scanner);
-      IOUtils.closeQuietly(reader);
-      IOUtils.closeQuietly(fsDataIStream);
+      IOUtils.cleanupWithLogger(LOG, scanner, reader, fsDataIStream);
     }
   }
 

+ 16 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java

@@ -38,11 +38,11 @@ public class LogAggregationUtils {
 
   /**
    * Constructs the full filename for an application's log file per node.
-   * @param remoteRootLogDir
-   * @param appId
-   * @param user
-   * @param nodeId
-   * @param suffix
+   * @param remoteRootLogDir the aggregated remote root log dir
+   * @param appId the application Id
+   * @param user the application owner
+   * @param nodeId the node id
+   * @param suffix the log dir suffix
    * @return the remote log file.
    */
   public static Path getRemoteNodeLogFileForApp(Path remoteRootLogDir,
@@ -53,10 +53,10 @@ public class LogAggregationUtils {
 
   /**
    * Gets the remote app log dir.
-   * @param remoteRootLogDir
-   * @param appId
-   * @param user
-   * @param suffix
+   * @param remoteRootLogDir the aggregated log remote root log dir
+   * @param appId the application id
+   * @param user the application owner
+   * @param suffix the log directory suffix
    * @return the remote application specific log dir.
    */
   public static Path getRemoteAppLogDir(Path remoteRootLogDir,
@@ -67,9 +67,9 @@ public class LogAggregationUtils {
 
   /**
    * Gets the remote suffixed log dir for the user.
-   * @param remoteRootLogDir
-   * @param user
-   * @param suffix
+   * @param remoteRootLogDir the aggregated log remote root log dir
+   * @param user the application owner
+   * @param suffix the log dir suffix
    * @return the remote suffixed log dir.
    */
   public static Path getRemoteLogSuffixedDir(Path remoteRootLogDir,
@@ -83,8 +83,8 @@ public class LogAggregationUtils {
 
   /**
    * Gets the remote log user dir.
-   * @param remoteRootLogDir
-   * @param user
+   * @param remoteRootLogDir the aggregated log remote root log dir
+   * @param user the application owner
    * @return the remote per user log dir.
    */
   public static Path getRemoteLogUserDir(Path remoteRootLogDir, String user) {
@@ -93,7 +93,7 @@ public class LogAggregationUtils {
 
   /**
    * Returns the suffix component of the log dir.
-   * @param conf
+   * @param conf the configuration
    * @return the suffix which will be appended to the user log dir.
    */
   public static String getRemoteNodeLogDirSuffix(Configuration conf) {
@@ -104,7 +104,7 @@ public class LogAggregationUtils {
   
   /**
    * Converts a nodeId to a form used in the app log file name.
-   * @param nodeId
+   * @param nodeId the nodeId
    * @return the node string to be used to construct the file name.
    */
   @VisibleForTesting

+ 33 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java

@@ -128,8 +128,17 @@ public class LogCLIHelpers implements Configurable {
   @VisibleForTesting
   public int dumpAContainerLogsForLogType(ContainerLogsRequest options,
       boolean outputFailure) throws IOException {
-    boolean foundAnyLogs = this.getFileController(options.getAppId(),
-        options.getAppOwner()).readAggregatedLogs(options, null);
+    LogAggregationFileController fc = null;
+    try {
+      fc = this.getFileController(
+          options.getAppId(), options.getAppOwner());
+    } catch (IOException ex) {
+      System.err.println(ex);
+    }
+    boolean foundAnyLogs = false;
+    if (fc != null) {
+      foundAnyLogs = fc.readAggregatedLogs(options, null);
+    }
     if (!foundAnyLogs) {
       if (outputFailure) {
         containerLogNotFound(options.getContainerId());
@@ -142,9 +151,17 @@ public class LogCLIHelpers implements Configurable {
   @Private
   public int dumpAContainerLogsForLogTypeWithoutNodeId(
       ContainerLogsRequest options) throws IOException {
-    boolean foundAnyLogs = getFileController(options.getAppId(),
-        options.getAppOwner()).readAggregatedLogs(
-        options, null);
+    LogAggregationFileController fc = null;
+    try {
+      fc = this.getFileController(
+          options.getAppId(), options.getAppOwner());
+    } catch (IOException ex) {
+      System.err.println(ex);
+    }
+    boolean foundAnyLogs = false;
+    if (fc != null) {
+      foundAnyLogs = fc.readAggregatedLogs(options, null);
+    }
     if (!foundAnyLogs) {
       containerLogNotFound(options.getContainerId());
       return -1;
@@ -155,9 +172,17 @@ public class LogCLIHelpers implements Configurable {
   @Private
   public int dumpAllContainersLogs(ContainerLogsRequest options)
       throws IOException {
-    boolean foundAnyLogs = getFileController(options.getAppId(),
-        options.getAppOwner()).readAggregatedLogs(
-        options, null);
+    LogAggregationFileController fc = null;
+    try {
+      fc = this.getFileController(
+          options.getAppId(), options.getAppOwner());
+    } catch (IOException ex) {
+      System.err.println(ex);
+    }
+    boolean foundAnyLogs = false;
+    if (fc != null) {
+      foundAnyLogs = fc.readAggregatedLogs(options, null);
+    }
     if (!foundAnyLogs) {
       emptyLogDir(LogAggregationUtils.getRemoteAppLogDir(
           conf, options.getAppId(), options.getAppOwner())

+ 11 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java

@@ -37,9 +37,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -47,6 +44,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -56,6 +54,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
 import org.apache.hadoop.yarn.webapp.View.ViewContext;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
@@ -68,7 +68,7 @@ import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
 @Unstable
 public abstract class LogAggregationFileController {
 
-  private static final Log LOG = LogFactory.getLog(
+  private static final Logger LOG = LoggerFactory.getLogger(
       LogAggregationFileController.class);
 
   /*
@@ -193,7 +193,7 @@ public abstract class LogAggregationFileController {
 
   protected void closePrintStream(OutputStream out) {
     if (out != System.out) {
-      IOUtils.closeQuietly(out);
+      IOUtils.cleanupWithLogger(LOG, out);
     }
   }
 
@@ -201,6 +201,7 @@ public abstract class LogAggregationFileController {
    * Output container log.
    * @param logRequest {@link ContainerLogsRequest}
    * @param os the output stream
+   * @return true if we can read the aggregated logs successfully
    * @throws IOException if we can not access the log file.
    */
   public abstract boolean readAggregatedLogs(ContainerLogsRequest logRequest,
@@ -228,9 +229,9 @@ public abstract class LogAggregationFileController {
   /**
    * Returns the owner of the application.
    *
-   * @param the aggregatedLog path.
-   * @return the application owner.
-   * @throws IOException
+   * @param aggregatedLogPath the aggregatedLog path
+   * @return the application owner
+   * @throws IOException if we can not get the application owner
    */
   public abstract String getApplicationOwner(Path aggregatedLogPath)
       throws IOException;
@@ -239,9 +240,9 @@ public abstract class LogAggregationFileController {
    * Returns ACLs for the application. An empty map is returned if no ACLs are
    * found.
    *
-   * @param the aggregatedLog path.
+   * @param aggregatedLogPath the aggregatedLog path.
    * @return a map of the Application ACLs.
-   * @throws IOException
+   * @throws IOException if we can not get the application acls
    */
   public abstract Map<ApplicationAccessType, String> getApplicationAcls(
       Path aggregatedLogPath) throws IOException;

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java

@@ -172,7 +172,6 @@ public class LogAggregationTFileController
     byte[] buf = new byte[65535];
     while (nodeFiles != null && nodeFiles.hasNext()) {
       final FileStatus thisNodeFile = nodeFiles.next();
-      LOG.error(thisNodeFile.getPath().toString());
       String nodeName = thisNodeFile.getPath().getName();
       if (nodeName.equals(appId + ".har")) {
         Path p = new Path("har:///"

+ 8 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java

@@ -144,7 +144,8 @@ public class AppBlock extends HtmlBlock {
         && webUiType.equals(YarnWebParams.RM_WEB_UI)
         && conf.getBoolean(YarnConfiguration.RM_WEBAPP_UI_ACTIONS_ENABLED,
           YarnConfiguration.DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED)
-            && !unsecuredUIForSecuredCluster) {
+            && !unsecuredUIForSecuredCluster
+            && !isAppInFinalState(app)) {
       // Application Kill
       html.div()
         .button()
@@ -419,4 +420,10 @@ public class AppBlock extends HtmlBlock {
     }
     return ret;
   }
+
+  private boolean isAppInFinalState(AppInfo app) {
+    return app.getAppState() == YarnApplicationState.FINISHED
+        || app.getAppState() == YarnApplicationState.FAILED
+        || app.getAppState() == YarnApplicationState.KILLED;
+  }
 }

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -275,7 +274,7 @@ public class RLESparseResourceAllocation {
   public NavigableMap<Long, Resource> getCumulative() {
     readLock.lock();
     try {
-      return Collections.unmodifiableNavigableMap(cumulativeCapacity);
+      return cumulativeCapacity;
     } finally {
       readLock.unlock();
     }
@@ -409,7 +408,7 @@ public class RLESparseResourceAllocation {
       // test for negative value and throws
       if (operator == RLEOperator.subtractTestNonNegative
           && (Resources.fitsIn(val, ZERO_RESOURCE)
-              && !Resources.equals(val, ZERO_RESOURCE))) {
+          && !Resources.equals(val, ZERO_RESOURCE))) {
         throw new PlanningException(
             "RLESparseResourceAllocation: merge failed as the "
                 + "resulting RLESparseResourceAllocation would be negative");