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 co...@apache.org on 2009/11/16 23:38:46 UTC

svn commit: r881017 - in /hadoop/hdfs/trunk: CHANGES.txt src/test/aop/org/apache/hadoop/fi/FiHFlushTestUtil.java src/test/aop/org/apache/hadoop/hdfs/HFlushAspects.aj src/test/aop/org/apache/hadoop/hdfs/TestFiHFlush.java

Author: cos
Date: Mon Nov 16 22:38:46 2009
New Revision: 881017

URL: http://svn.apache.org/viewvc?rev=881017&view=rev
Log:
HDFS-706. Intermittent failures in TestFiHFlush. Contributed by Konstantin Boudnik

Modified:
    hadoop/hdfs/trunk/CHANGES.txt
    hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/fi/FiHFlushTestUtil.java
    hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/HFlushAspects.aj
    hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/TestFiHFlush.java

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=881017&r1=881016&r2=881017&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Mon Nov 16 22:38:46 2009
@@ -61,6 +61,8 @@
     HDFS-741. TestHFlush test doesn't seek() past previously written part of
     the file (cos, szetszwo)
 
+    HDFS-706. Intermittent failures in TestFiHFlush (cos)
+
 Release 0.21.0 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/fi/FiHFlushTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/fi/FiHFlushTestUtil.java?rev=881017&r1=881016&r2=881017&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/fi/FiHFlushTestUtil.java (original)
+++ hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/fi/FiHFlushTestUtil.java Mon Nov 16 22:38:46 2009
@@ -61,5 +61,7 @@
   public static class HFlushTest extends DataTransferTest {
     public final ActionContainer<DatanodeID> fiCallHFlush = 
       new ActionContainer<DatanodeID>();
+    public final ActionContainer<Integer> fiErrorOnCallHFlush = 
+      new ActionContainer<Integer>();
   }
 }
\ No newline at end of file

Modified: hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/HFlushAspects.aj
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/HFlushAspects.aj?rev=881017&r1=881016&r2=881017&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/HFlushAspects.aj (original)
+++ hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/HFlushAspects.aj Mon Nov 16 22:38:46 2009
@@ -54,7 +54,11 @@
     }
     if (pt instanceof HFlushTest)
       for (int i=0; i<nodes.length; i++) {
-        ((HFlushTest)pt).fiCallHFlush.run(nodes[i]);
+        try {
+          ((HFlushTest)pt).fiCallHFlush.run(nodes[i]);
+        } catch (IOException ioe) {
+          ((HFlushTest)pt).fiErrorOnCallHFlush.run(i);
+        }
       }
   }
 }

Modified: hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/TestFiHFlush.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/TestFiHFlush.java?rev=881017&r1=881016&r2=881017&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/TestFiHFlush.java (original)
+++ hadoop/hdfs/trunk/src/test/aop/org/apache/hadoop/hdfs/TestFiHFlush.java Mon Nov 16 22:38:46 2009
@@ -18,15 +18,17 @@
 
 package org.apache.hadoop.hdfs;
 
-import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fi.DataTransferTestUtil;
 import org.apache.hadoop.fi.FiHFlushTestUtil;
-import org.apache.hadoop.fi.FiTestUtil;
 import org.apache.hadoop.fi.FiHFlushTestUtil.DerrAction;
 import org.apache.hadoop.fi.FiHFlushTestUtil.HFlushTest;
+import org.apache.hadoop.fi.FiTestUtil;
+import static org.junit.Assert.assertTrue;
 import org.junit.Test;
 
+import java.io.IOException;
+
 /** Class provides basic fault injection tests according to the test plan
     of HDFS-265
  */
@@ -42,16 +44,22 @@
    * @throws IOException in case of any errors
    */
   private static void runDiskErrorTest (final Configuration conf, 
-      final String methodName, final int block_size, DerrAction a)
+      final String methodName, final int block_size, DerrAction a, int index,
+      boolean trueVerification)
       throws IOException {
     FiTestUtil.LOG.info("Running " + methodName + " ...");
     final HFlushTest hft = (HFlushTest) FiHFlushTestUtil.initTest();
     hft.fiCallHFlush.set(a);
+    hft.fiErrorOnCallHFlush.set(new DataTransferTestUtil.VerificationAction(methodName, index));
     TestHFlush.doTheJob(conf, methodName, block_size, (short)3);
+    if (!trueVerification)
+      assertTrue(!hft.isSuccess());
+    else
+      assertTrue(hft.isSuccess());      
   }
   
   /** The tests calls 
-   * {@link #runDiskErrorTest(Configuration, String, int, DerrAction)}
+   * {@link #runDiskErrorTest(Configuration, String, int, DerrAction, int, boolean)}
    * to make a number of writes within a block boundaries.
    * Although hflush() is called the test shouldn't expect an IOException
    * in this case because the invocation is happening after write() call 
@@ -62,16 +70,16 @@
   public void hFlushFi01_a() throws IOException {
     final String methodName = FiTestUtil.getMethodName();
     runDiskErrorTest(new HdfsConfiguration(), methodName, 
-        AppendTestUtil.BLOCK_SIZE, new DerrAction(methodName, 0));
+        AppendTestUtil.BLOCK_SIZE, new DerrAction(methodName, 0), 0, false);
   }
 
   /** The tests calls 
-   * {@link #runDiskErrorTest(Configuration, String, int, DerrAction)}
+   * {@link #runDiskErrorTest(Configuration, String, int, DerrAction, int, boolean)}
    * to make a number of writes across a block boundaries.
    * hflush() is called after each write() during a pipeline life time.
    * Thus, injected fault ought to be triggered for 0th datanode
    */
