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 ha...@apache.org on 2010/03/30 00:12:32 UTC

svn commit: r928917 - in /hadoop/hdfs/trunk: CHANGES.txt src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

Author: hairong
Date: Mon Mar 29 22:12:32 2010
New Revision: 928917

URL: http://svn.apache.org/viewvc?rev=928917&view=rev
Log:
HDFS-1067. Create block recovery tests that handle errors. Contributed by Hairong Kuang.

Modified:
    hadoop/hdfs/trunk/CHANGES.txt
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
    hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=928917&r1=928916&r2=928917&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Mon Mar 29 22:12:32 2010
@@ -550,6 +550,8 @@ Release 0.21.0 - Unreleased
 
     HDFS-520. Create new tests for block recovery. (hairong)
 
+    HDFS-1067. Create block recovery tests that handle errors. (hairong)
+
   BUG FIXES
 
     HDFS-76. Better error message to users when commands fail because of 

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java?rev=928917&r1=928916&r2=928917&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java Mon Mar 29 22:12:32 2010
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.List;
@@ -173,14 +174,14 @@ public class TestInterDatanodeProtocol {
       }
     }
 
-    { //replica not found
+    { // BlockRecoveryFI_01: replica not found
       final long recoveryid = gs + 1;
       final Block b = new Block(firstblockid - 1, length, gs);
       ReplicaRecoveryInfo r = FSDataset.initReplicaRecovery(map, b, recoveryid);
       Assert.assertNull("Data-node should not have this replica.", r);
     }
     
-    { //case "THIS IS NOT SUPPOSED TO HAPPEN" with recovery id < gs  
+    { // BlockRecoveryFI_02: "THIS IS NOT SUPPOSED TO HAPPEN" with recovery id < gs  
       final long recoveryid = gs - 1;
       final Block b = new Block(firstblockid + 1, length, gs);
       try {
@@ -192,6 +193,19 @@ public class TestInterDatanodeProtocol {
       }
     }
 
+    // BlockRecoveryFI_03: Replica's gs is less than the block's gs
+    {
+      final long recoveryid = gs + 1;
+      final Block b = new Block(firstblockid, length, gs+1);
+      try {
+        FSDataset.initReplicaRecovery(map, b, recoveryid);
+        fail("InitReplicaRecovery should fail because replica's " +
+        		"gs is less than the block's gs");
+      } catch (IOException e) {
+        e.getMessage().startsWith(
+           "replica.getGenerationStamp() < block.getGenerationStamp(), block=");
+      }
+    }
   }
 
   /** Test {@link FSDataset#updateReplicaUnderRecovery(ReplicaUnderRecovery, long, long)} */

Modified: hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java?rev=928917&r1=928916&r2=928917&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java (original)
+++ hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java Mon Mar 29 22:12:32 2010
@@ -30,9 +30,11 @@ import org.apache.hadoop.hdfs.MiniDFSClu
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode.BlockRecord;
+import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.BlockWriteStreams;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -42,16 +44,21 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Daemon;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.*;
 
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
 
 /**
  * This tests if sync all replicas in block recovery works correctly
@@ -313,4 +320,148 @@ public class TestBlockRecovery {
     verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, minLen);
     verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, minLen);    
   }  
+
+  private Collection<RecoveringBlock> initRecoveringBlocks() {
+    Collection<RecoveringBlock> blocks = new ArrayList<RecoveringBlock>(1);
+    DatanodeInfo[] locs = new DatanodeInfo[] {
+        new DatanodeInfo(dn.dnRegistration),
+        mock(DatanodeInfo.class) };
+    RecoveringBlock rBlock = new RecoveringBlock(block, locs, RECOVERY_ID);
+    blocks.add(rBlock);
+    return blocks;
+  }
+  /**
+   * BlockRecoveryFI_05. One DN throws RecoveryInProgressException.
+   *
+   * @throws IOException
+   *           in case of an error
+   */
+  @Test
+  public void testRecoveryInProgressException()
+    throws IOException, InterruptedException {
+    LOG.debug("Running " + GenericTestUtils.getMethodName());
+    DataNode spyDN = spy(dn);
+    doThrow(new RecoveryInProgressException("Replica recovery is in progress")).
+       when(spyDN).initReplicaRecovery(any(RecoveringBlock.class));
+    Daemon d = spyDN.recoverBlocks(initRecoveringBlocks());
+    d.join();
+    verify(spyDN, never()).syncBlock(
+        any(RecoveringBlock.class), anyListOf(BlockRecord.class));
+  }
+
+  /**
+   * BlockRecoveryFI_06. all datanodes throws an exception.
+   *
+   * @throws IOException
+   *           in case of an error
+   */
+  @Test
+  public void testErrorReplicas() throws IOException, InterruptedException {
+    LOG.debug("Running " + GenericTestUtils.getMethodName());
+    DataNode spyDN = spy(dn);
+    doThrow(new IOException()).
+       when(spyDN).initReplicaRecovery(any(RecoveringBlock.class));
+    Daemon d = spyDN.recoverBlocks(initRecoveringBlocks());
+    d.join();
+    verify(spyDN, never()).syncBlock(
+        any(RecoveringBlock.class), anyListOf(BlockRecord.class));
+  }
+
+  /**
+   * BlockRecoveryFI_07. max replica length from all DNs is zero.
+   *
+   * @throws IOException in case of an error
+   */
+  @Test
+  public void testZeroLenReplicas() throws IOException, InterruptedException {
+    LOG.debug("Running " + GenericTestUtils.getMethodName());
+    DataNode spyDN = spy(dn);
+    doReturn(new ReplicaRecoveryInfo(block.getBlockId(), 0,
+        block.getGenerationStamp(), ReplicaState.FINALIZED)).when(spyDN).
+        initReplicaRecovery(any(RecoveringBlock.class));
+    Daemon d = spyDN.recoverBlocks(initRecoveringBlocks());
+    d.join();
+    verify(dn.namenode).commitBlockSynchronization(
+        block, RECOVERY_ID, 0, true, true, DatanodeID.EMPTY_ARRAY);
+  }
+
+  private List<BlockRecord> initBlockRecords(DataNode spyDN) {
+    List<BlockRecord> blocks = new ArrayList<BlockRecord>(1);
+    BlockRecord blockRecord = new BlockRecord(
+        new DatanodeID(dn.dnRegistration), spyDN,
+        new ReplicaRecoveryInfo(block.getBlockId(), block.getNumBytes(),
+            block.getGenerationStamp(), ReplicaState.FINALIZED));
+    blocks.add(blockRecord);
+    return blocks;
+  }
+
+  private final static RecoveringBlock rBlock =
+    new RecoveringBlock(block, null, RECOVERY_ID);
+
+  /**
+   * BlockRecoveryFI_09. some/all DNs failed to update replicas.
+   *
+   * @throws IOException in case of an error
+   */
+  @Test
+  public void testFailedReplicaUpdate() throws IOException {
+    LOG.debug("Running " + GenericTestUtils.getMethodName());
+    DataNode spyDN = spy(dn);
+    doThrow(new IOException()).when(spyDN).updateReplicaUnderRecovery(
+        block, RECOVERY_ID, block.getNumBytes());
+    try {
+      spyDN.syncBlock(rBlock, initBlockRecords(spyDN));
+      fail("Sync should fail");
+    } catch (IOException e) {
+      e.getMessage().startsWith("Cannot recover ");
+    }
+  }
+
+  /**
+   * BlockRecoveryFI_10. DN has no ReplicaUnderRecovery.
+   *
+   * @throws IOException in case of an error
+   */
+  @Test
+  public void testNoReplicaUnderRecovery() throws IOException {
+    LOG.debug("Running " + GenericTestUtils.getMethodName());
+    dn.data.createRbw(block);
+    try {
+      dn.syncBlock(rBlock, initBlockRecords(dn));
+      fail("Sync should fail");
+    } catch (IOException e) {
+      e.getMessage().startsWith("Cannot recover ");
+    }
+    verify(dn.namenode, never()).commitBlockSynchronization(
+        any(Block.class), anyLong(), anyLong(), anyBoolean(),
+        anyBoolean(), any(DatanodeID[].class));
+  }
+
+  /**
+   * BlockRecoveryFI_11. a replica's recovery id does not match new GS.
+   *
+   * @throws IOException in case of an error
+   */
+  @Test
+  public void testNotMatchedReplicaID() throws IOException {
+    LOG.debug("Running " + GenericTestUtils.getMethodName());
+    ReplicaInPipelineInterface replicaInfo = dn.data.createRbw(block);
+    BlockWriteStreams streams = null;
+    try {
+      streams = replicaInfo.createStreams(true, 0, 0);
+      streams.checksumOut.write('a');
+      dn.data.initReplicaRecovery(new RecoveringBlock(block, null, RECOVERY_ID+1));
+      try {
+        dn.syncBlock(rBlock, initBlockRecords(dn));
+        fail("Sync should fail");
+      } catch (IOException e) {
+        e.getMessage().startsWith("Cannot recover ");
+      }
+      verify(dn.namenode, never()).commitBlockSynchronization(
+          any(Block.class), anyLong(), anyLong(), anyBoolean(),
+          anyBoolean(), any(DatanodeID[].class));
+    } finally {
+      streams.close();
+    }
+  }
 }