You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2013/10/22 02:20:39 UTC

svn commit: r1534426 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/test/java/org/apache/hadoop/hdfs/server/namenode/

Author: szetszwo
Date: Tue Oct 22 00:20:39 2013
New Revision: 1534426

URL: http://svn.apache.org/r1534426
Log:
HDFS-4885. Improve the verifyBlockPlacement() API in BlockPlacementPolicy. Contributed by Junping Du

Added:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java
Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1534426&r1=1534425&r2=1534426&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Tue Oct 22 00:20:39 2013
@@ -332,6 +332,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-4511. Cover package org.apache.hadoop.hdfs.tools with unit test
     (Andrey Klochkov via jeagles)
 
+    HDFS-4885. Improve the verifyBlockPlacement() API in BlockPlacementPolicy.
+    (Junping Du via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java?rev=1534426&r1=1534425&r2=1534426&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java Tue Oct 22 00:20:39 2013
@@ -95,18 +95,17 @@ public abstract class BlockPlacementPoli
   }
 
   /**
-   * Verify that the block is replicated on at least minRacks different racks
-   * if there is more than minRacks rack in the system.
+   * Verify if the block's placement meets requirement of placement policy,
+   * i.e. replicas are placed on no less than minRacks racks in the system.
    * 
    * @param srcPath the full pathname of the file to be verified
    * @param lBlk block with locations
-   * @param minRacks number of racks the block should be replicated to
-   * @return the difference between the required and the actual number of racks
-   * the block is replicated to.
-   */
-  abstract public int verifyBlockPlacement(String srcPath,
-                                           LocatedBlock lBlk,
-                                           int minRacks);
+   * @param numOfReplicas replica number of file to be verified
+   * @return the result of verification
+   */
+  abstract public BlockPlacementStatus verifyBlockPlacement(String srcPath,
+      LocatedBlock lBlk,
+      int numOfReplicas);
   /**
    * Decide whether deleting the specified replica of the block still makes 
    * the block conform to the configured block placement policy.

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java?rev=1534426&r1=1534425&r2=1534426&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java Tue Oct 22 00:20:39 2013
@@ -655,22 +655,22 @@ public class BlockPlacementPolicyDefault
   }
 
   @Override
-  public int verifyBlockPlacement(String srcPath,
-                                  LocatedBlock lBlk,
-                                  int minRacks) {
+  public BlockPlacementStatus verifyBlockPlacement(String srcPath,
+      LocatedBlock lBlk, int numberOfReplicas) {
     DatanodeInfo[] locs = lBlk.getLocations();
     if (locs == null)
       locs = DatanodeDescriptor.EMPTY_ARRAY;
     int numRacks = clusterMap.getNumOfRacks();
     if(numRacks <= 1) // only one rack
-      return 0;
-    minRacks = Math.min(minRacks, numRacks);
+      return new BlockPlacementStatusDefault(
+          Math.min(numRacks, numberOfReplicas), numRacks);
+    int minRacks = Math.min(2, numberOfReplicas);
     // 1. Check that all locations are different.
     // 2. Count locations on different racks.
     Set<String> racks = new TreeSet<String>();
     for (DatanodeInfo dn : locs)
       racks.add(dn.getNetworkLocation());
-    return minRacks - racks.size();
+    return new BlockPlacementStatusDefault(racks.size(), minRacks);
   }
 
   @Override

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java?rev=1534426&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java Tue Oct 22 00:20:39 2013
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface BlockPlacementStatus {
+
+  /**
+   * Boolean value to identify if replicas of this block satisfy requirement of 
+   * placement policy
+   * @return if replicas satisfy placement policy's requirement 
+   */
+  public boolean isPlacementPolicySatisfied();
+  
+  /**
+   * Get description info for log or printed in case replicas are failed to meet
+   * requirement of placement policy
+   * @return description in case replicas are failed to meet requirement of
+   * placement policy
+   */
+  public String getErrorDescription();
+
+}

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java?rev=1534426&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java Tue Oct 22 00:20:39 2013
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+public class BlockPlacementStatusDefault implements BlockPlacementStatus {
+
+  private int requiredRacks = 0;
+  private int currentRacks = 0;
+  
+  public BlockPlacementStatusDefault(int currentRacks, int requiredRacks){
+    this.requiredRacks = requiredRacks;
+    this.currentRacks = currentRacks;
+  }
+  
+  @Override
+  public boolean isPlacementPolicySatisfied() {
+    return requiredRacks <= currentRacks;
+  }
+
+  @Override
+  public String getErrorDescription() {
+    if (isPlacementPolicySatisfied()) {
+      return null;
+    }
+    return "Block should be additionally replicated on " + 
+        (requiredRacks - currentRacks) + " more rack(s).";
+  }
+
+}

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java?rev=1534426&r1=1534425&r2=1534426&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java Tue Oct 22 00:20:39 2013
@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.H
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.net.NetUtils;
@@ -374,9 +375,10 @@ public class NamenodeFsck {
                     locs.length + " replica(s).");
       }
       // verify block placement policy