-  @Test(expected = IOException.class)
+  @Test
   public void hFlushFi01_b() throws IOException {
     final String methodName = FiTestUtil.getMethodName();
     Configuration conf = new HdfsConfiguration();
@@ -80,14 +88,14 @@
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
     runDiskErrorTest(conf, methodName, 
-        customBlockSize, new DerrAction(methodName, 0));
+        customBlockSize, new DerrAction(methodName, 0), 0, true);
   }
   
   /** Similar to {@link #hFlushFi01_b()} but writing happens
    * across block and checksum's boundaries
    */
-  @Test(expected = IOException.class)
-  public void hFlushFi01_c() throws IOException { 
+  @Test
+  public void hFlushFi01_c() throws Exception { 
     final String methodName = FiTestUtil.getMethodName();
     Configuration conf = new HdfsConfiguration();
     int customPerChecksumSize = 400;
@@ -95,7 +103,7 @@
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
     runDiskErrorTest(conf, methodName, 
-        customBlockSize, new DerrAction(methodName, 0));
+        customBlockSize, new DerrAction(methodName, 0), 0, true);
   }
 
   /** Similar to {@link #hFlushFi01_a()} but for a pipeline's 1st datanode
@@ -104,12 +112,12 @@
   public void hFlushFi02_a() throws IOException {
     final String methodName = FiTestUtil.getMethodName();
     runDiskErrorTest(new HdfsConfiguration(), methodName,
-        AppendTestUtil.BLOCK_SIZE, new DerrAction(methodName, 1));
+        AppendTestUtil.BLOCK_SIZE, new DerrAction(methodName, 1), 1, false);
   }
 
   /** Similar to {@link #hFlushFi01_b()} but for a pipeline's 1st datanode
    */
-@Test(expected = IOException.class)
+  @Test
   public void hFlushFi02_b() throws IOException {
     final String methodName = FiTestUtil.getMethodName();
     Configuration conf = new HdfsConfiguration();
@@ -118,12 +126,12 @@
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
     runDiskErrorTest(conf, methodName,
-        customBlockSize, new DerrAction(methodName, 1));
+        customBlockSize, new DerrAction(methodName, 1), 1, true);
   }
 
   /** Similar to {@link #hFlushFi01_c()} but for a pipeline's 1st datanode
    */
-  @Test(expected = IOException.class)
+  @Test
   public void hFlushFi02_c() throws IOException {
     final String methodName = FiTestUtil.getMethodName();
     Configuration conf = new HdfsConfiguration();
@@ -132,7 +140,7 @@
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
     runDiskErrorTest(conf, methodName,
-        customBlockSize, new DerrAction(methodName, 1));
+        customBlockSize, new DerrAction(methodName, 1), 1, true);
   }
   
   /** Similar to {@link #hFlushFi01_a()} but for a pipeline's 2nd datanode
@@ -141,12 +149,12 @@
   public void hFlushFi03_a() throws IOException {
     final String methodName = FiTestUtil.getMethodName();
     runDiskErrorTest(new HdfsConfiguration(), methodName,
-        AppendTestUtil.BLOCK_SIZE, new DerrAction(methodName, 2));
+        AppendTestUtil.BLOCK_SIZE, new DerrAction(methodName, 2), 2, false);
   }
   
   /** Similar to {@link #hFlushFi01_b()} but for a pipeline's 2nd datanode
    */
-  @Test(expected = IOException.class)
+  @Test
   public void hFlushFi03_b() throws IOException {
     final String methodName = FiTestUtil.getMethodName();
     Configuration conf = new HdfsConfiguration();
@@ -155,12 +163,12 @@
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
     runDiskErrorTest(conf, methodName,
-        customBlockSize, new DerrAction(methodName, 2));
+        customBlockSize, new DerrAction(methodName, 2), 2, true);
   }
 
   /** Similar to {@link #hFlushFi01_c()} but for a pipeline's 2nd datanode
    */
-  @Test(expected = IOException.class)
+  @Test
   public void hFlushFi03_c() throws IOException {
     final String methodName = FiTestUtil.getMethodName();
     Configuration conf = new HdfsConfiguration();
@@ -169,6 +177,6 @@
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
     runDiskErrorTest(conf, methodName,
-        customBlockSize, new DerrAction(methodName, 2));
+        customBlockSize, new DerrAction(methodName, 2), 2, true);
   }
 }