You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2015/07/22 19:38:54 UTC

[1/9] hadoop git commit: Revert "HDFS-8344. NameNode doesn't recover lease for files with missing blocks (raviprak)"

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 12bd96395 -> 43bed72d1


Revert "HDFS-8344. NameNode doesn't recover lease for files with missing blocks (raviprak)"

This reverts commit e4f756260f16156179ba4adad974ec92279c2fac.

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt


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

Branch: refs/heads/HDFS-7240
Commit: 5137b388fc9d4d716f780daf6d04292feeb9df96
Parents: 68d1f4b
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Tue Jul 21 11:29:35 2015 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Tue Jul 21 11:29:35 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 -
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  3 -
 .../BlockInfoUnderConstruction.java             | 19 +----
 .../server/blockmanagement/BlockManager.java    | 14 +---
 .../hdfs/server/namenode/FSNamesystem.java      | 10 ---
 .../src/main/resources/hdfs-default.xml         |  9 ---
 .../apache/hadoop/hdfs/TestLeaseRecovery.java   | 78 --------------------
 7 files changed, 4 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5137b388/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 223baaf..a29a090 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1056,9 +1056,6 @@ Release 2.8.0 - UNRELEASED
     HDFS-8778. TestBlockReportRateLimiting#testLeaseExpiration can deadlock.
     (Arpit Agarwal)
 
-    HDFS-8344. NameNode doesn't recover lease for files with missing blocks
-    (raviprak)
-
     HDFS-7582. Enforce maximum number of ACL entries separately per access
     and default. (vinayakumarb)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5137b388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 210d1e5..0e569f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -440,9 +440,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long    DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT = 10 * 1000;
   public static final String  DFS_BLOCK_INVALIDATE_LIMIT_KEY = "dfs.block.invalidate.limit";
   public static final int     DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT = 1000;
-  public static final String  DFS_BLOCK_UC_MAX_RECOVERY_ATTEMPTS = "dfs.block.uc.max.recovery.attempts";
-  public static final int     DFS_BLOCK_UC_MAX_RECOVERY_ATTEMPTS_DEFAULT = 5;
-
   public static final String  DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";
   public static final int     DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED = 500;
   /* Maximum number of blocks to process for initializing replication queues */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5137b388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
index 28f1633..9cd3987 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
@@ -60,11 +61,6 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
    */
   protected Block truncateBlock;
 
-  /** The number of times all replicas will be used to attempt recovery before
-   * giving up and marking the block under construction missing.
-   */
-  private int recoveryAttemptsBeforeMarkingBlockMissing;
-
   /**
    * ReplicaUnderConstruction contains information about replicas while
    * they are under construction.
@@ -178,8 +174,6 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
         "BlockInfoUnderConstruction cannot be in COMPLETE state");
     this.blockUCState = state;
     setExpectedLocations(targets);
-    this.recoveryAttemptsBeforeMarkingBlockMissing =
-      BlockManager.getMaxBlockUCRecoveries();
   }
 
   /** Set expected locations. */