-      int missingRacks = BlockPlacementPolicy.getInstance(conf, null, networktopology).
-                           verifyBlockPlacement(path, lBlk, Math.min(2,targetFileReplication));
-      if (missingRacks > 0) {
+      BlockPlacementStatus blockPlacementStatus = 
+          BlockPlacementPolicy.getInstance(conf, null, networktopology).
+              verifyBlockPlacement(path, lBlk, targetFileReplication);
+      if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
         res.numMisReplicatedBlocks++;
         misReplicatedPerFile++;
         if (!showFiles) {
@@ -385,9 +387,7 @@ public class NamenodeFsck {
           out.print(path + ": ");
         }
         out.println(" Replica placement policy is violated for " + 
-                    block +
-                    ". Block should be additionally replicated on " + 
-                    missingRacks + " more rack(s).");
+                    block + ". " + blockPlacementStatus.getErrorDescription());
       }
       report.append(i + ". " + blkName + " len=" + block.getNumBytes());
       if (locs.length == 0) {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java?rev=1534426&r1=1534425&r2=1534426&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java Tue Oct 22 00:20:39 2013
@@ -83,7 +83,6 @@ import org.apache.log4j.RollingFileAppen
 import org.junit.Test;
 
 import com.google.common.collect.Sets;
-import org.mockito.Mockito;
 import static org.mockito.Mockito.*;
 
 /**
@@ -892,6 +891,80 @@ public class TestFsck {
       }
     }
   }
+  
+  /**
+   * Tests that the # of misreplaced replicas is correct
+   * @throws IOException
+   */
+  @Test
+  public void testFsckMisPlacedReplicas() throws IOException {
+    // Desired replication factor
+    final short REPL_FACTOR = 2;
+    // Number of replicas to actually start
+    short NUM_DN = 2;
+    // Number of blocks to write
+    final short NUM_BLOCKS = 3;
+    // Set a small-ish blocksize
+    final long blockSize = 512;
+    
+    String [] racks = {"/rack1", "/rack1"};
+    String [] hosts = {"host1", "host2"};
+    
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    
+    MiniDFSCluster cluster = null;
+    DistributedFileSystem dfs = null;
+    
+    try {
+      // Startup a minicluster
+      cluster = 
+          new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DN).hosts(hosts)
+          .racks(racks).build();
+      assertNotNull("Failed Cluster Creation", cluster);
+      cluster.waitClusterUp();
+      dfs = (DistributedFileSystem) cluster.getFileSystem();
+      assertNotNull("Failed to get FileSystem", dfs);
+      
+      // Create a file that will be intentionally under-replicated
+      final String pathString = new String("/testfile");
+      final Path path = new Path(pathString);
+      long fileLen = blockSize * NUM_BLOCKS;
+      DFSTestUtil.createFile(dfs, path, fileLen, REPL_FACTOR, 1);
+      
+      // Create an under-replicated file
+      NameNode namenode = cluster.getNameNode();
+      NetworkTopology nettop = cluster.getNamesystem().getBlockManager()
+          .getDatanodeManager().getNetworkTopology();
+      // Add a new node on different rack, so previous blocks' replicas 
+      // are considered to be misplaced
+      nettop.add(DFSTestUtil.getDatanodeDescriptor("/rack2", "/host3"));
+      NUM_DN++;
+      
+      Map<String,String[]> pmap = new HashMap<String, String[]>();
+      Writer result = new StringWriter();
+      PrintWriter out = new PrintWriter(result, true);
+      InetAddress remoteAddress = InetAddress.getLocalHost();
+      NamenodeFsck fsck = new NamenodeFsck(conf, namenode, nettop, pmap, out, 
+          NUM_DN, (short)REPL_FACTOR, remoteAddress);
+      
+      // Run the fsck and check the Result
+      final HdfsFileStatus file = 
+          namenode.getRpcServer().getFileInfo(pathString);
+      assertNotNull(file);
+      Result res = new Result(conf);
+      fsck.check(pathString, file, res);
+      // check misReplicatedBlock number.
+      assertEquals(res.numMisReplicatedBlocks, NUM_BLOCKS);
+    } finally {
+      if(dfs != null) {
+        dfs.close();
+      }
+      if(cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 
   /** Test fsck with FileNotFound */
   @Test