Bladeren bron

Merge branch 'trunk' into HDFS-7240

Anu Engineer 7 jaren geleden
bovenliggende
commit
e319be93b9
49 gewijzigde bestanden met toevoegingen van 2215 en 754 verwijderingen
  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 = 
   private static final WeakHashMap<Configuration,Object> REGISTRY = 
     new WeakHashMap<Configuration,Object>();
     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 
    * List of default Resources. Resources are loaded in the order of the list 
    * entries
    * entries
@@ -738,6 +749,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   public Configuration(boolean loadDefaults) {
   public Configuration(boolean loadDefaults) {
     this.loadDefaults = loadDefaults;
     this.loadDefaults = loadDefaults;
     updatingResource = new ConcurrentHashMap<String, String[]>();
     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) {
     synchronized(Configuration.class) {
       REGISTRY.put(this, null);
       REGISTRY.put(this, null);
     }
     }
@@ -765,6 +782,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
      this.finalParameters = Collections.newSetFromMap(
      this.finalParameters = Collections.newSetFromMap(
          new ConcurrentHashMap<String, Boolean>());
          new ConcurrentHashMap<String, Boolean>());
      this.finalParameters.addAll(other.finalParameters);
      this.finalParameters.addAll(other.finalParameters);
+     this.REGISTERED_TAG_CLASS.putAll(other.REGISTERED_TAG_CLASS);
+     this.propertyTagsMap.putAll(other.propertyTagsMap);
    }
    }
    
    
     synchronized(Configuration.class) {
     synchronized(Configuration.class) {
@@ -2823,6 +2842,10 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
               } else if ("source".equals(propertyAttr)) {
               } else if ("source".equals(propertyAttr)) {
                 confSource.add(StringInterner.weakIntern(
                 confSource.add(StringInterner.weakIntern(
                     reader.getAttributeValue(i)));
                     reader.getAttributeValue(i)));
+              } else if ("tag".equals(propertyAttr)) {
+                //Read tags and put them in propertyTagsMap
+                readTagFromConfig(reader.getAttributeValue(i), confName,
+                    confValue, confSource);
               }
               }
             }
             }
             break;
             break;
@@ -2830,6 +2853,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           case "value":
           case "value":
           case "final":
           case "final":
           case "source":
           case "source":
+          case "tag":
             parseToken = true;
             parseToken = true;
             token.setLength(0);
             token.setLength(0);
             break;
             break;
@@ -2911,6 +2935,13 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           case "source":
           case "source":
             confSource.add(StringInterner.weakIntern(token.toString()));
             confSource.add(StringInterner.weakIntern(token.toString()));
             break;
             break;
