소스 검색

Merge r1291972 through r1293033 from 0.23.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23-PB@1293035 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 년 전
부모
커밋
024c23c95f
37개의 변경된 파일293개의 추가작업 그리고 203개의 파일을 삭제
  1. 4 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/pom.xml
  3. 13 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  4. 1 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  5. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
  6. 71 28
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  7. 15 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
  8. 34 70
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  9. 12 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  10. 26 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
  11. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  12. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  13. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DataNodeCluster.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  15. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java
  17. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
  18. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
  19. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  20. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
  21. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java
  22. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
  24. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java
  25. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
  26. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
  27. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
  28. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
  29. 37 43
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  30. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java
  31. 16 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
  32. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
  33. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
  34. 5 0
      hadoop-mapreduce-project/CHANGES.txt
  35. 9 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
  36. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
  37. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java

+ 4 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -91,6 +91,10 @@ Release 0.23.2 - UNRELEASED
 
   BUG FIXES
 
+    HADOOP-7660. Maven generated .classpath doesnot includes 
+    "target/generated-test-source/java" as source directory.
+    (Laxman via bobby)
+
     HADOOP-8042  When copying a file out of HDFS, modifying it, and uploading
     it back into HDFS, the put fails due to a CRC mismatch
     (Daryn Sharp via bobby)

+ 1 - 1
hadoop-common-project/hadoop-common/pom.xml

@@ -448,7 +448,7 @@
           </execution>
           <execution>
             <id>add-test-source</id>
-            <phase>generate-test-sources</phase>
+            <phase>generate-sources</phase>
             <goals>
               <goal>add-test-source</goal>
             </goals>

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -147,6 +147,11 @@ Release 0.23.2 - UNRELEASED
     HDFS-2725. hdfs script usage information is missing the information 
     about "dfs" command (Prashant Sharma via stevel)
 
+    HDFS-2907.  Add a conf property dfs.datanode.fsdataset.factory to make
+    FSDataset in Datanode pluggable.  (szetszwo)
+
+    HDFS-2985. Improve logging when replicas are marked as corrupt. (todd)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -174,6 +179,14 @@ Release 0.23.2 - UNRELEASED
 
     HDFS-2969. ExtendedBlock.equals is incorrectly implemented (todd)
 
+    HDFS-2944. Typo in hdfs-default.xml causes
+    dfs.client.block.write.replace-datanode-on-failure.enable to be mistakenly
+    disabled. (atm)
+
+    HDFS-2981. In hdfs-default.xml, the default value of
+    dfs.client.block.write.replace-datanode-on-failure.enable should be true.
+    (szetszwo)
+
 Release 0.23.1 - 2012-02-17 
 
   INCOMPATIBLE CHANGES

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

@@ -167,7 +167,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_CLIENT_RETRY_WINDOW_BASE= "dfs.client.retry.window.base";
   public static final String  DFS_METRICS_SESSION_ID_KEY = "dfs.metrics.session-id";
   public static final String  DFS_DATANODE_HOST_NAME_KEY = "dfs.datanode.hostname";
-  public static final String  DFS_DATANODE_STORAGEID_KEY = "dfs.datanode.StorageId";
   public static final String  DFS_NAMENODE_HOSTS_KEY = "dfs.namenode.hosts";
   public static final String  DFS_NAMENODE_HOSTS_EXCLUDE_KEY = "dfs.namenode.hosts.exclude";
   public static final String  DFS_CLIENT_SOCKET_TIMEOUT_KEY = "dfs.client.socket-timeout";
@@ -211,10 +210,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_DATANODE_NUMBLOCKS_DEFAULT = 64;
   public static final String  DFS_DATANODE_SCAN_PERIOD_HOURS_KEY = "dfs.datanode.scan.period.hours";
   public static final int     DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT = 0;
-  public static final String  DFS_DATANODE_SIMULATEDDATASTORAGE_KEY = "dfs.datanode.simulateddatastorage";
-  public static final boolean DFS_DATANODE_SIMULATEDDATASTORAGE_DEFAULT = false;
-  public static final String  DFS_DATANODE_SIMULATEDDATASTORAGE_CAPACITY_KEY = "dfs.datanode.simulateddatastorage.capacity";
-  public static final long    DFS_DATANODE_SIMULATEDDATASTORAGE_CAPACITY_DEFAULT = 2L<<40;
   public static final String  DFS_DATANODE_TRANSFERTO_ALLOWED_KEY = "dfs.datanode.transferTo.allowed";
   public static final boolean DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT = true;
   public static final String  DFS_DATANODE_BLOCKVOLUMECHOICEPOLICY = "dfs.datanode.block.volume.choice.policy";
@@ -282,6 +277,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
 
   //Keys with no defaults
   public static final String  DFS_DATANODE_PLUGINS_KEY = "dfs.datanode.plugins";
+  public static final String  DFS_DATANODE_FSDATASET_FACTORY_KEY = "dfs.datanode.fsdataset.factory";
   public static final String  DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY = "dfs.datanode.socket.write.timeout";
   public static final String  DFS_DATANODE_STARTUP_KEY = "dfs.datanode.startup";
   public static final String  DFS_NAMENODE_PLUGINS_KEY = "dfs.namenode.plugins";

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java