@@ -277,7 +271,7 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
     if (replicas.size() == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK* " +
           "BlockInfoUnderConstruction.initLeaseRecovery: " +
-          "No replicas found.");
+          "No blocks found, lease removed.");
     }
     boolean allLiveReplicasTriedAsPrimary = true;
     for (int i = 0; i < replicas.size(); i++) {
@@ -289,11 +283,6 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
       }
     }
     if (allLiveReplicasTriedAsPrimary) {
-      recoveryAttemptsBeforeMarkingBlockMissing--;
-      NameNode.blockStateChangeLog.info("Tried to recover " + this +" using all"
-          + " replicas. Will try " + recoveryAttemptsBeforeMarkingBlockMissing
-          + " more times");
-
       // Just set all the replicas to be chosen whether they are alive or not.
       for (int i = 0; i < replicas.size(); i++) {
         replicas.get(i).setChosenAsPrimary(false);
@@ -352,10 +341,6 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
     replicas.add(new ReplicaUnderConstruction(block, storage, rState));
   }
 
-  public int getNumRecoveryAttemptsLeft() {
-    return recoveryAttemptsBeforeMarkingBlockMissing;
-  }
-
   /**
    * Convert an under construction block to a complete block.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5137b388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 16dfe45..7dce2a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.IOException;
@@ -273,13 +274,6 @@ public class BlockManager implements BlockStatsMXBean {
   private BlockPlacementPolicy blockplacement;
   private final BlockStoragePolicySuite storagePolicySuite;
 
-  /** The number of times a block under construction's recovery will be
-    * attempted using all known replicas. e.g. if there are 3 replicas, each
-    * node will be tried 5 times (for a total of 15 retries across all nodes)*/
-  private static int maxBlockUCRecoveries =
-    DFSConfigKeys.DFS_BLOCK_UC_MAX_RECOVERY_ATTEMPTS_DEFAULT;
-  public static int getMaxBlockUCRecoveries() { return maxBlockUCRecoveries; }
-
   /** Check whether name system is running before terminating */
   private boolean checkNSRunning = true;
 
@@ -288,9 +282,6 @@ public class BlockManager implements BlockStatsMXBean {
     this.namesystem = namesystem;
     datanodeManager = new DatanodeManager(this, namesystem, conf);
     heartbeatManager = datanodeManager.getHeartbeatManager();
-    maxBlockUCRecoveries = conf.getInt(
-      DFSConfigKeys.DFS_BLOCK_UC_MAX_RECOVERY_ATTEMPTS,
-      DFSConfigKeys.DFS_BLOCK_UC_MAX_RECOVERY_ATTEMPTS_DEFAULT);
 
     startupDelayBlockDeletionInMs = conf.getLong(
         DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_KEY,
@@ -740,8 +731,7 @@ public class BlockManager implements BlockStatsMXBean {
   /**
    * Force the given block in the given file to be marked as complete,
    * regardless of whether enough replicas are present. This is necessary
-   * when tailing edit logs as a Standby or when recovering a lease on a file
-   * with missing blocks.
+   * when tailing edit logs as a Standby.
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
       final BlockInfoUnderConstruction block) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5137b388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index becd05a..7c6d6a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3287,16 +3287,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             + "Removed empty last block and closed file.");
         return true;
       }
-
-      //If the block's recovery has been attempted enough times, mark the block
-      //complete anyway and recover the lease
-      if(uc.getNumRecoveryAttemptsLeft() == 0) {
-        blockManager.forceCompleteBlock(pendingFile, uc);
-        finalizeINodeFileUnderConstruction(src, pendingFile,
-            iip.getLatestSnapshotId());
-        return true;
-      }
-
       // start recovery of the last block for this file
       long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc));
       lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5137b388/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index dee9111..8cb7d5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -425,15 +425,6 @@
 </property>
 
 <property>
-  <name>dfs.block.uc.max.recovery.attempts</name>
-  <value>5</value>
-  <description>The number of times a block under construction's recovery will be
-  attempted using all known replicas. e.g. if there are 3 replicas, each node
-  will be tried 5 times (for a total of 15 retries across all nodes).
-  </description>
-</property>
-
-<property>
   <name>dfs.datanode.data.dir</name>
   <value>file://${hadoop.tmp.dir}/dfs/data</value>
   <description>Determines where on the local filesystem an DFS data node

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5137b388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
index c9448ac..c9f3842 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
@@ -27,7 +27,6 @@ import java.util.EnumSet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -259,81 +258,4 @@ public class TestLeaseRecovery {
       }
     }
   }
-
-  /**
-   * Test that when a client was writing to a file and died, and before the
-   * lease can be recovered, all the datanodes to which the file was written
-   * also die, after some time (5 * lease recovery times) the file is indeed
-   * closed and lease recovered.
-   * We also check that if the datanode came back after some time, the data
-   * originally written is not truncated
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  @Test
-  public void testLeaseRecoveryWithMissingBlocks()
-    throws IOException, InterruptedException {
-    Configuration conf = new HdfsConfiguration();
-
-    //Start a cluster with 3 datanodes
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
-    cluster.setLeasePeriod(LEASE_PERIOD, LEASE_PERIOD);
-    cluster.waitActive();
-
-    //create a file (with replication 1)
-    Path file = new Path("/testRecoveryFile");
-    DistributedFileSystem dfs = cluster.getFileSystem();
-    FSDataOutputStream out = dfs.create(file, (short) 1);
-
-    //This keeps count of the number of bytes written (AND is also the data we
-    //are writing)
-    long writtenBytes = 0;
-    while (writtenBytes < 2 * 1024 * 1024) {
-      out.writeLong(writtenBytes);
-      writtenBytes += 8;
-    }
-    System.out.println("Written " + writtenBytes + " bytes");
-    out.hsync();
-    System.out.println("hsynced the data");
-
-    //Kill the datanode to which the file was written.
-    DatanodeInfo dn =
-      ((DFSOutputStream) out.getWrappedStream()).getPipeline()[0];
-    DataNodeProperties dnStopped = cluster.stopDataNode(dn.getName());
-
-    //Wait at most 20 seconds for the lease to be recovered
-    LeaseManager lm = NameNodeAdapter.getLeaseManager(cluster.getNamesystem());
-    int i = 40;
-    while(i-- > 0 && lm.countLease() != 0) {
-      System.out.println("Still got " + lm.countLease() + " lease(s)");
-      Thread.sleep(500);
-    }
-    assertTrue("The lease was not recovered", lm.countLease() == 0);
-    System.out.println("Got " + lm.countLease() + " leases");
-
-    //Make sure we can't read any data because the datanode is dead
-    FSDataInputStream in = dfs.open(file);
-    try {
-      in.readLong();
-      assertTrue("Shouldn't have reached here", false);
-    } catch(BlockMissingException bme) {
-      System.out.println("Correctly got BlockMissingException because datanode"
-        + " is still dead");
-    }
-
-    //Bring the dead datanode back.
-    cluster.restartDataNode(dnStopped);
-    System.out.println("Restart datanode");
-
-    //Make sure we can read all the data back (since we hsync'ed).
-    in = dfs.open(file);
-    int readBytes = 0;
-    while(in.available() != 0) {
-      assertEquals("Didn't read the data we wrote", in.readLong(), readBytes);
-      readBytes += 8;
-    }
-    assertEquals("Didn't get all the data", readBytes, writtenBytes);
-    System.out.println("Read back all the " + readBytes + " bytes");
-  }
-
 }


[9/9] hadoop git commit: HDFS-8753. Ozone: Unify StorageContainerConfiguration with ozone-default.xml & ozone-site.xml. Contributed by kanaka kumar avvaru

Posted by ae...@apache.org.
HDFS-8753. Ozone: Unify StorageContainerConfiguration with ozone-default.xml & ozone-site.xml. Contributed by kanaka kumar avvaru


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

Branch: refs/heads/HDFS-7240
Commit: 43bed72d1355d44f6e910ba7ab9858dfde6edc4f
Parents: 01094cbf
Author: Anu Engineer <an...@gmail.com>
Authored: Wed Jul 22 10:31:43 2015 -0700
Committer: Anu Engineer <an...@gmail.com>
Committed: Wed Jul 22 10:31:43 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |  2 ++
 .../apache/hadoop/ozone/OzoneConfiguration.java | 36 ++++++++++++++++++++
 .../ozone/StorageContainerConfiguration.java    | 35 -------------------
 .../web/localstorage/OzoneMetadataManager.java  |  4 +--
 .../StorageContainerConfiguration.java          | 32 -----------------
 .../StorageContainerManager.java                |  5 +--
 .../src/main/resources/ozone-default.xml        | 27 +++++++++++++++
 .../src/test/resources/ozone-site.xml           | 24 +++++++++++++
 8 files changed, 94 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43bed72d/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index db38851..5ec8a24 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -307,6 +307,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <configuration>
               <tasks>
                 <copy file="src/main/resources/hdfs-default.xml" todir="src/site/resources"/>
+                <copy file="src/main/resources/ozone-default.xml" todir="src/site/resources"/>
                 <copy file="src/main/xsl/configuration.xsl" todir="src/site/resources"/>
               </tasks>
             </configuration>
@@ -401,6 +402,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
               <includes>
                 <include>configuration.xsl</include>
                 <include>hdfs-default.xml</include>
+                <include>ozone-default.xml</include>
               </includes>
               <followSymlinks>false</followSymlinks>
             </fileset>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43bed72d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
new file mode 100644
index 0000000..70efa49
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Configuration for ozone.
+ */
+@InterfaceAudience.Private
+public class OzoneConfiguration extends Configuration {
+  static {
+    // adds the default resources
+    Configuration.addDefaultResource("hdfs-default.xml");
+    Configuration.addDefaultResource("hdfs-site.xml");
+    Configuration.addDefaultResource("ozone-default.xml");
+    Configuration.addDefaultResource("ozone-site.xml");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43bed72d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/StorageContainerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/StorageContainerConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/StorageContainerConfiguration.java
deleted file mode 100644
index a83fa33..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/StorageContainerConfiguration.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Configuration for ozone.
- */
-@InterfaceAudience.Private
-public class StorageContainerConfiguration extends Configuration {
-  static {
-    // adds the default resources
-    Configuration.addDefaultResource("hdfs-default.xml");
-    Configuration.addDefaultResource("hdfs-site.xml");
-    Configuration.addDefaultResource("ozone-site.xml");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43bed72d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
index 22b1ed2..b13ffd4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.ozone.web.localstorage;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.StorageContainerConfiguration;
+import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
@@ -125,7 +125,7 @@ public final class OzoneMetadataManager {
   private OzoneMetadataManager() {
 
     lock = new ReentrantReadWriteLock();
-    StorageContainerConfiguration conf = new StorageContainerConfiguration();
+    OzoneConfiguration conf = new OzoneConfiguration();
 
     String storageRoot =
         conf.getTrimmed(OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43bed72d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerConfiguration.java
deleted file mode 100644
index dbe28d2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerConfiguration.java
+++ /dev/null
@@ -1,32 +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.storagecontainer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-
-/**
- * Loads storage container specific configurations.
- */
-public class StorageContainerConfiguration extends HdfsConfiguration {
-  static {
-    // adds the default resources
-    Configuration.addDefaultResource("storage-container-site.xml");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43bed72d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerManager.java
index ab48542..bbbf5f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerManager.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.storagecontainer.protocol.ContainerLocationProtocol;
 import org.apache.hadoop.util.LightWeightGSet;
 import org.slf4j.Logger;
@@ -69,7 +70,7 @@ public class StorageContainerManager
   private final RPC.Server clientRpcServer;
   private final InetSocketAddress clientRpcAddress;
 
-  public StorageContainerManager(StorageContainerConfiguration conf)
+  public StorageContainerManager(OzoneConfiguration conf)
       throws IOException {
     BlocksMap containerMap = new BlocksMap(
         LightWeightGSet.computeCapacity(2.0, "BlocksMap"),
@@ -309,7 +310,7 @@ public class StorageContainerManager
   }
 
   public static void main(String[] argv) throws IOException {
-    StorageContainerConfiguration conf = new StorageContainerConfiguration();
+    OzoneConfiguration conf = new OzoneConfiguration();
     StorageContainerManager scm = new StorageContainerManager(conf);
     scm.start();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43bed72d/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/ozone-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/ozone-default.xml
new file mode 100644
index 0000000..209a280
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/ozone-default.xml
@@ -0,0 +1,27 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<!-- Do not modify this file directly.  Instead, copy entries that you -->
+<!-- wish to modify from this file into ozone-site.xml and change them -->
+<!-- there.  If ozone-site.xml does not already exist, create it.      -->
+
+<configuration>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43bed72d/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/ozone-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/ozone-site.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/ozone-site.xml
new file mode 100644
index 0000000..77dd7ef
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/ozone-site.xml
@@ -0,0 +1,24 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+</configuration>


[4/9] hadoop git commit: YARN-3878. AsyncDispatcher can hang while stopping if it is configured for draining events on stop. Contributed by Varun Saxena

Posted by ae...@apache.org.
YARN-3878. AsyncDispatcher can hang while stopping if it is configured for draining events on stop. Contributed by Varun Saxena


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

Branch: refs/heads/HDFS-7240
Commit: 393fe71771e3ac6bc0efe59d9aaf19d3576411b3
Parents: a26cc66
Author: Jian He <ji...@apache.org>
Authored: Tue Jul 21 15:05:41 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue Jul 21 15:05:41 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../hadoop/yarn/event/AsyncDispatcher.java      |  8 +++
 .../hadoop/yarn/event/DrainDispatcher.java      | 11 +++-
 .../hadoop/yarn/event/TestAsyncDispatcher.java  | 62 ++++++++++++++++++++
 4 files changed, 83 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/393fe717/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 79e9ae2..5100cdf 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -682,6 +682,9 @@ Release 2.7.2 - UNRELEASED
     YARN-3535. Scheduler must re-request container resources when RMContainer transitions
     from ALLOCATED to KILLED (rohithsharma and peng.zhang via asuresh)
 
+    YARN-3878. AsyncDispatcher can hang while stopping if it is configured for
+    draining events on stop. (Varun Saxena via jianhe)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/393fe717/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
index c54b9c7..48312a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
@@ -246,6 +246,9 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
         if (!stopped) {
           LOG.warn("AsyncDispatcher thread interrupted", e);
         }
+        // Need to reset drained flag to true if event queue is empty,
+        // otherwise dispatcher will hang on stop.
+        drained = eventQueue.isEmpty();
         throw new YarnRuntimeException(e);
       }
     };
@@ -287,6 +290,11 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
   }
 
   @VisibleForTesting
+  protected boolean isEventThreadWaiting() {
+    return eventHandlingThread.getState() == Thread.State.WAITING;
+  }
+
+  @VisibleForTesting
   protected boolean isDrained() {
     return this.drained;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/393fe717/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
index da5ae44..e4a5a82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
@@ -27,11 +27,20 @@ public class DrainDispatcher extends AsyncDispatcher {
     this(new LinkedBlockingQueue<Event>());
   }
 
-  private DrainDispatcher(BlockingQueue<Event> eventQueue) {
+  public DrainDispatcher(BlockingQueue<Event> eventQueue) {
     super(eventQueue);
   }
 
   /**
+   *  Wait till event thread enters WAITING state (i.e. waiting for new events).
+   */
+  public void waitForEventThreadToWait() {
+    while (!isEventThreadWaiting()) {
+      Thread.yield();
+    }
+  }
+
+  /**
    * Busy loop waiting for all queued events to drain.
    */
   public void await() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/393fe717/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
new file mode 100644
index 0000000..b5fd923
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.event;
+
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestAsyncDispatcher {
+
+  /* This test checks whether dispatcher hangs on close if following two things
+   * happen :
+   * 1. A thread which was putting event to event queue is interrupted.
+   * 2. Event queue is empty on close.
+   */
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  @Test(timeout=10000)
+  public void testDispatcherOnCloseIfQueueEmpty() throws Exception {
+    BlockingQueue<Event> eventQueue = spy(new LinkedBlockingQueue<Event>());
+    Event event = mock(Event.class);
+    doThrow(new InterruptedException()).when(eventQueue).put(event);
+    DrainDispatcher disp = new DrainDispatcher(eventQueue);
+    disp.init(new Configuration());
+    disp.setDrainEventsOnStop();
+    disp.start();
+    // Wait for event handler thread to start and begin waiting for events.
+    disp.waitForEventThreadToWait();
+    try {
+      disp.getEventHandler().handle(event);
+    } catch (YarnRuntimeException e) {
+    }
+    // Queue should be empty and dispatcher should not hang on close
+    Assert.assertTrue("Event Queue should have been empty",
+        eventQueue.isEmpty());
+    disp.close();
+  }
+}
+


[5/9] hadoop git commit: HDFS-8495. Consolidate append() related implementation into a single class. Contributed by Rakesh R.

Posted by ae...@apache.org.
HDFS-8495. Consolidate append() related implementation into a single class. Contributed by Rakesh R.


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

Branch: refs/heads/HDFS-7240
Commit: 31f117138a00794de4951ee8433e304d72b04094
Parents: 393fe71
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Jul 21 17:25:23 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Jul 21 17:25:23 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSDirAppendOp.java     | 261 +++++++++++++++++++
 .../server/namenode/FSDirStatAndListingOp.java  |   2 +-
 .../hdfs/server/namenode/FSDirTruncateOp.java   |  16 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |   6 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 241 ++---------------
 7 files changed, 304 insertions(+), 229 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/31f11713/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 8122045..50803de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -737,6 +737,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8721. Add a metric for number of encryption zones.
     (Rakesh R via cnauroth)
 
+    HDFS-8495. Consolidate append() related implementation into a single class.
+    (Rakesh R via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31f11713/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
new file mode 100644
index 0000000..abb2dc8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
@@ -0,0 +1,261 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.server.namenode;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion.Feature;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Helper class to perform append operation.
+ */
+final class FSDirAppendOp {
+
+  /**
+   * Private constructor for preventing FSDirAppendOp object creation.
+   * Static-only class.
+   */
+  private FSDirAppendOp() {}
+
+  /**
+   * Append to an existing file.
+   * <p>
+   *
+   * The method returns the last block of the file if this is a partial block,
+   * which can still be used for writing more data. The client uses the
+   * returned block locations to form the data pipeline for this block.<br>
+   * The {@link LocatedBlock} will be null if the last block is full.
+   * The client then allocates a new block with the next call using
+   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#addBlock}.
+   * <p>
+   *
+   * For description of parameters and exceptions thrown see
+   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#append}
+   *
+   * @param fsn namespace
+   * @param srcArg path name
+   * @param pc permission checker to check fs permission
+   * @param holder client name
+   * @param clientMachine client machine info
+   * @param newBlock if the data is appended to a new block
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
+   * @return the last block with status
+   */
+  static LastBlockWithStatus appendFile(final FSNamesystem fsn,
+      final String srcArg, final FSPermissionChecker pc, final String holder,
+      final String clientMachine, final boolean newBlock,
+      final boolean logRetryCache) throws IOException {
+    assert fsn.hasWriteLock();
+
+    final byte[][] pathComponents = FSDirectory
+        .getPathComponentsForReservedPath(srcArg);
+    final LocatedBlock lb;
+    final FSDirectory fsd = fsn.getFSDirectory();
+    final String src;
+    fsd.writeLock();
+    try {
+      src = fsd.resolvePath(pc, srcArg, pathComponents);
+      final INodesInPath iip = fsd.getINodesInPath4Write(src);
+      // Verify that the destination does not exist as a directory already
+      final INode inode = iip.getLastINode();
+      final String path = iip.getPath();
+      if (inode != null && inode.isDirectory()) {
+        throw new FileAlreadyExistsException("Cannot append to directory "
+            + path + "; already exists as a directory.");
+      }
+      if (fsd.isPermissionEnabled()) {
+        fsd.checkPathAccess(pc, iip, FsAction.WRITE);
+      }
+
+      if (inode == null) {
+        throw new FileNotFoundException(
+            "Failed to append to non-existent file " + path + " for client "
+                + clientMachine);
+      }
+      final INodeFile file = INodeFile.valueOf(inode, path, true);
+      BlockManager blockManager = fsd.getBlockManager();
+      final BlockStoragePolicy lpPolicy = blockManager
+          .getStoragePolicy("LAZY_PERSIST");
+      if (lpPolicy != null && lpPolicy.getId() == file.getStoragePolicyID()) {
+        throw new UnsupportedOperationException(
+            "Cannot append to lazy persist file " + path);
+      }
+      // Opening an existing file for append - may need to recover lease.
+      fsn.recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE, iip, path, holder,
+          clientMachine, false);
+
+      final BlockInfo lastBlock = file.getLastBlock();
+      // Check that the block has at least minimum replication.
+      if (lastBlock != null && lastBlock.isComplete()
+          && !blockManager.isSufficientlyReplicated(lastBlock)) {
+        throw new IOException("append: lastBlock=" + lastBlock + " of src="
+            + path + " is not sufficiently replicated yet.");
+      }
+      lb = prepareFileForAppend(fsn, iip, holder, clientMachine, newBlock,
+          true, logRetryCache);
+    } catch (IOException ie) {
+      NameNode.stateChangeLog
+          .warn("DIR* NameSystem.append: " + ie.getMessage());
+      throw ie;
+    } finally {
+      fsd.writeUnlock();
+    }
+
+    HdfsFileStatus stat = FSDirStatAndListingOp.getFileInfo(fsd, src, false,
+        FSDirectory.isReservedRawName(srcArg), true);
+    if (lb != null) {
+      NameNode.stateChangeLog.debug(
+          "DIR* NameSystem.appendFile: file {} for {} at {} block {} block"
+              + " size {}", srcArg, holder, clientMachine, lb.getBlock(), lb
+              .getBlock().getNumBytes());
+    }
+    return new LastBlockWithStatus(lb, stat);
+  }
+
+  /**
+   * Convert current node to under construction.
+   * Recreate in-memory lease record.
+   *
+   * @param fsn namespace
+   * @param iip inodes in the path containing the file
+   * @param leaseHolder identifier of the lease holder on this file
+   * @param clientMachine identifier of the client machine
+   * @param newBlock if the data is appended to a new block
+   * @param writeToEditLog whether to persist this change to the edit log
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
+   * @return the last block locations if the block is partial or null otherwise
+   * @throws IOException
+   */
+  static LocatedBlock prepareFileForAppend(final FSNamesystem fsn,
+      final INodesInPath iip, final String leaseHolder,
+      final String clientMachine, final boolean newBlock,
+      final boolean writeToEditLog, final boolean logRetryCache)
+      throws IOException {
+    assert fsn.hasWriteLock();
+
+    final INodeFile file = iip.getLastINode().asFile();
+    final QuotaCounts delta = verifyQuotaForUCBlock(fsn, file, iip);
+
+    file.recordModification(iip.getLatestSnapshotId());
+    file.toUnderConstruction(leaseHolder, clientMachine);
+
+    fsn.getLeaseManager().addLease(
+        file.getFileUnderConstructionFeature().getClientName(), file.getId());
+
+    LocatedBlock ret = null;
+    if (!newBlock) {
+      FSDirectory fsd = fsn.getFSDirectory();
+      ret = fsd.getBlockManager().convertLastBlockToUnderConstruction(file, 0);
+      if (ret != null && delta != null) {
+        Preconditions.checkState(delta.getStorageSpace() >= 0, "appending to"
+            + " a block with size larger than the preferred block size");
+        fsd.writeLock();
+        try {
+          fsd.updateCountNoQuotaCheck(iip, iip.length() - 1, delta);
+        } finally {
+          fsd.writeUnlock();
+        }
+      }
+    } else {
+      BlockInfo lastBlock = file.getLastBlock();
+      if (lastBlock != null) {
+        ExtendedBlock blk = new ExtendedBlock(fsn.getBlockPoolId(), lastBlock);
+        ret = new LocatedBlock(blk, new DatanodeInfo[0]);
+      }
+    }
+
+    if (writeToEditLog) {
+      final String path = iip.getPath();
+      if (NameNodeLayoutVersion.supports(Feature.APPEND_NEW_BLOCK,
+          fsn.getEffectiveLayoutVersion())) {
+        fsn.getEditLog().logAppendFile(path, file, newBlock, logRetryCache);
+      } else {
+        fsn.getEditLog().logOpenFile(path, file, false, logRetryCache);
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Verify quota when using the preferred block size for UC block. This is
+   * usually used by append and truncate.
+   *
+   * @throws QuotaExceededException when violating the storage quota
+   * @return expected quota usage update. null means no change or no need to
+   *         update quota usage later
+   */
+  private static QuotaCounts verifyQuotaForUCBlock(FSNamesystem fsn,
+      INodeFile file, INodesInPath iip) throws QuotaExceededException {
+    FSDirectory fsd = fsn.getFSDirectory();
+    if (!fsn.isImageLoaded() || fsd.shouldSkipQuotaChecks()) {
+      // Do not check quota if editlog is still being processed
+      return null;
+    }
+    if (file.getLastBlock() != null) {
+      final QuotaCounts delta = computeQuotaDeltaForUCBlock(fsn, file);
+      fsd.readLock();
+      try {
+        FSDirectory.verifyQuota(iip, iip.length() - 1, delta, null);
+        return delta;
+      } finally {
+        fsd.readUnlock();
+      }
+    }
+    return null;
+  }
+
+  /** Compute quota change for converting a complete block to a UC block. */
+  private static QuotaCounts computeQuotaDeltaForUCBlock(FSNamesystem fsn,
+      INodeFile file) {
+    final QuotaCounts delta = new QuotaCounts.Builder().build();
+    final BlockInfo lastBlock = file.getLastBlock();
+    if (lastBlock != null) {
+      final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
+      final short repl = file.getPreferredBlockReplication();
+      delta.addStorageSpace(diff * repl);
+      final BlockStoragePolicy policy = fsn.getFSDirectory()
+          .getBlockStoragePolicySuite().getPolicy(file.getStoragePolicyID());
+      List<StorageType> types = policy.chooseStorageTypes(repl);
+      for (StorageType t : types) {
+        if (t.supportTypeQuota()) {
+          delta.addTypeSpace(t, diff);
+        }
+      }
+    }
+    return delta;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31f11713/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 201dabc..14f4d66 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -508,7 +508,7 @@ class FSDirStatAndListingOp {
       final long fileSize = !inSnapshot && isUc ?
           fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
 
-      loc = fsd.getFSNamesystem().getBlockManager().createLocatedBlocks(
+      loc = fsd.getBlockManager().createLocatedBlocks(
           fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
           inSnapshot, feInfo);
       if (loc == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31f11713/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
index 9fc9def..e24bb2f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -79,11 +80,11 @@ final class FSDirTruncateOp {
     try {
       src = fsd.resolvePath(pc, srcArg, pathComponents);
       iip = fsd.getINodesInPath4Write(src, true);
-      if (fsn.isPermissionEnabled()) {
+      if (fsd.isPermissionEnabled()) {
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
       }
       INodeFile file = INodeFile.valueOf(iip.getLastINode(), src);
-      final BlockStoragePolicy lpPolicy = fsn.getBlockManager()
+      final BlockStoragePolicy lpPolicy = fsd.getBlockManager()
           .getStoragePolicy("LAZY_PERSIST");
 
       if (lpPolicy != null && lpPolicy.getId() == file.getStoragePolicyID()) {
@@ -178,7 +179,7 @@ final class FSDirTruncateOp {
           "Should be the same block.";
       if (oldBlock.getBlockId() != tBlk.getBlockId()
           && !file.isBlockInLatestSnapshot(oldBlock)) {
-        fsn.getBlockManager().removeBlockFromMap(oldBlock);
+        fsd.getBlockManager().removeBlockFromMap(oldBlock);
       }
     }
     assert onBlockBoundary == (truncateBlock == null) :
@@ -223,6 +224,7 @@ final class FSDirTruncateOp {
     }
 
     BlockInfoUnderConstruction truncatedBlockUC;
+    BlockManager blockManager = fsn.getFSDirectory().getBlockManager();
     if (shouldCopyOnTruncate) {
       // Add new truncateBlock into blocksMap and
       // use oldBlock as a source for copy-on-truncate recovery
@@ -230,9 +232,8 @@ final class FSDirTruncateOp {
           file.getPreferredBlockReplication());
       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
       truncatedBlockUC.setTruncateBlock(oldBlock);
-      file.setLastBlock(truncatedBlockUC,
-          fsn.getBlockManager().getStorages(oldBlock));
-      fsn.getBlockManager().addBlockCollection(truncatedBlockUC, file);
+      file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock));
+      blockManager.addBlockCollection(truncatedBlockUC, file);
 
       NameNode.stateChangeLog.debug(
           "BLOCK* prepareFileForTruncate: Scheduling copy-on-truncate to new"
@@ -241,8 +242,7 @@ final class FSDirTruncateOp {
           truncatedBlockUC.getTruncateBlock());
     } else {
       // Use new generation stamp for in-place truncate recovery
-      fsn.getBlockManager().convertLastBlockToUnderConstruction(file,
-          lastBlockDelta);
+      blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
       oldBlock = file.getLastBlock();
       assert !oldBlock.isComplete() : "oldBlock should be under construction";
       truncatedBlockUC = (BlockInfoUnderConstruction) oldBlock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31f11713/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 4830d5d..008a945 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -206,8 +206,8 @@ class FSDirWriteFileOp {
       DatanodeStorageInfo[] locs, long offset) throws IOException {
     LocatedBlock lBlk = BlockManager.newLocatedBlock(fsn.getExtendedBlock(blk),
                                                      locs, offset, false);
-    fsn.getBlockManager().setBlockToken(lBlk,
-                                        BlockTokenIdentifier.AccessMode.WRITE);
+    fsn.getFSDirectory().getBlockManager()
+        .setBlockToken(lBlk, BlockTokenIdentifier.AccessMode.WRITE);
     return lBlk;
   }
 
@@ -426,7 +426,7 @@ class FSDirWriteFileOp {
       fsd.setFileEncryptionInfo(src, feInfo);
       newNode = fsd.getInode(newNode.getId()).asFile();
     }
-    setNewINodeStoragePolicy(fsn.getBlockManager(), newNode, iip,
+    setNewINodeStoragePolicy(fsd.getBlockManager(), newNode, iip,
                              isLazyPersist);
     fsd.getEditLog().logOpenFile(src, newNode, overwrite, logRetryEntry);
     if (NameNode.stateChangeLog.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31f11713/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 63ef985..357684a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -392,7 +392,7 @@ public class FSEditLogLoader {
             FSNamesystem.LOG.debug("Reopening an already-closed file " +
                 "for append");
           }
-          LocatedBlock lb = fsNamesys.prepareFileForAppend(path, iip,
+          LocatedBlock lb = FSDirAppendOp.prepareFileForAppend(fsNamesys, iip,
               addCloseOp.clientName, addCloseOp.clientMachine, false, false,
               false);
           // add the op into retry cache if necessary
@@ -466,7 +466,7 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath4Write(path);
       INodeFile file = INodeFile.valueOf(iip.getLastINode(), path);
       if (!file.isUnderConstruction()) {
-        LocatedBlock lb = fsNamesys.prepareFileForAppend(path, iip,
+        LocatedBlock lb = FSDirAppendOp.prepareFileForAppend(fsNamesys, iip,
             appendOp.clientName, appendOp.clientMachine, appendOp.newBlock,
             false, false);
         // add the op into retry cache if necessary

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31f11713/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index fd37fbe..0b44431 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -142,7 +142,6 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -185,7 +184,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -250,7 +248,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
-import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RetriableException;
@@ -2174,175 +2171,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
-   * Append to an existing file for append.
-   * <p>
-   * 
-   * The method returns the last block of the file if this is a partial block,
-   * which can still be used for writing more data. The client uses the returned
-   * block locations to form the data pipeline for this block.<br>
-   * The method returns null if the last block is full. The client then
-   * allocates a new block with the next call using
-   * {@link ClientProtocol#addBlock}.
-   * <p>
-   * 
-   * For description of parameters and exceptions thrown see
-   * {@link ClientProtocol#append(String, String, EnumSetWritable)}
-   *
-   * @return the last block locations if the block is partial or null otherwise
-   */
-  private LocatedBlock appendFileInternal(FSPermissionChecker pc,
-      INodesInPath iip, String holder, String clientMachine, boolean newBlock,
-      boolean logRetryCache) throws IOException {
-    assert hasWriteLock();
-    // Verify that the destination does not exist as a directory already.
-    final INode inode = iip.getLastINode();
-    final String src = iip.getPath();
-    if (inode != null && inode.isDirectory()) {
-      throw new FileAlreadyExistsException("Cannot append to directory " + src
-          + "; already exists as a directory.");
-    }
-    if (isPermissionEnabled) {
-      dir.checkPathAccess(pc, iip, FsAction.WRITE);
-    }
-
-    try {
-      if (inode == null) {
-        throw new FileNotFoundException("failed to append to non-existent file "
-          + src + " for client " + clientMachine);
-      }
-      INodeFile myFile = INodeFile.valueOf(inode, src, true);
-      final BlockStoragePolicy lpPolicy =
-          blockManager.getStoragePolicy("LAZY_PERSIST");
-      if (lpPolicy != null &&
-          lpPolicy.getId() == myFile.getStoragePolicyID()) {
-        throw new UnsupportedOperationException(
-            "Cannot append to lazy persist file " + src);
-      }
-      // Opening an existing file for append - may need to recover lease.
-      recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE, iip, src, holder,
-                           clientMachine, false);
-      
-      final BlockInfo lastBlock = myFile.getLastBlock();
-      // Check that the block has at least minimum replication.
-      if(lastBlock != null && lastBlock.isComplete() &&
-          !getBlockManager().isSufficientlyReplicated(lastBlock)) {
-        throw new IOException("append: lastBlock=" + lastBlock +
-            " of src=" + src + " is not sufficiently replicated yet.");
-      }
-      return prepareFileForAppend(src, iip, holder, clientMachine, newBlock,
-          true, logRetryCache);
-    } catch (IOException ie) {
-      NameNode.stateChangeLog.warn("DIR* NameSystem.append: " +ie.getMessage());
-      throw ie;
-    }
-  }
-  
-  /**
-   * Convert current node to under construction.
-   * Recreate in-memory lease record.
-   * 
-   * @param src path to the file
-   * @param leaseHolder identifier of the lease holder on this file
-   * @param clientMachine identifier of the client machine
-   * @param newBlock if the data is appended to a new block
-   * @param writeToEditLog whether to persist this change to the edit log
-   * @param logRetryCache whether to record RPC ids in editlog for retry cache
-   *                      rebuilding
-   * @return the last block locations if the block is partial or null otherwise
-   * @throws UnresolvedLinkException
-   * @throws IOException
-   */
-  LocatedBlock prepareFileForAppend(String src, INodesInPath iip,
-      String leaseHolder, String clientMachine, boolean newBlock,
-      boolean writeToEditLog, boolean logRetryCache) throws IOException {
-    final INodeFile file = iip.getLastINode().asFile();
-    final QuotaCounts delta = verifyQuotaForUCBlock(file, iip);
-
-    file.recordModification(iip.getLatestSnapshotId());
-    file.toUnderConstruction(leaseHolder, clientMachine);
-
-    leaseManager.addLease(
-        file.getFileUnderConstructionFeature().getClientName(), file.getId());
-
-    LocatedBlock ret = null;
-    if (!newBlock) {
-      ret = blockManager.convertLastBlockToUnderConstruction(file, 0);
-      if (ret != null && delta != null) {
-        Preconditions.checkState(delta.getStorageSpace() >= 0,
-            "appending to a block with size larger than the preferred block size");
-        dir.writeLock();
-        try {
-          dir.updateCountNoQuotaCheck(iip, iip.length() - 1, delta);
-        } finally {
-          dir.writeUnlock();
-        }
-      }
-    } else {
-      BlockInfo lastBlock = file.getLastBlock();
-      if (lastBlock != null) {
-        ExtendedBlock blk = new ExtendedBlock(this.getBlockPoolId(), lastBlock);
-        ret = new LocatedBlock(blk, new DatanodeInfo[0]);
-      }
-    }
-
-    if (writeToEditLog) {
-      if (NameNodeLayoutVersion.supports(Feature.APPEND_NEW_BLOCK,
-          getEffectiveLayoutVersion())) {
-        getEditLog().logAppendFile(src, file, newBlock, logRetryCache);
-      } else {
-        getEditLog().logOpenFile(src, file, false, logRetryCache);
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * Verify quota when using the preferred block size for UC block. This is
-   * usually used by append and truncate
-   * @throws QuotaExceededException when violating the storage quota
-   * @return expected quota usage update. null means no change or no need to
-   *         update quota usage later
-   */
-  private QuotaCounts verifyQuotaForUCBlock(INodeFile file, INodesInPath iip)
-      throws QuotaExceededException {
-    if (!isImageLoaded() || dir.shouldSkipQuotaChecks()) {
-      // Do not check quota if editlog is still being processed
-      return null;
-    }
-    if (file.getLastBlock() != null) {
-      final QuotaCounts delta = computeQuotaDeltaForUCBlock(file);
-      dir.readLock();
-      try {
-        FSDirectory.verifyQuota(iip, iip.length() - 1, delta, null);
-        return delta;
-      } finally {
-        dir.readUnlock();
-      }
-    }
-    return null;
-  }
-
-  /** Compute quota change for converting a complete block to a UC block */
-  private QuotaCounts computeQuotaDeltaForUCBlock(INodeFile file) {
-    final QuotaCounts delta = new QuotaCounts.Builder().build();
-    final BlockInfo lastBlock = file.getLastBlock();
-    if (lastBlock != null) {
-      final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
-      final short repl = file.getPreferredBlockReplication();
-      delta.addStorageSpace(diff * repl);
-      final BlockStoragePolicy policy = dir.getBlockStoragePolicySuite()
-          .getPolicy(file.getStoragePolicyID());
-      List<StorageType> types = policy.chooseStorageTypes(repl);
-      for (StorageType t : types) {
-        if (t.supportTypeQuota()) {
-          delta.addTypeSpace(t, diff);
-        }
-      }
-    }
-    return delta;
-  }
-
-  /**
    * Recover lease;
    * Immediately revoke the lease of the current lease holder and start lease
    * recovery so that the file can be forced to be closed.
@@ -2487,62 +2315,45 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Append to an existing file in the namespace.
    */
-  LastBlockWithStatus appendFile(String src, String holder,
+  LastBlockWithStatus appendFile(String srcArg, String holder,
       String clientMachine, EnumSet<CreateFlag> flag, boolean logRetryCache)
       throws IOException {
     boolean newBlock = flag.contains(CreateFlag.NEW_BLOCK);
     if (newBlock) {
       requireEffectiveLayoutVersionForFeature(Feature.APPEND_NEW_BLOCK);
     }
-    try {
-      return appendFileInt(src, holder, clientMachine, newBlock, logRetryCache);
-    } catch (AccessControlException e) {
-      logAuditEvent(false, "append", src);
-      throw e;
-    }
-  }
 
-  private LastBlockWithStatus appendFileInt(final String srcArg, String holder,
-      String clientMachine, boolean newBlock, boolean logRetryCache)
-      throws IOException {
-    String src = srcArg;
     NameNode.stateChangeLog.debug(
         "DIR* NameSystem.appendFile: src={}, holder={}, clientMachine={}",
-        src, holder, clientMachine);
-    boolean skipSync = false;
-    LocatedBlock lb = null;
-    HdfsFileStatus stat = null;
-    FSPermissionChecker pc = getPermissionChecker();
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
-    writeLock();
+        srcArg, holder, clientMachine);
     try {
+      boolean skipSync = false;
+      LastBlockWithStatus lbs = null;
+      final FSPermissionChecker pc = getPermissionChecker();
       checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot append to file" + src);
-      src = dir.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = dir.getINodesInPath4Write(src);
-      lb = appendFileInternal(pc, iip, holder, clientMachine, newBlock,
-          logRetryCache);
-      stat = FSDirStatAndListingOp.getFileInfo(dir, src, false,
-          FSDirectory.isReservedRawName(srcArg), true);
-    } catch (StandbyException se) {
-      skipSync = true;
-      throw se;
-    } finally {
-      writeUnlock();
-      // There might be transactions logged while trying to recover the lease.
-      // They need to be sync'ed even when an exception was thrown.
-      if (!skipSync) {
-        getEditLog().logSync();
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot append to file" + srcArg);
+        lbs = FSDirAppendOp.appendFile(this, srcArg, pc, holder, clientMachine,
+            newBlock, logRetryCache);
+      } catch (StandbyException se) {
+        skipSync = true;
+        throw se;
+      } finally {
+        writeUnlock();
+        // There might be transactions logged while trying to recover the lease
+        // They need to be sync'ed even when an exception was thrown.
+        if (!skipSync) {
+          getEditLog().logSync();
+        }
       }
+      logAuditEvent(true, "append", srcArg);
+      return lbs;
+    } catch (AccessControlException e) {
+      logAuditEvent(false, "append", srcArg);
+      throw e;
     }
-    if (lb != null) {
-      NameNode.stateChangeLog.debug(
-          "DIR* NameSystem.appendFile: file {} for {} at {} block {} block" +
-          " size {}", src, holder, clientMachine, lb.getBlock(),
-          lb.getBlock().getNumBytes());
-    }
-    logAuditEvent(true, "append", srcArg);
-    return new LastBlockWithStatus(lb, stat);
   }
 
   ExtendedBlock getExtendedBlock(Block blk) {


[2/9] hadoop git commit: HDFS-8721. Add a metric for number of encryption zones. Contributed by Rakesh R.

Posted by ae...@apache.org.
HDFS-8721. Add a metric for number of encryption zones. Contributed by Rakesh R.


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

Branch: refs/heads/HDFS-7240
Commit: cb03768b1b2250b9b5a7944cf6ef918e8a974e20
Parents: 5137b38
Author: cnauroth <cn...@apache.org>
Authored: Tue Jul 21 13:55:58 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Tue Jul 21 13:55:58 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md                | 1 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                   | 3 +++
 .../hadoop/hdfs/server/namenode/EncryptionZoneManager.java    | 7 +++++++
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java  | 6 ++++++
 .../hdfs/server/namenode/metrics/FSNamesystemMBean.java       | 5 +++++
 .../test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java | 6 ++++++
 .../hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java    | 5 +++++
 7 files changed, 33 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb03768b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index ca89745..2b23508 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -216,6 +216,7 @@ Each metrics record contains tags such as HAState and Hostname as additional inf
 | `TotalLoad` | Current number of connections |
 | `SnapshottableDirectories` | Current number of snapshottable directories |
 | `Snapshots` | Current number of snapshots |
+| `NumEncryptionZones` | Current number of encryption zones |
 | `BlocksTotal` | Current number of allocated blocks in the system |
 | `FilesTotal` | Current number of files and directories |
 | `PendingReplicationBlocks` | Current number of blocks pending to be replicated |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb03768b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a29a090..7c771b0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -734,6 +734,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7483. Display information per tier on the Namenode UI.
     (Benoy Antony and wheat9 via wheat9)
 
+    HDFS-8721. Add a metric for number of encryption zones.
+    (Rakesh R via cnauroth)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb03768b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
index 3fe748d..7c3c895 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
@@ -360,4 +360,11 @@ public class EncryptionZoneManager {
     final boolean hasMore = (numResponses < tailMap.size());
     return new BatchedListEntries<EncryptionZone>(zones, hasMore);
   }
+
+  /**
+   * @return number of encryption zones.
+   */
+  public int getNumEncryptionZones() {
+    return encryptionZones.size();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb03768b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 7c6d6a1..fd37fbe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4075,6 +4075,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return JSON.toString(info);
   }
 
+  @Override // FSNamesystemMBean
+  @Metric({ "NumEncryptionZones", "The number of encryption zones" })
+  public int getNumEncryptionZones() {
+    return dir.ezManager.getNumEncryptionZones();
+  }
+
   int getNumberOfDatanodes(DatanodeReportType type) {
     readLock();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb03768b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
index b31b7b6..580cb78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
@@ -184,4 +184,9 @@ public interface FSNamesystemMBean {
    * @return JSON string
    */
   public String getTopUserOpCounts();
+
+  /**
+   * Return the number of encryption zones in the system.
+   */
+  int getNumEncryptionZones();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb03768b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index e0bd6f4..567a70a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -97,6 +97,8 @@ import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.anyString;
 import static org.apache.hadoop.hdfs.DFSTestUtil.verifyFilesEqual;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -120,6 +122,7 @@ public class TestEncryptionZones {
   protected DistributedFileSystem fs;
   private File testRootDir;
   protected final String TEST_KEY = "test_key";
+  private static final String NS_METRICS = "FSNamesystem";
 
   protected FileSystemTestWrapper fsWrapper;
   protected FileContextTestWrapper fcWrapper;
@@ -358,6 +361,9 @@ public class TestEncryptionZones {
     fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
     cluster.restartNameNode(true);
     assertNumZones(numZones);
+    assertEquals("Unexpected number of encryption zones!", numZones, cluster
+        .getNamesystem().getNumEncryptionZones());
+    assertGauge("NumEncryptionZones", numZones, getMetrics(NS_METRICS));
     assertZonePresent(null, zone1.toString());
 
     // Verify newly added ez is present after restarting the NameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb03768b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
index c044fb0..fb3179a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
@@ -117,6 +117,11 @@ public class TestFSNamesystemMBean {
         "PendingDeletionBlocks");
       assertNotNull(pendingDeletionBlocks);
       assertTrue(pendingDeletionBlocks instanceof Long);
+
+      Object encryptionZones = mbs.getAttribute(mxbeanName,
+          "NumEncryptionZones");
+      assertNotNull(encryptionZones);
+      assertTrue(encryptionZones instanceof Integer);
     } finally {
       if (cluster != null) {
         cluster.shutdown();


[8/9] hadoop git commit: Merge branch 'trunk' into HDFS-7240

Posted by ae...@apache.org.
Merge branch 'trunk' into HDFS-7240


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

Branch: refs/heads/HDFS-7240
Commit: 01094cbf0a44af89fd0a2e91812c9c0756de6934
Parents: 12bd963 4025326
Author: Anu Engineer <an...@gmail.com>
Authored: Wed Jul 22 10:28:28 2015 -0700
Committer: Anu Engineer <an...@gmail.com>
Committed: Wed Jul 22 10:28:28 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../hadoop-common/src/site/markdown/Metrics.md  |   6 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  14 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   3 -
 .../BlockInfoUnderConstruction.java             |  19 +-
 .../server/blockmanagement/BlockManager.java    |  14 +-
 .../blockmanagement/InvalidateBlocks.java       |   5 +-
 .../server/namenode/EncryptionZoneManager.java  |   7 +
 .../hdfs/server/namenode/FSDirAppendOp.java     | 261 +++++++++++++++++++
 .../server/namenode/FSDirStatAndListingOp.java  |   2 +-
 .../hdfs/server/namenode/FSDirTruncateOp.java   |  16 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |   6 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 257 +++---------------
 .../namenode/metrics/FSNamesystemMBean.java     |   5 +
 .../src/main/resources/hdfs-default.xml         |   9 -
 .../apache/hadoop/hdfs/TestEncryptionZones.java |   6 +
 .../apache/hadoop/hdfs/TestLeaseRecovery.java   |  78 ------
 .../server/namenode/TestFSNamesystemMBean.java  |   5 +
 .../org/apache/hadoop/tools/HadoopArchives.java |  21 +-
 .../src/site/markdown/HadoopArchives.md.vm      |   2 +-
 .../apache/hadoop/tools/TestHadoopArchives.java |  26 +-
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop/yarn/event/AsyncDispatcher.java      |   8 +
 .../hadoop/yarn/event/DrainDispatcher.java      |  11 +-
 .../hadoop/yarn/event/TestAsyncDispatcher.java  |  62 +++++
 26 files changed, 470 insertions(+), 383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/01094cbf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------


[3/9] hadoop git commit: HDFS-8773. Few FSNamesystem metrics are not documented in the Metrics page. Contributed by Rakesh R.

Posted by ae...@apache.org.
HDFS-8773. Few FSNamesystem metrics are not documented in the Metrics page. Contributed by Rakesh R.


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

Branch: refs/heads/HDFS-7240
Commit: a26cc66f38daec2342215a66b599bf59cee1112c
Parents: cb03768
Author: cnauroth <cn...@apache.org>
Authored: Tue Jul 21 14:12:03 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Tue Jul 21 14:12:03 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md                  | 5 +++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                     | 3 +++
 2 files changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a26cc66f/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index 2b23508..646cda5 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -231,6 +231,11 @@ Each metrics record contains tags such as HAState and Hostname as additional inf
 | `BlockCapacity` | Current number of block capacity |
 | `StaleDataNodes` | Current number of DataNodes marked stale due to delayed heartbeat |
 | `TotalFiles` | Current number of files and directories (same as FilesTotal) |
+| `MissingReplOneBlocks` | Current number of missing blocks with replication factor 1 |
+| `NumFilesUnderConstruction` | Current number of files under construction |
+| `NumActiveClients` | Current number of active clients holding lease |
+| `HAState` | (HA-only) Current state of the NameNode: initializing or active or standby or stopping state |
+| `FSState` | Current state of the file system: Safemode or Operational |
 
 JournalNode
 -----------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a26cc66f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7c771b0..8122045 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1062,6 +1062,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7582. Enforce maximum number of ACL entries separately per access
     and default. (vinayakumarb)
 
+    HDFS-8773. Few FSNamesystem metrics are not documented in the Metrics page.
+    (Rakesh R via cnauroth)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[7/9] hadoop git commit: HDFS-8795. Improve InvalidateBlocks#node2blocks. (yliu)

Posted by ae...@apache.org.
HDFS-8795. Improve InvalidateBlocks#node2blocks. (yliu)


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

Branch: refs/heads/HDFS-7240
Commit: 4025326288c0167ff300d4f7ecc96f84ed141912
Parents: 94c6a4a
Author: yliu <yl...@apache.org>
Authored: Wed Jul 22 15:16:50 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Wed Jul 22 15:16:50 2015 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                     | 2 ++
 .../hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java    | 5 +++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40253262/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 50803de..66cb89e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -740,6 +740,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8495. Consolidate append() related implementation into a single class.
     (Rakesh R via wheat9)
 
+    HDFS-8795. Improve InvalidateBlocks#node2blocks. (yliu)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40253262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
index a465f85..c486095 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
@@ -22,9 +22,9 @@ import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.GregorianCalendar;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.TreeMap;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -36,6 +36,7 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdfs.DFSUtil;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.slf4j.Logger;
 
 /**
@@ -47,7 +48,7 @@ import org.slf4j.Logger;
 class InvalidateBlocks {
   /** Mapping: DatanodeInfo -> Collection of Blocks */
   private final Map<DatanodeInfo, LightWeightHashSet<Block>> node2blocks =
-      new TreeMap<DatanodeInfo, LightWeightHashSet<Block>>();
+      new HashMap<DatanodeInfo, LightWeightHashSet<Block>>();
   /** The total number of blocks in the map. */
   private long numBlocks = 0L;
 


[6/9] hadoop git commit: HADOOP-12017. Hadoop archives command should use configurable replication factor when closing (Contributed by Bibin A Chundatt)

Posted by ae...@apache.org.
HADOOP-12017. Hadoop archives command should use configurable replication factor when closing (Contributed by Bibin A Chundatt)


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

Branch: refs/heads/HDFS-7240
Commit: 94c6a4aa85e7d98e9b532b330f30783315f4334b
Parents: 31f1171
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Jul 22 10:25:49 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Wed Jul 22 10:25:49 2015 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +++
 .../org/apache/hadoop/tools/HadoopArchives.java | 21 ++++++++++------
 .../src/site/markdown/HadoopArchives.md.vm      |  2 +-
 .../apache/hadoop/tools/TestHadoopArchives.java | 26 ++++++++++++--------
 4 files changed, 33 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94c6a4aa/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 5b51bce..3d101d4 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -992,6 +992,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12051. ProtobufRpcEngine.invoke() should use Exception.toString()
     over getMessage() in logging/span events. (Varun Saxena via stevel)
 
+    HADOOP-12017. Hadoop archives command should use configurable replication
+    factor when closing (Bibin A Chundatt via vinayakumarb)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94c6a4aa/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java b/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
index 330830b..ee14850 100644
--- a/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
+++ b/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
@@ -100,15 +100,17 @@ public class HadoopArchives implements Tool {
   static final String SRC_PARENT_LABEL = NAME + ".parent.path";
   /** the size of the blocks that will be created when archiving **/
   static final String HAR_BLOCKSIZE_LABEL = NAME + ".block.size";
-  /**the size of the part files that will be created when archiving **/
+  /** the replication factor for the file in archiving. **/
+  static final String HAR_REPLICATION_LABEL = NAME + ".replication.factor";
+  /** the size of the part files that will be created when archiving **/
   static final String HAR_PARTSIZE_LABEL = NAME + ".partfile.size";
 
   /** size of each part file size **/
   long partSize = 2 * 1024 * 1024 * 1024l;
   /** size of blocks in hadoop archives **/
   long blockSize = 512 * 1024 * 1024l;
-  /** the desired replication degree; default is 10 **/
-  short repl = 10;
+  /** the desired replication degree; default is 3 **/
+  short repl = 3;
 
   private static final String usage = "archive"
   + " <-archiveName <NAME>.har> <-p <parent path>> [-r <replication factor>]" +
@@ -475,6 +477,7 @@ public class HadoopArchives implements Tool {
     conf.setLong(HAR_PARTSIZE_LABEL, partSize);
     conf.set(DST_HAR_LABEL, archiveName);
     conf.set(SRC_PARENT_LABEL, parentPath.makeQualified(fs).toString());
+    conf.setInt(HAR_REPLICATION_LABEL, repl);
     Path outputPath = new Path(dest, archiveName);
     FileOutputFormat.setOutputPath(conf, outputPath);
     FileSystem outFs = outputPath.getFileSystem(conf);
@@ -549,8 +552,6 @@ public class HadoopArchives implements Tool {
     } finally {
       srcWriter.close();
     }
-    //increase the replication of src files
-    jobfs.setReplication(srcFiles, repl);
     conf.setInt(SRC_COUNT_LABEL, numFiles);
     conf.setLong(TOTAL_SIZE_LABEL, totalSize);
     int numMaps = (int)(totalSize/partSize);
@@ -587,6 +588,7 @@ public class HadoopArchives implements Tool {
     FileSystem destFs = null;
     byte[] buffer;
     int buf_size = 128 * 1024;
+    private int replication = 3;
     long blockSize = 512 * 1024 * 1024l;
 
     // configure the mapper and create 
@@ -595,7 +597,7 @@ public class HadoopArchives implements Tool {
     // tmp files. 
     public void configure(JobConf conf) {
       this.conf = conf;
-
+      replication = conf.getInt(HAR_REPLICATION_LABEL, 3);
       // this is tightly tied to map reduce
       // since it does not expose an api 
       // to get the partition
@@ -712,6 +714,7 @@ public class HadoopArchives implements Tool {
     public void close() throws IOException {
       // close the part files.
       partStream.close();
+      destFs.setReplication(tmpOutput, (short) replication);
     }
   }
   
@@ -732,6 +735,7 @@ public class HadoopArchives implements Tool {
     private int numIndexes = 1000;
     private Path tmpOutputDir = null;
     private int written = 0;
+    private int replication = 3;
     private int keyVal = 0;
     
     // configure 
@@ -740,6 +744,7 @@ public class HadoopArchives implements Tool {
       tmpOutputDir = FileOutputFormat.getWorkOutputPath(this.conf);
       masterIndex = new Path(tmpOutputDir, "_masterindex");
       index = new Path(tmpOutputDir, "_index");
+      replication = conf.getInt(HAR_REPLICATION_LABEL, 3);
       try {
         fs = masterIndex.getFileSystem(conf);
         if (fs.exists(masterIndex)) {
@@ -798,8 +803,8 @@ public class HadoopArchives implements Tool {
       outStream.close();
       indexStream.close();
       // try increasing the replication 
-      fs.setReplication(index, (short) 5);
-      fs.setReplication(masterIndex, (short) 5);
+      fs.setReplication(index, (short) replication);
+      fs.setReplication(masterIndex, (short) replication);
     }
     
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94c6a4aa/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm b/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm
index be557a7..8bbb1ea 100644
--- a/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm
+++ b/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm
@@ -53,7 +53,7 @@ How to Create an Archive
   sections.
 
   -r indicates the desired replication factor; if this optional argument is
-  not specified, a replication factor of 10 will be used.
+  not specified, a replication factor of 3 will be used.
 
   If you just want to archive a single directory /foo/bar then you can just use
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94c6a4aa/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java b/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java
index d8222dc..165c515 100644
--- a/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java
+++ b/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.tools;
 import java.io.ByteArrayOutputStream;
 import java.io.FilterInputStream;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.io.PrintStream;
 import java.net.URI;
 import java.util.ArrayList;
@@ -39,7 +38,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.HarFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.JarFinder;
@@ -110,13 +111,9 @@ public class TestHadoopArchives {
     conf.set(CapacitySchedulerConfiguration.PREFIX
         + CapacitySchedulerConfiguration.ROOT + ".default."
         + CapacitySchedulerConfiguration.CAPACITY, "100");
-    dfscluster = new MiniDFSCluster
-      .Builder(conf)
-      .checkExitOnShutdown(true)
-      .numDataNodes(2)
-      .format(true)
-      .racks(null)
-      .build();
+    dfscluster =
+        new MiniDFSCluster.Builder(conf).checkExitOnShutdown(true)
+            .numDataNodes(3).format(true).racks(null).build();
 
     fs = dfscluster.getFileSystem();
     
@@ -753,12 +750,21 @@ public class TestHadoopArchives {
 
     final String harName = "foo.har";
     final String fullHarPathStr = prefix + harName;
-    final String[] args = { "-archiveName", harName, "-p", inputPathStr, "-r",
-        "3", "*", archivePath.toString() };
+    final String[] args =
+        { "-archiveName", harName, "-p", inputPathStr, "-r", "2", "*",
+            archivePath.toString() };
     System.setProperty(HadoopArchives.TEST_HADOOP_ARCHIVES_JAR_PATH,
         HADOOP_ARCHIVES_JAR);
     final HadoopArchives har = new HadoopArchives(conf);
     assertEquals(0, ToolRunner.run(har, args));
+    RemoteIterator<LocatedFileStatus> listFiles =
+        fs.listFiles(new Path(archivePath.toString() + "/" + harName), false);
+    while (listFiles.hasNext()) {
+      LocatedFileStatus next = listFiles.next();
+      if (!next.getPath().toString().endsWith("_SUCCESS")) {
+        assertEquals(next.getPath().toString(), 2, next.getReplication());
+      }
+    }
     return fullHarPathStr;
   }