+          case "tag":
+            if (token.length() > 0) {
+              //Read tags and put them in propertyTagsMap
+              readTagFromConfig(token.toString(), confName,
+                  confValue, confSource);
+            }
+            break;
           case "include":
           case "include":
             if (fallbackAllowed && !fallbackEntered) {
             if (fallbackAllowed && !fallbackEntered) {
               throw new IOException("Fetch fail on include for '"
               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) {
   private void overlay(Properties to, Properties from) {
     for (Entry<Object, Object> entry: from.entrySet()) {
     for (Entry<Object, Object> entry: from.entrySet()) {
       to.put(entry.getKey(), entry.getValue());
       to.put(entry.getKey(), entry.getValue());
@@ -3438,4 +3511,45 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
     }
     return false;
     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.
  * A simplified StopWatch implementation which can measure times in nanoseconds.
  */
  */
 public class StopWatch implements Closeable {
 public class StopWatch implements Closeable {
+  private final Timer timer;
   private boolean isStarted;
   private boolean isStarted;
   private long startNanos;
   private long startNanos;
   private long currentElapsedNanos;
   private long currentElapsedNanos;
 
 
   public StopWatch() {
   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");
       throw new IllegalStateException("StopWatch is already running");
     }
     }
     isStarted = true;
     isStarted = true;
-    startNanos = System.nanoTime();
+    startNanos = timer.monotonicNowNanos();
     return this;
     return this;
   }
   }
 
 
@@ -61,7 +72,7 @@ public class StopWatch implements Closeable {
     if (!isStarted) {
     if (!isStarted) {
       throw new IllegalStateException("StopWatch is already stopped");
       throw new IllegalStateException("StopWatch is already stopped");
     }
     }
-    long now = System.nanoTime();
+    long now = timer.monotonicNowNanos();
     isStarted = false;
     isStarted = false;
     currentElapsedNanos += now - startNanos;
     currentElapsedNanos += now - startNanos;
     return this;
     return this;
@@ -90,7 +101,7 @@ public class StopWatch implements Closeable {
    */
    */
   public long now() {
   public long now() {
     return isStarted ?
     return isStarted ?
-        System.nanoTime() - startNanos + currentElapsedNanos :
+        timer.monotonicNowNanos() - startNanos + currentElapsedNanos :
         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.HashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Random;
 import java.util.Random;
 import java.util.Set;
 import java.util.Set;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
@@ -53,7 +54,6 @@ import static org.junit.Assert.assertArrayEquals;
 
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -135,6 +135,7 @@ public class TestConfiguration {
 
 
   private void endConfig() throws IOException{
   private void endConfig() throws IOException{
     out.write("</configuration>\n");
     out.write("</configuration>\n");
+    out.flush();
     out.close();
     out.close();
   }
   }
 
 
@@ -577,6 +578,34 @@ public class TestConfiguration {
     out.write("</property>\n");
     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 {
   void appendCompactFormatProperty(String name, String val) throws IOException {
     appendCompactFormatProperty(name, val, false);
     appendCompactFormatProperty(name, val, false);
   }
   }
@@ -2215,4 +2244,85 @@ public class TestConfiguration {
       TestConfiguration.class.getName()
       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;
 package org.apache.hadoop.hdfs.protocol;
 
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
+
 /**
 /**
  * A response of add an ErasureCoding policy.
  * A response of add an ErasureCoding policy.
  */
  */
@@ -38,7 +40,7 @@ public class AddECPolicyResponse {
   }
   }
 
 
   public AddECPolicyResponse(ErasureCodingPolicy policy,
   public AddECPolicyResponse(ErasureCodingPolicy policy,
-      IllegalECPolicyException e) {
+      HadoopIllegalArgumentException e) {
     this(policy, e.getMessage());
     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_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_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_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 String  DFS_DATANODE_BP_READY_TIMEOUT_KEY = "dfs.datanode.bp-ready.timeout";
   public static final long    DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT = 20;
   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_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY = "dfs.journalnode.kerberos.internal.spnego.principal";
   public static final String DFS_JOURNALNODE_ENABLE_SYNC_KEY =
   public static final String DFS_JOURNALNODE_ENABLE_SYNC_KEY =
       "dfs.journalnode.enable.sync";
       "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 =
   public static final String DFS_JOURNALNODE_SYNC_INTERVAL_KEY =
       "dfs.journalnode.sync.interval";
       "dfs.journalnode.sync.interval";
   public static final long DFS_JOURNALNODE_SYNC_INTERVAL_DEFAULT = 2*60*1000L;
   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 com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 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.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -144,7 +144,7 @@ public final class ErasureCodingPolicyManager {
             policyName,
             policyName,
             DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
             DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
             names);
             names);
-        throw new IllegalArgumentException(msg);
+        throw new HadoopIllegalArgumentException(msg);
       }
       }
       enabledPoliciesByName.put(ecPolicy.getName(), ecPolicy);
       enabledPoliciesByName.put(ecPolicy.getName(), ecPolicy);
     }
     }
@@ -230,33 +230,34 @@ public final class ErasureCodingPolicyManager {
    * Add an erasure coding policy.
    * Add an erasure coding policy.
    * @return the added 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.
     // Set policy state into DISABLED when adding into Hadoop.
     policy.setState(ErasureCodingPolicyState.DISABLED);
     policy.setState(ErasureCodingPolicyState.DISABLED);
 
 
     if (!CodecUtil.hasCodec(policy.getCodecName())) {
     if (!CodecUtil.hasCodec(policy.getCodecName())) {
-      throw new IllegalECPolicyException("Codec name "
+      throw new HadoopIllegalArgumentException("Codec name "
           + policy.getCodecName() + " is not supported");
           + policy.getCodecName() + " is not supported");
     }
     }
 
 
     if (policy.getCellSize() > maxCellSize) {
     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(
     String assignedNewName = ErasureCodingPolicy.composePolicyName(
         policy.getSchema(), policy.getCellSize());
         policy.getSchema(), policy.getCellSize());
     for (ErasureCodingPolicy p : getPolicies()) {
     for (ErasureCodingPolicy p : getPolicies()) {
       if (p.getName().equals(assignedNewName)) {
       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()) &&
       if (p.getSchema().equals(policy.getSchema()) &&
           p.getCellSize() == policy.getCellSize()) {
           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 "
             + policy.getSchema().toString() + " and cell size "
-            + p.getCellSize() + " is already exists");
+            + p.getCellSize() + " already exists");
       }
       }
     }
     }
     policy.setName(assignedNewName);
     policy.setName(assignedNewName);
@@ -281,12 +282,12 @@ public final class ErasureCodingPolicyManager {
   public synchronized void removePolicy(String name) {
   public synchronized void removePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
     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()) {
     if (ecPolicy.isSystemPolicy()) {
-      throw new IllegalArgumentException("System erasure coding policy " +
+      throw new HadoopIllegalArgumentException("System erasure coding policy " +
           name + " cannot be removed");
           name + " cannot be removed");
     }
     }
 
 
@@ -317,8 +318,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void disablePolicy(String name) {
   public synchronized void disablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
     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)) {
     if (enabledPoliciesByName.containsKey(name)) {
@@ -336,8 +337,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void enablePolicy(String name) {
   public synchronized void enablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
     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);
     enabledPoliciesByName.put(name, ecPolicy);
@@ -346,4 +347,4 @@ public final class ErasureCodingPolicyManager {
         enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
         enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
     LOG.info("Enable the erasure coding policy " + name);
     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.DFSConfigKeys;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 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.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -212,7 +211,7 @@ final class FSDirErasureCodingOp {
   }
   }
 
 
   static ErasureCodingPolicy addErasureCodePolicy(final FSNamesystem fsn,
   static ErasureCodingPolicy addErasureCodePolicy(final FSNamesystem fsn,
-      ErasureCodingPolicy policy) throws IllegalECPolicyException {
+      ErasureCodingPolicy policy) {
     Preconditions.checkNotNull(policy);
     Preconditions.checkNotNull(policy);
     return fsn.getErasureCodingPolicyManager().addPolicy(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.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.SortedSet;
 import java.util.SortedSet;
@@ -202,6 +203,10 @@ public class FSDirectory implements Closeable {
 
 
   private INodeAttributeProvider attributeProvider;
   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) {
   public void setINodeAttributeProvider(INodeAttributeProvider provider) {
     attributeProvider = provider;
     attributeProvider = provider;
   }
   }
@@ -357,6 +362,49 @@ public class FSDirectory implements Closeable {
     this.quotaInitThreads = conf.getInt(
     this.quotaInitThreads = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_KEY,
         DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_KEY,
         DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_DEFAULT);
         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,
   FSPermissionChecker getPermissionChecker(String fsOwner, String superGroup,
       UserGroupInformation ugi) throws AccessControlException {
       UserGroupInformation ugi) throws AccessControlException {
     return new FSPermissionChecker(
     return new FSPermissionChecker(
-        fsOwner, superGroup, ugi, attributeProvider);
+        fsOwner, superGroup, ugi, getUserFilteredAttributeProvider(ugi));
   }
   }
 
 
   void checkOwner(FSPermissionChecker pc, INodesInPath iip)
   void checkOwner(FSPermissionChecker pc, INodesInPath iip)
@@ -1896,18 +1944,20 @@ public class FSDirectory implements Closeable {
   }
   }
 
 
   INodeAttributes getAttributes(INodesInPath iip)
   INodeAttributes getAttributes(INodesInPath iip)
-      throws FileNotFoundException {
+      throws IOException {
     INode node = FSDirectory.resolveLastINode(iip);
     INode node = FSDirectory.resolveLastINode(iip);
     int snapshot = iip.getPathSnapshotId();
     int snapshot = iip.getPathSnapshotId();
     INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
     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
       // permission checking sends the full components array including the
       // first empty component for the root.  however file status
       // first empty component for the root.  however file status
       // related calls are expected to strip out the root component according
       // related calls are expected to strip out the root component according
       // to TestINodeAttributeProvider.
       // to TestINodeAttributeProvider.
       byte[][] components = iip.getPathComponents();
       byte[][] components = iip.getPathComponents();
       components = Arrays.copyOfRange(components, 1, components.length);
       components = Arrays.copyOfRange(components, 1, components.length);
-      nodeAttrs = attributeProvider.getAttributes(components, nodeAttrs);
+      nodeAttrs = ap.getAttributes(components, nodeAttrs);
     }
     }
     return 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.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 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.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -7207,7 +7206,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
               FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
           addECPolicyName = newPolicy.getName();
           addECPolicyName = newPolicy.getName();
           responses.add(new AddECPolicyResponse(newPolicy));
           responses.add(new AddECPolicyResponse(newPolicy));
-        } catch (IllegalECPolicyException e) {
+        } catch (HadoopIllegalArgumentException e) {
           responses.add(new AddECPolicyResponse(policy, 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>
 <property>
   <name>dfs.journalnode.enable.sync</name>
   <name>dfs.journalnode.enable.sync</name>
-  <value>false</value>
+  <value>true</value>
   <description>
   <description>
     If true, the journal nodes wil sync with each other. The journal nodes
     If true, the journal nodes wil sync with each other. The journal nodes
     will periodically gossip with other journal nodes to compare edit log
     will periodically gossip with other journal nodes to compare edit log
@@ -4159,6 +4159,18 @@
   </description>
   </description>
 </property>
 </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>
 <property>
   <name>dfs.namenode.max-num-blocks-to-log</name>
   <name>dfs.namenode.max-num-blocks-to-log</name>
   <value>1000</value>
   <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");
         fs.enableErasureCodingPolicy("notExistECName");
         Assert.fail("enable the policy that doesn't exist should fail");
         Assert.fail("enable the policy that doesn't exist should fail");
       } catch (Exception e) {
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
         // pass
       }
       }
 
 
@@ -1626,7 +1626,7 @@ public class TestDistributedFileSystem {
         fs.disableErasureCodingPolicy("notExistECName");
         fs.disableErasureCodingPolicy("notExistECName");
         Assert.fail("disable the policy that doesn't exist should fail");
         Assert.fail("disable the policy that doesn't exist should fail");
       } catch (Exception e) {
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
         // 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 java.util.Set;
 
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 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.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 
 
 public class TestINodeAttributeProvider {
 public class TestINodeAttributeProvider {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestINodeAttributeProvider.class);
+
   private MiniDFSCluster miniDFS;
   private MiniDFSCluster miniDFS;
   private static final Set<String> CALLED = new HashSet<String>();
   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 {
   public static class MyAuthorizationProvider extends INodeAttributeProvider {
 
 
@@ -112,7 +119,8 @@ public class TestINodeAttributeProvider {
 
 
         @Override
         @Override
         public long getPermissionLong() {
         public long getPermissionLong() {
-          return (useDefault) ? inode.getPermissionLong() : 0770;
+          return (useDefault) ? inode.getPermissionLong() :
+            (long)PROVIDER_PERMISSION;
         }
         }
 
 
         @Override
         @Override
@@ -177,6 +185,9 @@ public class TestINodeAttributeProvider {
     conf.set(DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY,
     conf.set(DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY,
         MyAuthorizationProvider.class.getName());
         MyAuthorizationProvider.class.getName());
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
     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);
     EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
     miniDFS = new MiniDFSCluster.Builder(conf).build();
     miniDFS = new MiniDFSCluster.Builder(conf).build();
   }
   }
@@ -195,8 +206,11 @@ public class TestINodeAttributeProvider {
   public void testDelegationToProvider() throws Exception {
   public void testDelegationToProvider() throws Exception {
     Assert.assertTrue(CALLED.contains("start"));
     Assert.assertTrue(CALLED.contains("start"));
     FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
     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",
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting("u1",
         new String[]{"g1"});
         new String[]{"g1"});
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
@@ -204,17 +218,19 @@ public class TestINodeAttributeProvider {
       public Void run() throws Exception {
       public Void run() throws Exception {
         FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
         FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
         CALLED.clear();
         CALLED.clear();
-        fs.mkdirs(new Path("/tmp/foo"));
+        fs.mkdirs(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         Assert.assertTrue(CALLED.contains("checkPermission|WRITE|null|null"));
         Assert.assertTrue(CALLED.contains("checkPermission|WRITE|null|null"));
+
         CALLED.clear();
         CALLED.clear();
-        fs.listStatus(new Path("/tmp/foo"));
+        fs.listStatus(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(
         Assert.assertTrue(
             CALLED.contains("checkPermission|null|null|READ_EXECUTE"));
             CALLED.contains("checkPermission|null|null|READ_EXECUTE"));
+
         CALLED.clear();
         CALLED.clear();
-        fs.getAclStatus(new Path("/tmp/foo"));
+        fs.getAclStatus(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         return 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
   @Test
   public void testCustomProvider() throws Exception {
   public void testCustomProvider() throws Exception {
     FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
     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();
     streamStatistics.streamOpened();
     try {
     try {
       GetObjectRequest request = new GetObjectRequest(bucket, key)
       GetObjectRequest request = new GetObjectRequest(bucket, key)
-          .withRange(targetPos, contentRangeFinish);
+          .withRange(targetPos, contentRangeFinish - 1);
       if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
       if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
           StringUtils.isNotBlank(serverSideEncryptionKey)){
           StringUtils.isNotBlank(serverSideEncryptionKey)){
         request.setSSECustomerKey(new SSECustomerKey(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;
   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
    * Configuration key to indicate the set of directories in WASB where
    * we should do atomic folder rename synchronized with createNonRecursive.
    * we should do atomic folder rename synchronized with createNonRecursive.
@@ -527,6 +544,12 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     // User-agent
     // User-agent
     userAgentId = conf.get(USER_AGENT_ID_KEY, USER_AGENT_ID_DEFAULT);
     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.
     // Extract directories that should have atomic rename applied.
     atomicRenameDirs = getDirectorySet(KEY_ATOMIC_RENAME_DIRECTORIES);
     atomicRenameDirs = getDirectorySet(KEY_ATOMIC_RENAME_DIRECTORIES);
     String hbaseRoot;
     String hbaseRoot;
@@ -1164,6 +1187,17 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     return isKeyForDirectorySet(key, pageBlobDirs);
     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
    * Checks if the given key in Azure storage should have synchronized
    * atomic folder rename createNonRecursive implemented.
    * atomic folder rename createNonRecursive implemented.
@@ -1356,7 +1390,9 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   }
   }
 
 
   @Override
   @Override
-  public DataOutputStream storefile(String key, PermissionStatus permissionStatus)
+  public DataOutputStream storefile(String keyEncoded,
+                                    PermissionStatus permissionStatus,
+                                    String key)
       throws AzureException {
       throws AzureException {
     try {
     try {
 
 
@@ -1417,12 +1453,26 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
 
       // Get the blob reference from the store's container and
       // Get the blob reference from the store's container and
       // return it.
       // return it.
-      CloudBlobWrapper blob = getBlobReference(key);
+      CloudBlobWrapper blob = getBlobReference(keyEncoded);
       storePermissionStatus(blob, permissionStatus);
       storePermissionStatus(blob, permissionStatus);
 
 
       // Create the output stream for the Azure blob.
       // 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);
       DataOutputStream dataOutStream = new SyncableDataOutputStream(outputStream);
       return dataOutStream;
       return dataOutStream;
     } catch (Exception e) {
     } catch (Exception e) {
@@ -2869,10 +2919,21 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
 
       CloudBlobWrapper blob =  this.container.getBlockBlobReference(key);
       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) {
     } catch(Exception ex) {
       throw new AzureException(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.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 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.List;
 import java.util.UUID;
 import java.util.UUID;
 import java.util.Random;
 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.LinkedBlockingQueue;
+import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 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.hadoop.fs.FSExceptionMessages;
 import org.apache.commons.codec.binary.Base64;
 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.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.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 import com.microsoft.azure.storage.AccessCondition;
 import com.microsoft.azure.storage.AccessCondition;
 import com.microsoft.azure.storage.OperationContext;
 import com.microsoft.azure.storage.OperationContext;
 import com.microsoft.azure.storage.StorageException;
 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.BlobRequestOptions;
 import com.microsoft.azure.storage.blob.BlockEntry;
 import com.microsoft.azure.storage.blob.BlockEntry;
 import com.microsoft.azure.storage.blob.BlockListingFilter;
 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 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.
    * 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;
   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 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;
   private static final int MAX_BLOCK_COUNT = 100000;
 
 
+  /**
+   * The upload thread pool executor.
+   */
   private ThreadPoolExecutor ioThreadPool;
   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.
    * 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.
    * 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,
   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 {
           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.blob = blob;
     this.opContext = opContext;
     this.opContext = opContext;
     this.key = aKey;
     this.key = aKey;
-    this.bufferSize = bufferSize;
+    this.maxBlockSize = new AtomicInteger(bufferSize);
     this.threadSequenceNumber = new AtomicInteger(0);
     this.threadSequenceNumber = new AtomicInteger(0);
     this.blockIdPrefix = null;
     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 {
     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) });
     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
    * Writes length bytes from the specified byte array starting at offset to
    * this output stream.
    * this output stream.
@@ -312,529 +464,678 @@ public class BlockBlobAppendStream extends OutputStream {
    *           thrown if the output stream has been closed.
    *           thrown if the output stream has been closed.
    */
    */
   @Override
   @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 {
       throws IOException {
+    Preconditions.checkArgument(data != null, "null data");
 
 
     if (offset < 0 || length < 0 || length > data.length - offset) {
     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) {
     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 {
         try {
           lease.free();
           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.
    * @return String representing the block ID generated.
-   * @throws IOException
+   * @throws IOException if the stream is in invalid state
    */
    */
   private String generateBlockId() throws IOException {
   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.
    * @return String representing the block ID generated.
    */
    */
   private String generateOlderVersionBlockId(long id) {
   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.
    * @return String representing the block ID generated.
    */
    */
   private String generateNewerVersionBlockId(String prefix, long id) {
   private String generateNewerVersionBlockId(String prefix, long id) {
 
 
     String blockIdSuffix  = String.format("%06d", 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);
     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 {
         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();
           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;
           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.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Seekable;
 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.AzureFileSystemInstrumentation;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemMetricsSystem;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemMetricsSystem;
 import org.apache.hadoop.fs.azure.security.Constants;
 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.
      * method.
-     * 
+     *
      * Produce a string in double quotes with backslash sequences in all the
      * Produce a string in double quotes with backslash sequences in all the
      * right places. A backslash will be inserted within </, allowing JSON
      * right places. A backslash will be inserted within </, allowing JSON
      * text to be delivered in HTML. In JSON text, a string cannot contain a
      * 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 key;
     private String keyEncoded;
     private String keyEncoded;
     private OutputStream out;
     private OutputStream out;
@@ -983,6 +985,48 @@ public class NativeAzureFileSystem extends FileSystem {
       setEncodedKey(anEncodedKey);
       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
     @Override
     public synchronized void close() throws IOException {
     public synchronized void close() throws IOException {
       if (out != null) {
       if (out != null) {
@@ -990,8 +1034,11 @@ public class NativeAzureFileSystem extends FileSystem {
         // before returning to the caller.
         // before returning to the caller.
         //
         //
         out.close();
         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
      * 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,
      * <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
      * @param b
      *          Byte array to be written.
      *          Byte array to be written.
@@ -1749,7 +1796,7 @@ public class NativeAzureFileSystem extends FileSystem {
     OutputStream bufOutStream;
     OutputStream bufOutStream;
     if (store.isPageBlobKey(key)) {
     if (store.isPageBlobKey(key)) {
       // Store page blobs directly in-place without renames.
       // Store page blobs directly in-place without renames.
-      bufOutStream = store.storefile(key, permissionStatus);
+      bufOutStream = store.storefile(key, permissionStatus, key);
     } else {
     } else {
       // This is a block blob, so open the output blob stream based on the
       // This is a block blob, so open the output blob stream based on the
       // encoded key.
       // encoded key.
@@ -1777,7 +1824,7 @@ public class NativeAzureFileSystem extends FileSystem {
       // these
       // these
       // blocks.
       // blocks.
       bufOutStream = new NativeAzureFsOutputStream(store.storefile(
       bufOutStream = new NativeAzureFsOutputStream(store.storefile(
-          keyEncoded, permissionStatus), key, keyEncoded);
+          keyEncoded, permissionStatus, key), key, keyEncoded);
     }
     }
     // Construct the data output stream from the buffered output stream.
     // Construct the data output stream from the buffered output stream.
     FSDataOutputStream fsOut = new FSDataOutputStream(bufOutStream, statistics);
     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;
   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);
   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
     @Override
     public SelfRenewingLease acquireLease() throws StorageException {
     public SelfRenewingLease acquireLease() throws StorageException {
-      return new SelfRenewingLease(this);
+      return new SelfRenewingLease(this, false);
     }
     }
   }
   }
 
 
@@ -557,10 +557,12 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
     }
     }
 
 
     @Override
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition,
+        InputStream sourceStream,
         long length, BlobRequestOptions options,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
         OperationContext opContext) throws IOException, StorageException {
-      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, null, options, opContext);
+      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length,
+          accessCondition, options, opContext);
     }
     }
 
 
     @Override
     @Override
@@ -593,4 +595,4 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
           null, options, opContext);
           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 java.util.concurrent.atomic.AtomicInteger;
 
 
+import static com.microsoft.azure.storage.StorageErrorCodeStrings.LEASE_ALREADY_PRESENT;
+
 /**
 /**
  * An Azure blob lease that automatically renews itself indefinitely
  * An Azure blob lease that automatically renews itself indefinitely
  * using a background thread. Use it to synchronize distributed processes,
  * using a background thread. Use it to synchronize distributed processes,
@@ -66,7 +68,7 @@ public class SelfRenewingLease {
   @VisibleForTesting
   @VisibleForTesting
   static final int LEASE_ACQUIRE_RETRY_INTERVAL = 2000;
   static final int LEASE_ACQUIRE_RETRY_INTERVAL = 2000;
 
 
-  public SelfRenewingLease(CloudBlobWrapper blobWrapper)
+  public SelfRenewingLease(CloudBlobWrapper blobWrapper, boolean throwIfPresent)
       throws StorageException {
       throws StorageException {
 
 
     this.leaseFreed = false;
     this.leaseFreed = false;
@@ -79,10 +81,14 @@ public class SelfRenewingLease {
         leaseID = blob.acquireLease(LEASE_TIMEOUT, null);
         leaseID = blob.acquireLease(LEASE_TIMEOUT, null);
       } catch (StorageException e) {
       } catch (StorageException e) {
 
 
+        if (throwIfPresent && e.getErrorCode().equals(LEASE_ALREADY_PRESENT)) {
+          throw e;
+        }
+
         // Throw again if we don't want to keep waiting.
         // Throw again if we don't want to keep waiting.
         // We expect it to be that the lease is already present,
         // We expect it to be that the lease is already present,
         // or in some cases that the blob does not exist.
         // or in some cases that the blob does not exist.
-        if (!"LeaseAlreadyPresent".equals(e.getErrorCode())) {
+        if (!LEASE_ALREADY_PRESENT.equals(e.getErrorCode())) {
           LOG.info(
           LOG.info(
             "Caught exception when trying to get lease on blob "
             "Caught exception when trying to get lease on blob "
             + blobWrapper.getUri().toString() + ". " + e.getMessage());
             + 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
      * @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.
      *                     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
      * @param sourceStream An {@link InputStream} object that represents the input stream to write to the
      *                     block blob.
      *                     block blob.
      * @param length       A long which represents the length, in bytes, of the stream data,
      * @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 IOException  If an I/O error occurred.
      * @throws StorageException If a storage service 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,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException;
         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));
       return new CloudBlockBlobWrapperImpl(container.getBlockBlobReference(relativePath));
     }
     }
-    
+
     @Override
     @Override
     public CloudBlobWrapper getPageBlobReference(String relativePath)
     public CloudBlobWrapper getPageBlobReference(String relativePath)
         throws URISyntaxException, StorageException {
         throws URISyntaxException, StorageException {
@@ -286,7 +286,7 @@ class StorageInterfaceImpl extends StorageInterface {
     }
     }
 
 
   }
   }
-  
+
   abstract static class CloudBlobWrapperImpl implements CloudBlobWrapper {
   abstract static class CloudBlobWrapperImpl implements CloudBlobWrapper {
     private final CloudBlob blob;
     private final CloudBlob blob;
 
 
@@ -441,10 +441,10 @@ class StorageInterfaceImpl extends StorageInterface {
 
 
     @Override
     @Override
     public SelfRenewingLease acquireLease() throws StorageException {
     public SelfRenewingLease acquireLease() throws StorageException {
-      return new SelfRenewingLease(this);
+      return new SelfRenewingLease(this, false);
     }
     }
   }
   }
-  
+
 
 
   //
   //
   // CloudBlockBlobWrapperImpl
   // CloudBlockBlobWrapperImpl
@@ -479,10 +479,10 @@ class StorageInterfaceImpl extends StorageInterface {
     }
     }
 
 
     @Override
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition, InputStream sourceStream,
         long length, BlobRequestOptions options,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
         OperationContext opContext) throws IOException, StorageException {
-      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, null, options, opContext);
+      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, accessCondition, options, opContext);
     }
     }
 
 
     @Override
     @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.StreamCapabilities;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.classification.InterfaceAudience;
 
 
 /**
 /**
  * Support the Syncable interface on top of a DataOutputStream.
  * Support the Syncable interface on top of a DataOutputStream.
@@ -38,6 +39,16 @@ public class SyncableDataOutputStream extends DataOutputStream
     super(out);
     super(out);
   }
   }
 
 
+  /**
+   * Get a reference to the wrapped output stream.
+   *
+   * @return the underlying output stream
+   */
+  @InterfaceAudience.LimitedPrivate({"HDFS"})
+  public OutputStream getOutStream() {
+    return out;
+  }
+
   @Override
   @Override
   public boolean hasCapability(String capability) {
   public boolean hasCapability(String capability) {
     if (out instanceof StreamCapabilities) {
     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
 ### Page Blob Support and Configuration
 
 
 The Azure Blob Storage interface for Hadoop supports two kinds of blobs,
 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");
       throw new UnsupportedOperationException("downloadBlockList not used in Mock Tests");
     }
     }
     @Override
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition,
+        InputStream sourceStream,
         long length, BlobRequestOptions options,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
         OperationContext opContext) throws IOException, StorageException {
       throw new UnsupportedOperationException("uploadBlock not used in Mock Tests");
       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(
           outputStream = writerStorageAccount.getStore().storefile(
               key,
               key,
-              new PermissionStatus("", "", FsPermission.getDefault()));
+              new PermissionStatus("", "", FsPermission.getDefault()),
+              key);
 
 
           Arrays.fill(dataBlockWrite, (byte) (i % 256));
           Arrays.fill(dataBlockWrite, (byte) (i % 256));
           for (int j = 0; j < NUMBER_OF_BLOCKS; j++) {
           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.
    // reading.  This eliminates the race between the reader and writer threads.
    OutputStream outputStream = testAccount.getStore().storefile(
    OutputStream outputStream = testAccount.getStore().storefile(
        "WASB_String.txt",
        "WASB_String.txt",
-       new PermissionStatus("", "", FsPermission.getDefault()));
+       new PermissionStatus("", "", FsPermission.getDefault()),
+           "WASB_String.txt");
    Arrays.fill(dataBlockWrite, (byte) 255);
    Arrays.fill(dataBlockWrite, (byte) 255);
    for (int i = 0; i < NUMBER_OF_BLOCKS; i++) {
    for (int i = 0; i < NUMBER_OF_BLOCKS; i++) {
      outputStream.write(dataBlockWrite);
      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.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.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
     @Override
     public void onStartContainerError(ContainerId containerId, Throwable t) {
     public void onStartContainerError(ContainerId containerId, Throwable t) {
       LOG.error("Failed to start Container " + containerId, t);
       LOG.error("Failed to start Container " + containerId, t);
@@ -1050,10 +1046,25 @@ public class ApplicationMaster {
       containers.remove(containerId);
       containers.remove(containerId);
     }
     }
 
 
+    @Deprecated
     @Override
     @Override
     public void onIncreaseContainerResourceError(
     public void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t) {}
         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 YarnException YarnException.
    * @throws IOException IOException.
    * @throws IOException IOException.
    */
    */
+  @Deprecated
   public abstract void increaseContainerResource(Container container)
   public abstract void increaseContainerResource(Container container)
       throws YarnException, IOException;
       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>
    * <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(
   public abstract void startContainerAsync(
       Container container, ContainerLaunchContext containerLaunchContext);
       Container container, ContainerLaunchContext containerLaunchContext);
 
 
+  @Deprecated
   public abstract void increaseContainerResourceAsync(Container container);
   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>
    * <p>Re-Initialize the Container.</p>
    *
    *
@@ -301,9 +315,20 @@ public abstract class NMClientAsync extends AbstractService {
      * @param containerId the Id of the container
      * @param containerId the Id of the container
      * @param resource the target resource of the container
      * @param resource the target resource of the container
      */
      */
+    @Deprecated
     public abstract void onContainerResourceIncreased(
     public abstract void onContainerResourceIncreased(
         ContainerId containerId, Resource resource);
         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
      * The API is called when an exception is raised in the process of
      * querying the status of a container.
      * querying the status of a container.
@@ -321,9 +346,20 @@ public abstract class NMClientAsync extends AbstractService {
      * @param containerId the Id of the container
      * @param containerId the Id of the container
      * @param t the raised exception
      * @param t the raised exception
      */
      */
+    @Deprecated
     public abstract void onIncreaseContainerResourceError(
     public abstract void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t);
         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
      * The API is called when an exception is raised in the process of
      * stopping a container.
      * 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) {
   public void increaseContainerResourceAsync(Container container) {
     if (!(callbackHandler instanceof AbstractCallbackHandler)) {
     if (!(callbackHandler instanceof AbstractCallbackHandler)) {
       LOG.error("Callback handler does not implement container resource "
       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"));
                   " is neither started nor scheduled to start"));
     }
     }
     try {
     try {
-      events.put(new IncreaseContainerResourceEvent(container));
+      events.put(new UpdateContainerResourceEvent(container, true));
     } catch (InterruptedException e) {
     } catch (InterruptedException e) {
       LOG.warn("Exception when scheduling the event of increasing resource of "
       LOG.warn("Exception when scheduling the event of increasing resource of "
           + "Container " + container.getId());
           + "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
   @Override
   public void reInitializeContainerAsync(ContainerId containerId,
   public void reInitializeContainerAsync(ContainerId containerId,
       ContainerLaunchContext containerLaunchContex, boolean autoCommit){
       ContainerLaunchContext containerLaunchContex, boolean autoCommit){
@@ -427,7 +452,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
     START_CONTAINER,
     START_CONTAINER,
     STOP_CONTAINER,
     STOP_CONTAINER,
     QUERY_CONTAINER,
     QUERY_CONTAINER,
-    INCREASE_CONTAINER_RESOURCE,
+    UPDATE_CONTAINER_RESOURCE,
     REINITIALIZE_CONTAINER,
     REINITIALIZE_CONTAINER,
     RESTART_CONTAINER,
     RESTART_CONTAINER,
     ROLLBACK_LAST_REINIT,
     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 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(),
       super(container.getId(), container.getNodeId(),
           container.getContainerToken(),
           container.getContainerToken(),
-              ContainerEventType.INCREASE_CONTAINER_RESOURCE);
+          ContainerEventType.UPDATE_CONTAINER_RESOURCE);
       this.container = container;
       this.container = container;
+      this.isIncreaseEvent = isIncreaseEvent;
     }
     }
 
 
     public Container getContainer() {
     public Container getContainer() {
@@ -536,8 +567,8 @@ public class NMClientAsyncImpl extends NMClientAsync {
 
 
             // Transitions from RUNNING state
             // Transitions from RUNNING state
             .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
             .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
-                ContainerEventType.INCREASE_CONTAINER_RESOURCE,
-                new IncreaseContainerResourceTransition())
+                ContainerEventType.UPDATE_CONTAINER_RESOURCE,
+                new UpdateContainerResourceTransition())
 
 
             // Transitions for Container Upgrade
             // Transitions for Container Upgrade
             .addTransition(ContainerState.RUNNING,
             .addTransition(ContainerState.RUNNING,
@@ -566,7 +597,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
             .addTransition(ContainerState.DONE, ContainerState.DONE,
             .addTransition(ContainerState.DONE, ContainerState.DONE,
                 EnumSet.of(ContainerEventType.START_CONTAINER,
                 EnumSet.of(ContainerEventType.START_CONTAINER,
                     ContainerEventType.STOP_CONTAINER,
                     ContainerEventType.STOP_CONTAINER,
-                    ContainerEventType.INCREASE_CONTAINER_RESOURCE))
+                    ContainerEventType.UPDATE_CONTAINER_RESOURCE))
 
 
             // Transition from FAILED state
             // Transition from FAILED state
             .addTransition(ContainerState.FAILED, ContainerState.FAILED,
             .addTransition(ContainerState.FAILED, ContainerState.FAILED,
@@ -576,7 +607,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
                     ContainerEventType.RESTART_CONTAINER,
                     ContainerEventType.RESTART_CONTAINER,
                     ContainerEventType.COMMIT_LAST_REINT,
                     ContainerEventType.COMMIT_LAST_REINT,
                     ContainerEventType.ROLLBACK_LAST_REINIT,
                     ContainerEventType.ROLLBACK_LAST_REINIT,
-                    ContainerEventType.INCREASE_CONTAINER_RESOURCE));
+                    ContainerEventType.UPDATE_CONTAINER_RESOURCE));
 
 
     protected static class StartContainerTransition implements
     protected static class StartContainerTransition implements
         MultipleArcTransition<StatefulContainer, ContainerEvent,
         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> {
         SingleArcTransition<StatefulContainer, ContainerEvent> {
+
+      @SuppressWarnings("deprecation")
       @Override
       @Override
       public void transition(
       public void transition(
           StatefulContainer container, ContainerEvent event) {
           StatefulContainer container, ContainerEvent event) {
+        boolean isIncreaseEvent = false;
         if (!(container.nmClientAsync.getCallbackHandler()
         if (!(container.nmClientAsync.getCallbackHandler()
             instanceof AbstractCallbackHandler)) {
             instanceof AbstractCallbackHandler)) {
           LOG.error("Callback handler does not implement container resource "
           LOG.error("Callback handler does not implement container resource "
-              + "increase callback methods");
+              + "update callback methods");
           return;
           return;
         }
         }
         AbstractCallbackHandler handler =
         AbstractCallbackHandler handler =
             (AbstractCallbackHandler) container.nmClientAsync
             (AbstractCallbackHandler) container.nmClientAsync
                 .getCallbackHandler();
                 .getCallbackHandler();
         try {
         try {
-          if (!(event instanceof IncreaseContainerResourceEvent)) {
+          if (!(event instanceof UpdateContainerResourceEvent)) {
             throw new AssertionError("Unexpected event type. Expecting:"
             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 {
           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) {
           } catch (Throwable thr) {
             // Don't process user created unchecked exception
             // Don't process user created unchecked exception
             LOG.info("Unchecked exception is thrown from "
             LOG.info("Unchecked exception is thrown from "
-                + "onContainerResourceIncreased for Container "
+                + "onContainerResourceUpdated for Container "
                 + event.getContainerId(), thr);
                 + event.getContainerId(), thr);
           }
           }
         } catch (Exception e) {
         } catch (Exception e) {
           try {
           try {
-            handler.onIncreaseContainerResourceError(event.getContainerId(), e);
+            if (isIncreaseEvent) {
+              handler
+                  .onIncreaseContainerResourceError(event.getContainerId(), e);
+            } else {
+              handler.onUpdateContainerResourceError(event.getContainerId(), e);
+            }
           } catch (Throwable thr) {
           } catch (Throwable thr) {
             // Don't process user created unchecked exception
             // Don't process user created unchecked exception
             LOG.info("Unchecked exception is thrown from "
             LOG.info("Unchecked exception is thrown from "
-                + "onIncreaseContainerResourceError for Container "
+                + "onUpdateContainerResourceError for Container "
                 + event.getContainerId(), thr);
                 + 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
   @Override
   public void increaseContainerResource(Container container)
   public void increaseContainerResource(Container container)
       throws YarnException, IOException {
       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
   @Override
   public void stopContainer(ContainerId containerId, NodeId nodeId)
   public void stopContainer(ContainerId containerId, NodeId nodeId)
       throws YarnException, IOException {
       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;
         int t = containerId.getId() % 5;
         switch (t) {
         switch (t) {
         case 0:
         case 0:
-          asyncClient.increaseContainerResourceAsync(container);
+          asyncClient.updateContainerResourceAsync(container);
           break;
           break;
         case 1:
         case 1:
           asyncClient.reInitializeContainerAsync(containerId,
           asyncClient.reInitializeContainerAsync(containerId,
@@ -295,7 +295,7 @@ public class TestNMClientAsync {
       // containerId
       // containerId
       Container container = Container.newInstance(
       Container container = Container.newInstance(
           containerId, nodeId, null, null, null, containerToken);
           containerId, nodeId, null, null, null, containerToken);
-      asyncClient.increaseContainerResourceAsync(container);
+      asyncClient.updateContainerResourceAsync(container);
 
 
       // Shouldn't crash the test thread
       // Shouldn't crash the test thread
       throw new RuntimeException("Ignorable Exception");
       throw new RuntimeException("Ignorable Exception");
@@ -320,6 +320,25 @@ public class TestNMClientAsync {
       throw new RuntimeException("Ignorable Exception");
       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")
     @SuppressWarnings("deprecation")
     @Override
     @Override
     public void onContainerReInitialize(ContainerId containerId) {
     public void onContainerReInitialize(ContainerId containerId) {
@@ -450,6 +469,27 @@ public class TestNMClientAsync {
       throw new RuntimeException("Ignorable Exception");
       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")
     @SuppressWarnings("deprecation")
     @Override
     @Override
     public void onContainerReInitializeError(ContainerId containerId,
     public void onContainerReInitializeError(ContainerId containerId,
@@ -673,7 +713,7 @@ public class TestNMClientAsync {
         when(client.getContainerStatus(any(ContainerId.class),
         when(client.getContainerStatus(any(ContainerId.class),
             any(NodeId.class))).thenReturn(
             any(NodeId.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
                 recordFactory.newRecordInstance(ContainerStatus.class));
-        doNothing().when(client).increaseContainerResource(
+        doNothing().when(client).updateContainerResource(
             any(Container.class));
             any(Container.class));
         doNothing().when(client).reInitializeContainer(
         doNothing().when(client).reInitializeContainer(
             any(ContainerId.class), any(ContainerLaunchContext.class),
             any(ContainerId.class), any(ContainerLaunchContext.class),
@@ -703,7 +743,7 @@ public class TestNMClientAsync {
             any(NodeId.class))).thenReturn(
             any(NodeId.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
                 recordFactory.newRecordInstance(ContainerStatus.class));
         doThrow(RPCUtil.getRemoteException("Increase Resource Exception"))
         doThrow(RPCUtil.getRemoteException("Increase Resource Exception"))
-            .when(client).increaseContainerResource(any(Container.class));
+            .when(client).updateContainerResource(any(Container.class));
         doThrow(RPCUtil.getRemoteException("ReInitialize Exception"))
         doThrow(RPCUtil.getRemoteException("ReInitialize Exception"))
             .when(client).reInitializeContainer(
             .when(client).reInitializeContainer(
             any(ContainerId.class), any(ContainerLaunchContext.class),
             any(ContainerId.class), any(ContainerLaunchContext.class),
@@ -818,10 +858,16 @@ public class TestNMClientAsync {
         ContainerStatus containerStatus) {
         ContainerStatus containerStatus) {
     }
     }
 
 
+    @Deprecated
     @Override
     @Override
     public void onContainerResourceIncreased(
     public void onContainerResourceIncreased(
         ContainerId containerId, Resource resource) {}
         ContainerId containerId, Resource resource) {}
 
 
+    @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+    }
+
     @Override
     @Override
     public void onContainerStopped(ContainerId containerId) {
     public void onContainerStopped(ContainerId containerId) {
     }
     }
@@ -847,10 +893,16 @@ public class TestNMClientAsync {
         Throwable t) {
         Throwable t) {
     }
     }
 
 
+    @Deprecated
     @Override
     @Override
     public void onIncreaseContainerResourceError(
     public void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t) {}
         ContainerId containerId, Throwable t) {}
 
 
+    @Override
+    public void onUpdateContainerResourceError(ContainerId containerId,
+        Throwable t) {
+    }
+
     @Override
     @Override
     public void onStopContainerError(ContainerId containerId, Throwable t) {
     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();
     amClient.ask.clear();
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   private void updateContainerExecType(AllocateResponse allocResponse,
   private void updateContainerExecType(AllocateResponse allocResponse,
       ExecutionType expectedExecType, NMClientImpl nmClient)
       ExecutionType expectedExecType, NMClientImpl nmClient)
       throws IOException, YarnException {
       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.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -93,6 +92,7 @@ public class TestAMRMClientOnRMRestart {
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
     conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
     conf.setLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
     conf.setLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
+    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, "0.0.0.0:0");
   }
   }
 
 
   @AfterClass
   @AfterClass
@@ -496,8 +496,6 @@ public class TestAMRMClientOnRMRestart {
       .getMasterKey().getKeyId() == newToken.decodeIdentifier().getKeyId());
       .getMasterKey().getKeyId() == newToken.decodeIdentifier().getKeyId());
 
 
     // start 2nd RM
     // start 2nd RM
-    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, "0.0.0.0:"
-        + ServerSocketUtil.getPort(45020, 10));
     final MyResourceManager2 rm2 = new MyResourceManager2(conf, memStore);
     final MyResourceManager2 rm2 = new MyResourceManager2(conf, memStore);
     rm2.start();
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     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",
         assertTrue("The thrown exception is not expected",
             e.getMessage().contains("is not handled by this NodeManager"));
             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
       // otherwise, NodeManager cannot find the container
       try {
       try {
-        nmClient.increaseContainerResource(container);
+        nmClient.updateContainerResource(container);
         fail("Exception is expected");
         fail("Exception is expected");
       } catch (YarnException e) {
       } catch (YarnException e) {
         assertTrue("The thrown exception is not expected",
         assertTrue("The thrown exception is not expected",
@@ -469,6 +469,7 @@ public class TestNMClient {
     }
     }
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   private void testIncreaseContainerResource(Container container)
   private void testIncreaseContainerResource(Container container)
     throws YarnException, IOException {
     throws YarnException, IOException {
     try {
     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)
   @Test (timeout = 15000)
   public void testFetchFinishedApplictionLogs() throws Exception {
   public void testFetchFinishedApplictionLogs() throws Exception {
     String remoteLogRootDir = "target/logs/";
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -646,7 +646,7 @@ public class TestLogsCLI {
         any(ContainerId.class));
         any(ContainerId.class));
 
 
     // create local logs
     // create local logs
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     FileSystem fs = FileSystem.get(configuration);
     FileSystem fs = FileSystem.get(configuration);
     String rootLogDir = "target/LocalLogs";
     String rootLogDir = "target/LocalLogs";
     Path rootLogDirPath = new Path(rootLogDir);
     Path rootLogDirPath = new Path(rootLogDir);
@@ -790,7 +790,7 @@ public class TestLogsCLI {
     UserGroupInformation testUgi = UserGroupInformation
     UserGroupInformation testUgi = UserGroupInformation
         .createRemoteUser(testUser);
         .createRemoteUser(testUser);
 
 
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -921,7 +921,7 @@ public class TestLogsCLI {
   public void testLogsCLIWithInvalidArgs() throws Exception {
   public void testLogsCLIWithInvalidArgs() throws Exception {
     String localDir = "target/SaveLogs";
     String localDir = "target/SaveLogs";
     Path localPath = new Path(localDir);
     Path localPath = new Path(localDir);
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     FileSystem fs = FileSystem.get(configuration);
     FileSystem fs = FileSystem.get(configuration);
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     YarnClient mockYarnClient =
     YarnClient mockYarnClient =
@@ -992,7 +992,7 @@ public class TestLogsCLI {
     String localDir = "target/SaveLogs";
     String localDir = "target/SaveLogs";
     Path localPath = new Path(localDir);
     Path localPath = new Path(localDir);
 
 
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1083,7 +1083,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   @Test (timeout = 15000)
   public void testPrintContainerLogMetadata() throws Exception {
   public void testPrintContainerLogMetadata() throws Exception {
     String remoteLogRootDir = "target/logs/";
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1188,7 +1188,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   @Test (timeout = 15000)
   public void testListNodeInfo() throws Exception {
   public void testListNodeInfo() throws Exception {
     String remoteLogRootDir = "target/logs/";
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1239,7 +1239,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   @Test (timeout = 15000)
   public void testFetchApplictionLogsHar() throws Exception {
   public void testFetchApplictionLogsHar() throws Exception {
     String remoteLogRootDir = "target/logs/";
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
         .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.Map.Entry;
 import java.util.Set;
 import java.util.Set;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
-import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.input.BoundedInputStream;
 import org.apache.commons.io.input.BoundedInputStream;
 import org.apache.commons.io.output.WriterOutputStream;
 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.commons.math3.util.Pair;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 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.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.file.tfile.TFile;
 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.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Times;
 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.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterables;
@@ -81,7 +80,8 @@ import com.google.common.collect.Sets;
 @Evolving
 @Evolving
 public class AggregatedLogFormat {
 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_ACL_KEY = new LogKey("APPLICATION_ACL");
   private static final LogKey APPLICATION_OWNER_KEY = new LogKey("APPLICATION_OWNER");
   private static final LogKey APPLICATION_OWNER_KEY = new LogKey("APPLICATION_OWNER");
   private static final LogKey VERSION_KEY = new LogKey("VERSION");
   private static final LogKey VERSION_KEY = new LogKey("VERSION");
@@ -247,7 +247,7 @@ public class AggregatedLogFormat {
           in = secureOpenFile(logFile);
           in = secureOpenFile(logFile);
         } catch (IOException e) {
         } catch (IOException e) {
           logErrorMessage(logFile, e);
           logErrorMessage(logFile, e);
-          IOUtils.closeQuietly(in);
+          IOUtils.cleanupWithLogger(LOG, in);
           continue;
           continue;
         }
         }
 
 
@@ -285,7 +285,7 @@ public class AggregatedLogFormat {
           String message = logErrorMessage(logFile, e);
           String message = logErrorMessage(logFile, e);
           out.write(message.getBytes(Charset.forName("UTF-8")));
           out.write(message.getBytes(Charset.forName("UTF-8")));
         } finally {
         } finally {
-          IOUtils.closeQuietly(in);
+          IOUtils.cleanupWithLogger(LOG, in);
         }
         }
       }
       }
     }
     }
@@ -555,7 +555,7 @@ public class AggregatedLogFormat {
       } catch (Exception e) {
       } catch (Exception e) {
         LOG.warn("Exception closing writer", e);
         LOG.warn("Exception closing writer", e);
       } finally {
       } finally {
-        IOUtils.closeQuietly(this.fsDataOStream);
+        IOUtils.cleanupWithLogger(LOG, this.fsDataOStream);
       }
       }
     }
     }
   }
   }
@@ -583,9 +583,9 @@ public class AggregatedLogFormat {
 
 
     /**
     /**
      * Returns the owner of the application.
      * Returns the owner of the application.
-     * 
+     *
      * @return the application owner.
      * @return the application owner.
-     * @throws IOException
+     * @throws IOException if we can not get the application owner.
      */
      */
     public String getApplicationOwner() throws IOException {
     public String getApplicationOwner() throws IOException {
       TFile.Reader.Scanner ownerScanner = null;
       TFile.Reader.Scanner ownerScanner = null;
@@ -603,16 +603,16 @@ public class AggregatedLogFormat {
         }
         }
         return null;
         return null;
       } finally {
       } finally {
-        IOUtils.closeQuietly(ownerScanner);
+        IOUtils.cleanupWithLogger(LOG, ownerScanner);
       }
       }
     }
     }
 
 
     /**
     /**
      * Returns ACLs for the application. An empty map is returned if no ACLs are
      * Returns ACLs for the application. An empty map is returned if no ACLs are
      * found.
      * found.
-     * 
+     *
      * @return a map of the Application ACLs.
      * @return a map of the Application ACLs.
-     * @throws IOException
+     * @throws IOException if we can not get the application acls.
      */
      */
     public Map<ApplicationAccessType, String> getApplicationAcls()
     public Map<ApplicationAccessType, String> getApplicationAcls()
         throws IOException {
         throws IOException {
@@ -649,16 +649,17 @@ public class AggregatedLogFormat {
         }
         }
         return acls;
         return acls;
       } finally {
       } finally {
-        IOUtils.closeQuietly(aclScanner);
+        IOUtils.cleanupWithLogger(LOG, aclScanner);
       }
       }
     }
     }
 
 
     /**
     /**
      * Read the next key and return the value-stream.
      * 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 {
     public DataInputStream next(LogKey key) throws IOException {
       if (!this.atBeginning) {
       if (!this.atBeginning) {
@@ -683,10 +684,10 @@ public class AggregatedLogFormat {
      * Get a ContainerLogsReader to read the logs for
      * Get a ContainerLogsReader to read the logs for
      * the specified container.
      * the specified container.
      *
      *
-     * @param containerId
+     * @param containerId the containerId
      * @return object to read the container's logs or null if the
      * @return object to read the container's logs or null if the
      *         logs could not be found
      *         logs could not be found
-     * @throws IOException
+     * @throws IOException if we can not get the container log reader.
      */
      */
     @Private
     @Private
     public ContainerLogsReader getContainerLogsReader(
     public ContainerLogsReader getContainerLogsReader(
@@ -752,10 +753,10 @@ public class AggregatedLogFormat {
 
 
     /**
     /**
      * Writes all logs for a single container to the provided writer.
      * 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,
     public static void readAcontainerLogs(DataInputStream valueStream,
         Writer writer, long logUploadedTime) throws IOException {
         Writer writer, long logUploadedTime) throws IOException {
@@ -773,16 +774,15 @@ public class AggregatedLogFormat {
           }
           }
         }
         }
       } finally {
       } finally {
-        IOUtils.closeQuietly(ps);
-        IOUtils.closeQuietly(os);
+        IOUtils.cleanupWithLogger(LOG, ps, os);
       }
       }
     }
     }
 
 
     /**
     /**
      * Writes all logs for a single container to the provided writer.
      * 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,
     public static void readAcontainerLogs(DataInputStream valueStream,
         Writer writer) throws IOException {
         Writer writer) throws IOException {
@@ -849,10 +849,11 @@ public class AggregatedLogFormat {
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container.
      * 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(
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime)
         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
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container for the specific bytes.
      * 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(
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
         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
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container.
      * 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(
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out)
         DataInputStream valueStream, PrintStream out)
@@ -893,11 +894,11 @@ public class AggregatedLogFormat {
     /**
     /**
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * the specific types for a single container.
      * 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(
     public static int readContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
         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
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * the specific types for a single container.
      * 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(
     public static int readContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
@@ -999,9 +1000,7 @@ public class AggregatedLogFormat {
     }
     }
 
 
     public void close() {
     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.
    * 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.
    * @return the remote log file.
    */
    */
   public static Path getRemoteNodeLogFileForApp(Path remoteRootLogDir,
   public static Path getRemoteNodeLogFileForApp(Path remoteRootLogDir,
@@ -53,10 +53,10 @@ public class LogAggregationUtils {
 
 
   /**
   /**
    * Gets the remote app log dir.
    * 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.
    * @return the remote application specific log dir.
    */
    */
   public static Path getRemoteAppLogDir(Path remoteRootLogDir,
   public static Path getRemoteAppLogDir(Path remoteRootLogDir,
@@ -67,9 +67,9 @@ public class LogAggregationUtils {
 
 
   /**
   /**
    * Gets the remote suffixed log dir for the user.
    * 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.
    * @return the remote suffixed log dir.
    */
    */
   public static Path getRemoteLogSuffixedDir(Path remoteRootLogDir,
   public static Path getRemoteLogSuffixedDir(Path remoteRootLogDir,
@@ -83,8 +83,8 @@ public class LogAggregationUtils {
 
 
   /**
   /**
    * Gets the remote log user dir.
    * 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.
    * @return the remote per user log dir.
    */
    */
   public static Path getRemoteLogUserDir(Path remoteRootLogDir, String user) {
   public static Path getRemoteLogUserDir(Path remoteRootLogDir, String user) {
@@ -93,7 +93,7 @@ public class LogAggregationUtils {
 
 
   /**
   /**
    * Returns the suffix component of the log dir.
    * 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.
    * @return the suffix which will be appended to the user log dir.
    */
    */
   public static String getRemoteNodeLogDirSuffix(Configuration conf) {
   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.
    * 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.
    * @return the node string to be used to construct the file name.
    */
    */
   @VisibleForTesting
   @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
   @VisibleForTesting
   public int dumpAContainerLogsForLogType(ContainerLogsRequest options,
   public int dumpAContainerLogsForLogType(ContainerLogsRequest options,
       boolean outputFailure) throws IOException {
       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 (!foundAnyLogs) {
       if (outputFailure) {
       if (outputFailure) {
         containerLogNotFound(options.getContainerId());
         containerLogNotFound(options.getContainerId());
@@ -142,9 +151,17 @@ public class LogCLIHelpers implements Configurable {
   @Private
   @Private
   public int dumpAContainerLogsForLogTypeWithoutNodeId(
   public int dumpAContainerLogsForLogTypeWithoutNodeId(
       ContainerLogsRequest options) throws IOException {
       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) {
     if (!foundAnyLogs) {
       containerLogNotFound(options.getContainerId());
       containerLogNotFound(options.getContainerId());
       return -1;
       return -1;
@@ -155,9 +172,17 @@ public class LogCLIHelpers implements Configurable {
   @Private
   @Private
   public int dumpAllContainersLogs(ContainerLogsRequest options)
   public int dumpAllContainersLogs(ContainerLogsRequest options)
       throws IOException {
       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) {
     if (!foundAnyLogs) {
       emptyLogDir(LogAggregationUtils.getRemoteAppLogDir(
       emptyLogDir(LogAggregationUtils.getRemoteAppLogDir(
           conf, options.getAppId(), options.getAppOwner())
           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.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Set;
 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.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 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.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 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.logaggregation.LogAggregationUtils;
 import org.apache.hadoop.yarn.webapp.View.ViewContext;
 import org.apache.hadoop.yarn.webapp.View.ViewContext;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block;
 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.LogKey;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
@@ -68,7 +68,7 @@ import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
 @Unstable
 @Unstable
 public abstract class LogAggregationFileController {
 public abstract class LogAggregationFileController {
 
 
-  private static final Log LOG = LogFactory.getLog(
+  private static final Logger LOG = LoggerFactory.getLogger(
       LogAggregationFileController.class);
       LogAggregationFileController.class);
 
 
   /*
   /*
@@ -193,7 +193,7 @@ public abstract class LogAggregationFileController {
 
 
   protected void closePrintStream(OutputStream out) {
   protected void closePrintStream(OutputStream out) {
     if (out != System.out) {
     if (out != System.out) {
-      IOUtils.closeQuietly(out);
+      IOUtils.cleanupWithLogger(LOG, out);
     }
     }
   }
   }
 
 
@@ -201,6 +201,7 @@ public abstract class LogAggregationFileController {
    * Output container log.
    * Output container log.
    * @param logRequest {@link ContainerLogsRequest}
    * @param logRequest {@link ContainerLogsRequest}
    * @param os the output stream
    * @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.
    * @throws IOException if we can not access the log file.
    */
    */
   public abstract boolean readAggregatedLogs(ContainerLogsRequest logRequest,
   public abstract boolean readAggregatedLogs(ContainerLogsRequest logRequest,
@@ -228,9 +229,9 @@ public abstract class LogAggregationFileController {
   /**
   /**
    * Returns the owner of the application.
    * 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)
   public abstract String getApplicationOwner(Path aggregatedLogPath)
       throws IOException;
       throws IOException;
@@ -239,9 +240,9 @@ public abstract class LogAggregationFileController {
    * Returns ACLs for the application. An empty map is returned if no ACLs are
    * Returns ACLs for the application. An empty map is returned if no ACLs are
    * found.
    * found.
    *
    *
-   * @param the aggregatedLog path.
+   * @param aggregatedLogPath the aggregatedLog path.
    * @return a map of the Application ACLs.
    * @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(
   public abstract Map<ApplicationAccessType, String> getApplicationAcls(
       Path aggregatedLogPath) throws IOException;
       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];
     byte[] buf = new byte[65535];
     while (nodeFiles != null && nodeFiles.hasNext()) {
     while (nodeFiles != null && nodeFiles.hasNext()) {
       final FileStatus thisNodeFile = nodeFiles.next();
       final FileStatus thisNodeFile = nodeFiles.next();
-      LOG.error(thisNodeFile.getPath().toString());
       String nodeName = thisNodeFile.getPath().getName();
       String nodeName = thisNodeFile.getPath().getName();
       if (nodeName.equals(appId + ".har")) {
       if (nodeName.equals(appId + ".har")) {
         Path p = new Path("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)
         && webUiType.equals(YarnWebParams.RM_WEB_UI)
         && conf.getBoolean(YarnConfiguration.RM_WEBAPP_UI_ACTIONS_ENABLED,
         && conf.getBoolean(YarnConfiguration.RM_WEBAPP_UI_ACTIONS_ENABLED,
           YarnConfiguration.DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED)
           YarnConfiguration.DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED)
-            && !unsecuredUIForSecuredCluster) {
+            && !unsecuredUIForSecuredCluster
+            && !isAppInFinalState(app)) {
       // Application Kill
       // Application Kill
       html.div()
       html.div()
         .button()
         .button()
@@ -419,4 +420,10 @@ public class AppBlock extends HtmlBlock {
     }
     }
     return ret;
     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;
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
@@ -275,7 +274,7 @@ public class RLESparseResourceAllocation {
   public NavigableMap<Long, Resource> getCumulative() {
   public NavigableMap<Long, Resource> getCumulative() {
     readLock.lock();
     readLock.lock();
     try {
     try {
-      return Collections.unmodifiableNavigableMap(cumulativeCapacity);
+      return cumulativeCapacity;
     } finally {
     } finally {
       readLock.unlock();
       readLock.unlock();
     }
     }
@@ -409,7 +408,7 @@ public class RLESparseResourceAllocation {
       // test for negative value and throws
       // test for negative value and throws
       if (operator == RLEOperator.subtractTestNonNegative
       if (operator == RLEOperator.subtractTestNonNegative
           && (Resources.fitsIn(val, ZERO_RESOURCE)
           && (Resources.fitsIn(val, ZERO_RESOURCE)
-              && !Resources.equals(val, ZERO_RESOURCE))) {
+          && !Resources.equals(val, ZERO_RESOURCE))) {
         throw new PlanningException(
         throw new PlanningException(
             "RLESparseResourceAllocation: merge failed as the "
             "RLESparseResourceAllocation: merge failed as the "
                 + "resulting RLESparseResourceAllocation would be negative");
                 + "resulting RLESparseResourceAllocation would be negative");