@@ -88,7 +88,6 @@ public class HdfsConfiguration extends Configuration {
     deprecate("fs.checkpoint.period", DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY);
     deprecate("dfs.upgrade.permission", DFSConfigKeys.DFS_NAMENODE_UPGRADE_PERMISSION_KEY);
     deprecate("heartbeat.recheck.interval", DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY);
-    deprecate("StorageId", DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY);
     deprecate("dfs.https.client.keystore.resource", DFSConfigKeys.DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY);
     deprecate("dfs.https.need.client.auth", DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY);
     deprecate("slave.host.name", DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY);

+ 71 - 28
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -807,9 +807,11 @@ public class BlockManager {
    * Mark the block belonging to datanode as corrupt
    * @param blk Block to be marked as corrupt
    * @param dn Datanode which holds the corrupt replica
+   * @param reason a textual reason why the block should be marked corrupt,
+   * for logging purposes
    */
   public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
-      final DatanodeInfo dn) throws IOException {
+      final DatanodeInfo dn, String reason) throws IOException {
     namesystem.writeLock();
     try {
       final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
@@ -822,14 +824,15 @@ public class BlockManager {
             + blk + " not found.");
         return;
       }
-      markBlockAsCorrupt(storedBlock, dn);
+      markBlockAsCorrupt(storedBlock, dn, reason);
     } finally {
       namesystem.writeUnlock();
     }
   }
 
   private void markBlockAsCorrupt(BlockInfo storedBlock,
-                                  DatanodeInfo dn) throws IOException {
+                                  DatanodeInfo dn,
+                                  String reason) throws IOException {
     assert storedBlock != null : "storedBlock should not be null";
     DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
     if (node == null) {
@@ -853,7 +856,7 @@ public class BlockManager {
     node.addBlock(storedBlock);
 
     // Add this replica to corruptReplicas Map
-    corruptReplicas.addToCorruptReplicasMap(storedBlock, node);
+    corruptReplicas.addToCorruptReplicasMap(storedBlock, node, reason);
     if (countNodes(storedBlock).liveReplicas() >= inode.getReplication()) {
       // the block is over-replicated so invalidate the replicas immediately
       invalidateBlock(storedBlock, node);
@@ -1315,6 +1318,21 @@ public class BlockManager {
       this.reportedState = reportedState;
     }
   }
+  
+  /**
+   * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
+   * list of blocks that should be considered corrupt due to a block report.
+   */
+  private static class BlockToMarkCorrupt {
+    final BlockInfo blockInfo;
+    final String reason;
+    
+    BlockToMarkCorrupt(BlockInfo blockInfo, String reason) {
+      super();
+      this.blockInfo = blockInfo;
+      this.reason = reason;
+    }
+  }
 
   /**
    * The given datanode is reporting all its blocks.
@@ -1369,7 +1387,7 @@ public class BlockManager {
     Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
     Collection<Block> toRemove = new LinkedList<Block>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
-    Collection<BlockInfo> toCorrupt = new LinkedList<BlockInfo>();
+    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
     reportDiff(node, report, toAdd, toRemove, toInvalidate, toCorrupt, toUC);
 
@@ -1389,8 +1407,8 @@ public class BlockManager {
           + " does not belong to any file.");
       addToInvalidates(b, node);
     }
-    for (BlockInfo b : toCorrupt) {
-      markBlockAsCorrupt(b, node);
+    for (BlockToMarkCorrupt b : toCorrupt) {
+      markBlockAsCorrupt(b.blockInfo, node, b.reason);
     }
   }
 
@@ -1421,8 +1439,10 @@ public class BlockManager {
       
       // If block is corrupt, mark it and continue to next block.
       BlockUCState ucState = storedBlock.getBlockUCState();
-      if (isReplicaCorrupt(iblk, reportedState, storedBlock, ucState, node)) {
-        markBlockAsCorrupt(storedBlock, node);
+      BlockToMarkCorrupt c = checkReplicaCorrupt(
+          iblk, reportedState, storedBlock, ucState, node);
+      if (c != null) {
+        markBlockAsCorrupt(c.blockInfo, node, c.reason);
         continue;
       }
       
@@ -1444,7 +1464,7 @@ public class BlockManager {
       Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
       Collection<Block> toRemove,           // remove from DatanodeDescriptor
       Collection<Block> toInvalidate,       // should be removed from DN
-      Collection<BlockInfo> toCorrupt,      // add to corrupt replicas list
+      Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
       Collection<StatefulBlockInfo> toUC) { // add to under-construction list
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
@@ -1507,7 +1527,7 @@ public class BlockManager {
       final Block block, final ReplicaState reportedState, 
       final Collection<BlockInfo> toAdd, 
       final Collection<Block> toInvalidate, 
-      final Collection<BlockInfo> toCorrupt,
+      final Collection<BlockToMarkCorrupt> toCorrupt,
       final Collection<StatefulBlockInfo> toUC) {
     
     if(LOG.isDebugEnabled()) {
@@ -1538,8 +1558,10 @@ public class BlockManager {
       return storedBlock;
     }
 
-    if (isReplicaCorrupt(block, reportedState, storedBlock, ucState, dn)) {
-      toCorrupt.add(storedBlock);
+    BlockToMarkCorrupt c = checkReplicaCorrupt(
+        block, reportedState, storedBlock, ucState, dn);
+    if (c != null) {
+      toCorrupt.add(c);
       return storedBlock;
     }
 
@@ -1563,8 +1585,11 @@ public class BlockManager {
    * as switch statements, on the theory that it is easier to understand
    * the combinatorics of reportedState and ucState that way.  It should be
    * at least as efficient as boolean expressions.
+   * 
+   * @return a BlockToMarkCorrupt object, or null if the replica is not corrupt
    */
-  private boolean isReplicaCorrupt(Block iblk, ReplicaState reportedState, 
+  private BlockToMarkCorrupt checkReplicaCorrupt(
+      Block iblk, ReplicaState reportedState, 
       BlockInfo storedBlock, BlockUCState ucState, 
       DatanodeDescriptor dn) {
     switch(reportedState) {
@@ -1572,17 +1597,31 @@ public class BlockManager {
       switch(ucState) {
       case COMPLETE:
       case COMMITTED:
-        return (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()
-            || storedBlock.getNumBytes() != iblk.getNumBytes());
+        if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
+          return new BlockToMarkCorrupt(storedBlock,
+              "block is " + ucState + " and reported genstamp " +
+              iblk.getGenerationStamp() + " does not match " +
+              "genstamp in block map " + storedBlock.getGenerationStamp());
+        } else if (storedBlock.getNumBytes() != iblk.getNumBytes()) {
+          return new BlockToMarkCorrupt(storedBlock,
+              "block is " + ucState + " and reported length " +
+              iblk.getNumBytes() + " does not match " +
+              "length in block map " + storedBlock.getNumBytes());
+        } else {
+          return null; // not corrupt
+        }
       default:
-        return false;
+        return null;
       }
     case RBW:
     case RWR:
       if (!storedBlock.isComplete()) {
-        return false;
+        return null; // not corrupt
       } else if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
-        return true;
+        return new BlockToMarkCorrupt(storedBlock,
+            "reported " + reportedState + " replica with genstamp " +
+            iblk.getGenerationStamp() + " does not match COMPLETE block's " +
+            "genstamp in block map " + storedBlock.getGenerationStamp());
       } else { // COMPLETE block, same genstamp
         if (reportedState == ReplicaState.RBW) {
           // If it's a RBW report for a COMPLETE block, it may just be that
@@ -1592,18 +1631,22 @@ public class BlockManager {
           LOG.info("Received an RBW replica for block " + storedBlock +
               " on " + dn.getName() + ": ignoring it, since the block is " +
               "complete with the same generation stamp.");
-          return false;
+          return null;
         } else {
-          return true;
+          return new BlockToMarkCorrupt(storedBlock,
+              "reported replica has invalid state " + reportedState);
         }
       }
     case RUR:       // should not be reported
     case TEMPORARY: // should not be reported
     default:
-      LOG.warn("Unexpected replica state " + reportedState
-          + " for block: " + storedBlock + 
-          " on " + dn.getName() + " size " + storedBlock.getNumBytes());
-      return true;
+      String msg = "Unexpected replica state " + reportedState
+      + " for block: " + storedBlock + 
+      " on " + dn.getName() + " size " + storedBlock.getNumBytes();
+      // log here at WARN level since this is really a broken HDFS
+      // invariant
+      LOG.warn(msg);
+      return new BlockToMarkCorrupt(storedBlock, msg);
     }
   }
 
@@ -2134,7 +2177,7 @@ public class BlockManager {
     // blockReceived reports a finalized block
     Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
-    Collection<BlockInfo> toCorrupt = new LinkedList<BlockInfo>();
+    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
     processReportedBlock(node, block, ReplicaState.FINALIZED,
                               toAdd, toInvalidate, toCorrupt, toUC);
@@ -2155,8 +2198,8 @@ public class BlockManager {
           + " does not belong to any file.");
       addToInvalidates(b, node);
     }
-    for (BlockInfo b : toCorrupt) {
-      markBlockAsCorrupt(b, node);
+    for (BlockToMarkCorrupt b : toCorrupt) {
+      markBlockAsCorrupt(b.blockInfo, node, b.reason);
     }
   }
 

+ 15 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java

@@ -44,25 +44,37 @@ public class CorruptReplicasMap{
    *
    * @param blk Block to be added to CorruptReplicasMap
    * @param dn DatanodeDescriptor which holds the corrupt replica
+   * @param reason a textual reason (for logging purposes)
    */
-  public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn) {
+  public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
+      String reason) {
     Collection<DatanodeDescriptor> nodes = getNodes(blk);
     if (nodes == null) {
       nodes = new TreeSet<DatanodeDescriptor>();
       corruptReplicasMap.put(blk, nodes);
     }
+    
+    String reasonText;
+    if (reason != null) {
+      reasonText = " because " + reason;
+    } else {
+      reasonText = "";
+    }
+    
     if (!nodes.contains(dn)) {
       nodes.add(dn);
       NameNode.stateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
                                    blk.getBlockName() +
                                    " added as corrupt on " + dn.getName() +
-                                   " by " + Server.getRemoteIp());
+                                   " by " + Server.getRemoteIp() +
+                                   reasonText);
     } else {
       NameNode.stateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
                                    "duplicate requested for " + 
                                    blk.getBlockName() + " to add as corrupt " +
                                    "on " + dn.getName() +
-                                   " by " + Server.getRemoteIp());
+                                   " by " + Server.getRemoteIp() +
+                                   reasonText);
     }
   }
 

+ 34 - 70
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -43,10 +43,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
 
@@ -162,13 +159,11 @@ import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.util.GenericOptionsParser;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 
-import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingService;
 
 
@@ -438,13 +433,14 @@ public class DataNode extends Configured
     }
   }
 
-  private synchronized void setClusterId(String cid) throws IOException {
-    if(clusterId != null && !clusterId.equals(cid)) {
-      throw new IOException ("cluster id doesn't match. old cid=" + clusterId 
-          + " new cid="+ cid);
+  private synchronized void setClusterId(final String nsCid, final String bpid
+      ) throws IOException {
+    if(clusterId != null && !clusterId.equals(nsCid)) {
+      throw new IOException ("Cluster IDs not matched: dn cid=" + clusterId 
+          + " but ns cid="+ nsCid + "; bpid=" + bpid);
     }
     // else
-    clusterId = cid;    
+    clusterId = nsCid;
   }
 
   private static String getHostName(Configuration config)
@@ -847,51 +843,22 @@ public class DataNode extends Configured
    */
   void initBlockPool(BPOfferService bpos) throws IOException {
     NamespaceInfo nsInfo = bpos.getNamespaceInfo();
-    Preconditions.checkState(nsInfo != null,
-        "Block pool " + bpos + " should have retrieved " +
-        "its namespace info before calling initBlockPool.");
+    if (nsInfo == null) {
+      throw new IOException("NamespaceInfo not found: Block pool " + bpos
+          + " should have retrieved namespace info before initBlockPool.");
+    }
     
-    String blockPoolId = nsInfo.getBlockPoolID();
-
     // Register the new block pool with the BP manager.
     blockPoolManager.addBlockPool(bpos);
 
-    synchronized (this) {
-      // we do not allow namenode from different cluster to register
-      if(clusterId != null && !clusterId.equals(nsInfo.clusterID)) {
-        throw new IOException(
-            "cannot register with the namenode because clusterid do not match:"
-            + " nn=" + nsInfo.getBlockPoolID() + "; nn cid=" + nsInfo.clusterID + 
-            ";dn cid=" + clusterId);
-      }
-
-      setClusterId(nsInfo.clusterID);
-    }
-    
-    StartupOption startOpt = getStartupOption(conf);
-    assert startOpt != null : "Startup option must be set.";
-
-    boolean simulatedFSDataset = conf.getBoolean(
-        DFS_DATANODE_SIMULATEDDATASTORAGE_KEY,
-        DFS_DATANODE_SIMULATEDDATASTORAGE_DEFAULT);
-    
-    if (!simulatedFSDataset) {
-      // read storage info, lock data dirs and transition fs state if necessary          
-      storage.recoverTransitionRead(DataNode.this, blockPoolId, nsInfo,
-          dataDirs, startOpt);
-      StorageInfo bpStorage = storage.getBPStorage(blockPoolId);
-      LOG.info("setting up storage: nsid=" +
-          bpStorage.getNamespaceID() + ";bpid="
-          + blockPoolId + ";lv=" + storage.getLayoutVersion() +
-          ";nsInfo=" + nsInfo);
-    }
+    setClusterId(nsInfo.clusterID, nsInfo.getBlockPoolID());
     
     // In the case that this is the first block pool to connect, initialize
     // the dataset, block scanners, etc.
-    initFsDataSet();
+    initStorage(nsInfo);
     initPeriodicScanners(conf);
     
-    data.addBlockPool(blockPoolId, conf);
+    data.addBlockPool(nsInfo.getBlockPoolID(), conf);
   }
 
   /**
@@ -918,31 +885,28 @@ public class DataNode extends Configured
    * Initializes the {@link #data}. The initialization is done only once, when
    * handshake with the the first namenode is completed.
    */
-  private synchronized void initFsDataSet() throws IOException {
-    if (data != null) { // Already initialized
-      return;
-    }
-
-    // get version and id info from the name-node
-    boolean simulatedFSDataset = conf.getBoolean(
-        DFS_DATANODE_SIMULATEDDATASTORAGE_KEY,
-        DFS_DATANODE_SIMULATEDDATASTORAGE_DEFAULT);
-
-    if (simulatedFSDataset) {
-      storage.createStorageID(getPort());
-      // it would have been better to pass storage as a parameter to
-      // constructor below - need to augment ReflectionUtils used below.
-      conf.set(DFS_DATANODE_STORAGEID_KEY, getStorageId());
-      try {
-        data = (FSDatasetInterface) ReflectionUtils.newInstance(
-            Class.forName(
-            "org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset"),
-            conf);
-      } catch (ClassNotFoundException e) {
-        throw new IOException(StringUtils.stringifyException(e));
+  private void initStorage(final NamespaceInfo nsInfo) throws IOException {
+    final FSDatasetInterface.Factory factory
+        = FSDatasetInterface.Factory.getFactory(conf);
+    
+    if (!factory.isSimulated()) {
+      final StartupOption startOpt = getStartupOption(conf);
+      if (startOpt == null) {
+        throw new IOException("Startup option not set.");
+      }
+      final String bpid = nsInfo.getBlockPoolID();
+      //read storage info, lock data dirs and transition fs state if necessary
+      storage.recoverTransitionRead(this, bpid, nsInfo, dataDirs, startOpt);
+      final StorageInfo bpStorage = storage.getBPStorage(bpid);
+      LOG.info("Setting up storage: nsid=" + bpStorage.getNamespaceID()
+          + ";bpid=" + bpid + ";lv=" + storage.getLayoutVersion()
+          + ";nsInfo=" + nsInfo);
+    }
+
+    synchronized(this)  {
+      if (data == null) {
+        data = factory.createFSDatasetInterface(this, storage, conf);
       }
-    } else {
-      data = new FSDataset(this, storage, conf);
     }
   }
 

+ 12 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -75,6 +75,16 @@ import org.apache.hadoop.util.ReflectionUtils;
  ***************************************************/
 @InterfaceAudience.Private
 class FSDataset implements FSDatasetInterface {
+  /**
+   * A factory for creating FSDataset objects.
+   */
+  static class Factory extends FSDatasetInterface.Factory {
+    @Override
+    public FSDatasetInterface createFSDatasetInterface(DataNode datanode,
+        DataStorage storage, Configuration conf) throws IOException {
+      return new FSDataset(datanode, storage, conf);
+    }
+  }
 
   /**
    * A node type that can be built into a tree reflecting the
@@ -1056,8 +1066,8 @@ class FSDataset implements FSDatasetInterface {
   /**
    * An FSDataset has a directory where it loads its data files.
    */
-  FSDataset(DataNode datanode, DataStorage storage, Configuration conf)
-      throws IOException {
+  private FSDataset(DataNode datanode, DataStorage storage, Configuration conf
+      ) throws IOException {
     this.datanode = datanode;
     this.maxBlocksPerDir = 
       conf.getInt(DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_KEY,

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java

@@ -29,6 +29,7 @@ import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlo
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 /**
@@ -49,6 +51,30 @@ import org.apache.hadoop.util.DiskChecker.DiskErrorException;
  */
 @InterfaceAudience.Private
 public interface FSDatasetInterface extends FSDatasetMBean {
+  /**
+   * A factory for creating FSDatasetInterface objects.
+   */
+  public abstract class Factory {
+    /** @return the configured factory. */
+    public static Factory getFactory(Configuration conf) {
+      final Class<? extends Factory> clazz = conf.getClass(
+          DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
+          FSDataset.Factory.class,
+          Factory.class);
+      return ReflectionUtils.newInstance(clazz, conf);
+    }
+
+    /** Create a FSDatasetInterface object. */
+    public abstract FSDatasetInterface createFSDatasetInterface(
+        DataNode datanode, DataStorage storage, Configuration conf
+        ) throws IOException;
+
+    /** Does the factory create simulated objects? */
+    public boolean isSimulated() {
+      return false;
+    }
+  }
+
   /**
    * This is an interface for the underlying volume.
    * @see org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume

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

@@ -550,7 +550,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
       DatanodeInfo[] nodes = blocks[i].getLocations();
       for (int j = 0; j < nodes.length; j++) {
         DatanodeInfo dn = nodes[j];
-        namesystem.getBlockManager().findAndMarkBlockAsCorrupt(blk, dn);
+        namesystem.getBlockManager().findAndMarkBlockAsCorrupt(blk, dn,
+            "client machine reported it");
       }
     }
   }

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

@@ -347,7 +347,7 @@ creations/deletions), or "all".</description>
 
 <property>
   <name>dfs.client.block.write.replace-datanode-on-failure.enable</name>
-  <value>ture</value>
+  <value>true</value>
   <description>
     If there is a datanode/network failure in the write pipeline,
     DFSClient will try to remove the failed datanode from the pipeline
@@ -355,7 +355,7 @@ creations/deletions), or "all".</description>
     the number of datanodes in the pipeline is decreased.  The feature is
     to add new datanodes to the pipeline.
 
-    This is a site-wise property to enable/disable the feature.
+    This is a site-wide property to enable/disable the feature.
 
     See also dfs.client.block.write.replace-datanode-on-failure.policy
   </description>

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DataNodeCluster.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.namenode.CreateEditsLog;
 import org.apache.hadoop.net.DNS;
@@ -122,10 +123,9 @@ public class DataNodeCluster {
         }
         dataNodeDirs = args[i];
       } else if (args[i].equals("-simulated")) {
-        conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+        SimulatedFSDataset.setFactory(conf);
       } else if (args[i].equals("-inject")) {
-        if (!conf.getBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED,
-                                                                false) ) {
+        if (!FSDatasetInterface.Factory.getFactory(conf).isSimulated()) {
           System.out.print("-inject is valid only for simulated");
           printUsageExit(); 
         }
@@ -158,7 +158,7 @@ public class DataNodeCluster {
       System.exit(-1);
     }
     boolean simulated = 
-      conf.getBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, false);
+        FSDatasetInterface.Factory.getFactory(conf).isSimulated();
     System.out.println("Starting " + numDataNodes + 
           (simulated ? " Simulated " : " ") +
           " Data Nodes that will connect to Name Node at " + nameNodeAdr);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -842,7 +842,7 @@ public class MiniDFSCluster {
         conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs);
       }
       if (simulatedCapacities != null) {
-        dnConf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+        SimulatedFSDataset.setFactory(dnConf);
         dnConf.setLong(SimulatedFSDataset.CONFIG_PROPERTY_CAPACITY,
             simulatedCapacities[i-curDatanodesNum]);
       }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java

@@ -107,7 +107,7 @@ public class TestFileAppend{
   public void testCopyOnWrite() throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
@@ -178,7 +178,7 @@ public class TestFileAppend{
   public void testSimpleFlush() throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -234,7 +234,7 @@ public class TestFileAppend{
   public void testComplexFlush() throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -283,7 +283,7 @@ public class TestFileAppend{
   public void testFileNotFound() throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java

@@ -82,7 +82,7 @@ public class TestFileAppend2 extends TestCase {
   public void testSimpleAppend() throws IOException {
     final Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
     conf.setBoolean("dfs.support.append", true);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java

@@ -77,7 +77,7 @@ public class TestFileAppend4 {
   public void setUp() throws Exception {
     this.conf = new Configuration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java

@@ -147,7 +147,7 @@ public class TestFileCorruption extends TestCase {
       DatanodeRegistration dnR = 
         DataNodeTestUtils.getDNRegistrationForBP(dataNode, blk.getBlockPoolId());
       cluster.getNamesystem().getBlockManager().findAndMarkBlockAsCorrupt(
-          blk, new DatanodeInfo(dnR));
+          blk, new DatanodeInfo(dnR), "TEST");
       
       // open the file
       fs.open(FILE_PATH);

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -144,7 +144,7 @@ public class TestFileCreation extends junit.framework.TestCase {
   public void testFileCreation() throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
@@ -223,7 +223,7 @@ public class TestFileCreation extends junit.framework.TestCase {
   public void testDeleteOnExit() throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
@@ -287,7 +287,7 @@ public class TestFileCreation extends junit.framework.TestCase {
     conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFS_HEARTBEAT_INTERVAL_KEY, 1);
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     // create cluster
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -361,7 +361,7 @@ public class TestFileCreation extends junit.framework.TestCase {
     conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFS_HEARTBEAT_INTERVAL_KEY, 1);
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     // create cluster
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -460,7 +460,7 @@ public class TestFileCreation extends junit.framework.TestCase {
     conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFS_HEARTBEAT_INTERVAL_KEY, 1);
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
 
     // create cluster
@@ -599,7 +599,7 @@ public class TestFileCreation extends junit.framework.TestCase {
     Configuration conf = new HdfsConfiguration();
     System.out.println("Testing adbornal client death.");
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
@@ -634,7 +634,7 @@ public class TestFileCreation extends junit.framework.TestCase {
   public void testFileCreationNonRecursive() throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();

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

@@ -136,7 +136,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
       Configuration conf = new HdfsConfiguration();
       conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, Integer.toString(numDataNodes));
       conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, checksumSize);
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
       //first time format
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
       cluster.waitActive();
@@ -159,7 +159,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
       
       LOG.info("Restarting minicluster");
       conf = new HdfsConfiguration();
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
       conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, "0.0f"); 
       
       cluster = new MiniDFSCluster.Builder(conf)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java

@@ -174,7 +174,7 @@ public class TestLargeBlock {
 
     Configuration conf = new Configuration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java

@@ -206,7 +206,7 @@ public class TestPread extends TestCase {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
     conf.setLong(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 4096);
     if (simulatedStorage) {
-      conf.setBoolean("dfs.datanode.simulateddatastorage", true);
+      SimulatedFSDataset.setFactory(conf);
     }
     if (disableTransferTo) {
       conf.setBoolean("dfs.datanode.transferTo.allowed", false);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java

@@ -199,7 +199,7 @@ public class TestReplication extends TestCase {
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
     if (simulated) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
                                                .numDataNodes(numDatanodes)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java

@@ -28,7 +28,7 @@ public class TestSetrepIncreasing extends TestCase {
   static void setrep(int fromREP, int toREP, boolean simulatedStorage) throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, "" + fromREP);
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);

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

@@ -124,7 +124,7 @@ public class TestShortCircuitLocalRead {
     conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
         UserGroupInformation.getCurrentUser().getShortUserName());
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .format(true).build();
@@ -248,7 +248,7 @@ public class TestShortCircuitLocalRead {
     conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
         UserGroupInformation.getCurrentUser().getShortUserName());
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .format(true).build();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java

@@ -92,7 +92,7 @@ public class TestSmallBlock extends TestCase {
   public void testSmallBlock() throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean("dfs.datanode.simulateddatastorage", true);
+      SimulatedFSDataset.setFactory(conf);
     }
     conf.set(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, "1");
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java

@@ -77,7 +77,7 @@ public class TestBalancer extends TestCase {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
     conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
-    conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+    SimulatedFSDataset.setFactory(conf);
     conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
   }
 

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java

@@ -83,14 +83,14 @@ public class TestCorruptReplicaInfo extends TestCase {
       DatanodeDescriptor dn1 = new DatanodeDescriptor();
       DatanodeDescriptor dn2 = new DatanodeDescriptor();
       
-      crm.addToCorruptReplicasMap(getBlock(0), dn1);
+      crm.addToCorruptReplicasMap(getBlock(0), dn1, "TEST");
       assertEquals("Number of corrupt blocks not returning correctly",
                    1, crm.size());
-      crm.addToCorruptReplicasMap(getBlock(1), dn1);
+      crm.addToCorruptReplicasMap(getBlock(1), dn1, "TEST");
       assertEquals("Number of corrupt blocks not returning correctly",
                    2, crm.size());
       
-      crm.addToCorruptReplicasMap(getBlock(1), dn2);
+      crm.addToCorruptReplicasMap(getBlock(1), dn2, "TEST");
       assertEquals("Number of corrupt blocks not returning correctly",
                    2, crm.size());
       
@@ -103,7 +103,7 @@ public class TestCorruptReplicaInfo extends TestCase {
                    0, crm.size());
       
       for (Long block_id: block_ids) {
-        crm.addToCorruptReplicasMap(getBlock(block_id), dn1);
+        crm.addToCorruptReplicasMap(getBlock(block_id), dn1, "TEST");
       }
             
       assertEquals("Number of corrupt blocks not returning correctly",

+ 37 - 43
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -32,7 +31,6 @@ import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
 import javax.management.StandardMBean;
 
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -63,21 +61,33 @@ import org.apache.hadoop.util.DiskChecker.DiskErrorException;
  * 
  * Note the synchronization is coarse grained - it is at each method. 
  */
+public class SimulatedFSDataset implements FSDatasetInterface {
+  static class Factory extends FSDatasetInterface.Factory {
+    @Override
+    public FSDatasetInterface createFSDatasetInterface(DataNode datanode,
+        DataStorage storage, Configuration conf) throws IOException {
+      return new SimulatedFSDataset(datanode, storage, conf);
+    }
 
-public class SimulatedFSDataset  implements FSDatasetInterface, Configurable{
+    @Override
+    public boolean isSimulated() {
+      return true;
+    }
+  }
+  
+  public static void setFactory(Configuration conf) {
+    conf.set(DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
+        Factory.class.getName());
+  }
   
-  public static final String CONFIG_PROPERTY_SIMULATED =
-                                    "dfs.datanode.simulateddatastorage";
   public static final String CONFIG_PROPERTY_CAPACITY =
-                            "dfs.datanode.simulateddatastorage.capacity";
+      "dfs.datanode.simulateddatastorage.capacity";
   
   public static final long DEFAULT_CAPACITY = 2L<<40; // 1 terabyte
-  public static final byte DEFAULT_DATABYTE = 9; // 1 terabyte
-  byte simulatedDataByte = DEFAULT_DATABYTE;
-  Configuration conf = null;
+  public static final byte DEFAULT_DATABYTE = 9;
   
-  static byte[] nullCrcFileData;
-  {
+  static final byte[] nullCrcFileData;
+  static {
     DataChecksum checksum = DataChecksum.newDataChecksum( DataChecksum.
                               CHECKSUM_NULL, 16*1024 );
     byte[] nullCrcHeader = checksum.getHeader();
@@ -360,31 +370,22 @@ public class SimulatedFSDataset  implements FSDatasetInterface, Configurable{
     }
   }
   
-  private Map<String, Map<Block, BInfo>> blockMap = null;
-  private SimulatedStorage storage = null;
-  private String storageId;
-  
-  public SimulatedFSDataset(Configuration conf) throws IOException {
-    setConf(conf);
-  }
+  private final Map<String, Map<Block, BInfo>> blockMap
+      = new HashMap<String, Map<Block,BInfo>>();
+  private final SimulatedStorage storage;
+  private final String storageId;
   
-  // Constructor used for constructing the object using reflection
-  @SuppressWarnings("unused")
-  private SimulatedFSDataset() { // real construction when setConf called..
-  }
-  
-  public Configuration getConf() {
-    return conf;
-  }
-
-  public void setConf(Configuration iconf)  {
-    conf = iconf;
-    storageId = conf.get(DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY, "unknownStorageId" +
-                                        new Random().nextInt());
+  public SimulatedFSDataset(DataNode datanode, DataStorage storage,
+      Configuration conf) {
+    if (storage != null) {
+      storage.createStorageID(datanode.getPort());
+      this.storageId = storage.getStorageID();
+    } else {
+      this.storageId = "unknownStorageId" + new Random().nextInt();
+    }
     registerMBean(storageId);
-    storage = new SimulatedStorage(
+    this.storage = new SimulatedStorage(
         conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY));
-    blockMap = new HashMap<String, Map<Block,BInfo>>(); 
   }
 
   public synchronized void injectBlocks(String bpid,
@@ -441,23 +442,16 @@ public class SimulatedFSDataset  implements FSDatasetInterface, Configurable{
 
   @Override
   public synchronized BlockListAsLongs getBlockReport(String bpid) {
+    final List<Block> blocks = new ArrayList<Block>();
     final Map<Block, BInfo> map = blockMap.get(bpid);
-    Block[] blockTable = new Block[map.size()];
     if (map != null) {
-      int count = 0;
       for (BInfo b : map.values()) {
         if (b.isFinalized()) {
-          blockTable[count++] = b.theBlock;
+          blocks.add(b.theBlock);
         }
       }
-      if (count != blockTable.length) {
-        blockTable = Arrays.copyOf(blockTable, count);
-      }
-    } else {
-      blockTable = new Block[0];
     }
-    return new BlockListAsLongs(
-        new ArrayList<Block>(Arrays.asList(blockTable)), null);
+    return new BlockListAsLongs(blocks, null);
   }
 
   @Override // FSDatasetMBean

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java

@@ -34,7 +34,7 @@ public class TestDataNodeMetrics extends TestCase {
   
   public void testDataNodeMetrics() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+    SimulatedFSDataset.setFactory(conf);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     try {
       FileSystem fs = cluster.getFileSystem();

+ 16 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java

@@ -44,8 +44,8 @@ public class TestSimulatedFSDataset extends TestCase {
 
   protected void setUp() throws Exception {
     super.setUp();
-      conf = new HdfsConfiguration();
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+    conf = new HdfsConfiguration();
+    SimulatedFSDataset.setFactory(conf);
   }
 
   protected void tearDown() throws Exception {
@@ -86,6 +86,18 @@ public class TestSimulatedFSDataset extends TestCase {
   int addSomeBlocks(FSDatasetInterface fsdataset ) throws IOException {
     return addSomeBlocks(fsdataset, 1);
   }
+  
+  public void testFSDatasetFactory() {
+    final Configuration conf = new Configuration();
+    FSDatasetInterface.Factory f = FSDatasetInterface.Factory.getFactory(conf);
+    assertEquals(FSDataset.Factory.class, f.getClass());
+    assertFalse(f.isSimulated());
+
+    SimulatedFSDataset.setFactory(conf);
+    FSDatasetInterface.Factory s = FSDatasetInterface.Factory.getFactory(conf);
+    assertEquals(SimulatedFSDataset.Factory.class, s.getClass());
+    assertTrue(s.isSimulated());
+  }
 
   public void testGetMetaData() throws IOException {
     FSDatasetInterface fsdataset = getSimulatedFSDataset(); 
@@ -287,8 +299,8 @@ public class TestSimulatedFSDataset extends TestCase {
     }
   }
   
-  private SimulatedFSDataset getSimulatedFSDataset() throws IOException {
-    SimulatedFSDataset fsdataset = new SimulatedFSDataset(conf); 
+  private SimulatedFSDataset getSimulatedFSDataset() {
+    SimulatedFSDataset fsdataset = new SimulatedFSDataset(null, null, conf); 
     fsdataset.addBlockPool(bpid, conf);
     return fsdataset;
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java

@@ -83,7 +83,7 @@ public class TestFileLimit extends TestCase {
     int currentNodes = 0;
     
     if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+      SimulatedFSDataset.setFactory(conf);
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java

@@ -174,7 +174,8 @@ public class TestNameNodeMetrics {
     // Corrupt first replica of the block
     LocatedBlock block = NameNodeAdapter.getBlockLocations(
         cluster.getNameNode(), file.toString(), 0, 1).get(0);
-    bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0]);
+    bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0],
+        "TEST");
     updateMetrics();
     MetricsRecordBuilder rb = getMetrics(NS_METRICS);
     assertGauge("CorruptBlocks", 1L, rb);
@@ -213,7 +214,8 @@ public class TestNameNodeMetrics {
     // Corrupt the only replica of the block to result in a missing block
     LocatedBlock block = NameNodeAdapter.getBlockLocations(
         cluster.getNameNode(), file.toString(), 0, 1).get(0);
-    bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0]);
+    bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0],
+        "TEST");
     updateMetrics();
     MetricsRecordBuilder rb = getMetrics(NS_METRICS);
     assertGauge("UnderReplicatedBlocks", 1L, rb);

+ 5 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -69,6 +69,11 @@ Release 0.23.2 - UNRELEASED
 
     MAPREDUCE-3798. Fixed failing TestJobCleanup.testCusomCleanup() and moved it
     to the maven build. (Ravi Prakash via vinodkv)
+
+    MAPREDUCE-3884. PWD should be first in the classpath of MR tasks (tucu)
+
+    MAPREDUCE-3878. Null user on filtered jobhistory job page (Jonathon Eagles
+    via tgraves)
  
 Release 0.23.1 - 2012-02-17 
 

+ 9 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java

@@ -343,9 +343,15 @@ public class AppController extends Controller implements AMParams {
    * @return True if the requesting user has permission to view the job
    */
   boolean checkAccess(Job job) {
-    UserGroupInformation callerUgi = UserGroupInformation.createRemoteUser(
-        request().getRemoteUser());
-    return job.checkAccess(callerUgi, JobACL.VIEW_JOB);
+    String remoteUser = request().getRemoteUser();
+    UserGroupInformation callerUGI = null;
+    if (remoteUser != null) {
+      callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
+    }
+    if (callerUGI != null && !job.checkAccess(callerUGI, JobACL.VIEW_JOB)) {
+      return false;
+    }
+    return true;
   }
 
   /**

+ 3 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java

@@ -230,6 +230,9 @@ public class MRApps extends Apps {
     boolean userClassesTakesPrecedence = 
       conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_USER_CLASSPATH_FIRST, false);
 
+    Apps.addToEnvironment(environment,
+      Environment.CLASSPATH.name(),
+      Environment.PWD.$());
     if (!userClassesTakesPrecedence) {
       MRApps.setMRFrameworkClasspath(environment, conf);
     }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java

@@ -130,7 +130,7 @@ public class TestMRApps {
     Job job = Job.getInstance();
     Map<String, String> environment = new HashMap<String, String>();
     MRApps.setClasspath(environment, job.getConfiguration());
-    assertEquals("$HADOOP_CONF_DIR:" +
+    assertEquals("$PWD:$HADOOP_CONF_DIR:" +
         "$HADOOP_COMMON_HOME/share/hadoop/common/*:" +
         "$HADOOP_COMMON_HOME/share/hadoop/common/lib/*:" +
         "$HADOOP_HDFS_HOME/share/hadoop/hdfs/*:" +
@@ -152,7 +152,7 @@ public class TestMRApps {
     }
     String env_str = env.get("CLASSPATH");
     assertSame("MAPREDUCE_JOB_USER_CLASSPATH_FIRST set, but not taking effect!",
-      env_str.indexOf("job.jar"), 0);
+      env_str.indexOf("$PWD:job.jar"), 0);
   }
 
   @Test public void testSetClasspathWithNoUserPrecendence() {
@@ -166,7 +166,7 @@ public class TestMRApps {
     }
     String env_str = env.get("CLASSPATH");
     assertNotSame("MAPREDUCE_JOB_USER_CLASSPATH_FIRST false, but taking effect!",
-      env_str.indexOf("job.jar"), 0);
+      env_str.indexOf("$PWD:job.jar"), 0);
   }
 
 }