Explorar o código

HDFS-4004. TestJournalNode#testJournal fails because of test case execution order. Contributed by Chao Shi.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3077@1393870 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon %!s(int64=12) %!d(string=hai) anos
pai
achega
10a0eaaf0a

+ 11 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java

@@ -25,6 +25,7 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
@@ -48,6 +49,8 @@ import com.google.common.collect.Sets;
  */
 public abstract class GenericTestUtils {
 
+  private static final AtomicInteger sequence = new AtomicInteger();
+
   /**
    * Extracts the name of the method where the invocation has happened
    * @return String name of the invoking method
@@ -55,6 +58,14 @@ public abstract class GenericTestUtils {
   public static String getMethodName() {
     return Thread.currentThread().getStackTrace()[2].getMethodName();
   }
+
+  /**
+   * Generates a process-wide unique sequence number.
+   * @return an unique sequence number
+   */
+  public static int uniqueSequenceId() {
+    return sequence.incrementAndGet();
+  }
   
   /**
    * Assert that a given file exists.

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt

@@ -82,3 +82,5 @@ HDFS-3950. QJM: misc TODO cleanup, improved log messages, etc. (todd)
 HDFS-3955. QJM: Make acceptRecovery() atomic. (todd)
 
 HDFS-3956. QJM: purge temporary files when no longer within retention period (todd)
+
+HDFS-4004. TestJournalNode#testJournal fails because of test case execution order (Chao Shi via todd)

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -59,12 +59,12 @@ import com.google.common.primitives.Ints;
 public class TestJournalNode {
   private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
       12345, "mycluster", "my-bp", 0L);
-  private static final String JID = "test-journalid";
 
   private JournalNode jn;
   private Journal journal; 
   private Configuration conf = new Configuration();
   private IPCLoggerChannel ch;
+  private String journalId;
 
   static {
     // Avoid an error when we double-initialize JvmMetrics
@@ -84,10 +84,11 @@ public class TestJournalNode {
     jn = new JournalNode();
     jn.setConf(conf);
     jn.start();
-    journal = jn.getOrCreateJournal(JID);
+    journalId = "test-journalid-" + GenericTestUtils.uniqueSequenceId();
+    journal = jn.getOrCreateJournal(journalId);
     journal.format(FAKE_NSINFO);
     
-    ch = new IPCLoggerChannel(conf, FAKE_NSINFO, JID, jn.getBoundIpcAddress());
+    ch = new IPCLoggerChannel(conf, FAKE_NSINFO, journalId, jn.getBoundIpcAddress());
   }
   
   @After
@@ -104,7 +105,7 @@ public class TestJournalNode {
     MetricsAsserts.assertGauge("CurrentLagTxns", 0L, metrics);
 
     IPCLoggerChannel ch = new IPCLoggerChannel(
-        conf, FAKE_NSINFO, JID, jn.getBoundIpcAddress());
+        conf, FAKE_NSINFO, journalId, jn.getBoundIpcAddress());
     ch.newEpoch(1).get();
     ch.setEpoch(1);
     ch.startLogSegment(1).get();
@@ -177,7 +178,7 @@ public class TestJournalNode {
     // Create some edits on server side
     byte[] EDITS_DATA = QJMTestUtil.createTxnData(1, 3);
     IPCLoggerChannel ch = new IPCLoggerChannel(
-        conf, FAKE_NSINFO, JID, jn.getBoundIpcAddress());
+        conf, FAKE_NSINFO, journalId, jn.getBoundIpcAddress());
     ch.newEpoch(1).get();
     ch.setEpoch(1);
     ch.startLogSegment(1).get();
@@ -187,7 +188,7 @@ public class TestJournalNode {
     // Attempt to retrieve via HTTP, ensure we get the data back
     // including the header we expected
     byte[] retrievedViaHttp = DFSTestUtil.urlGetBytes(new URL(urlRoot +
-        "/getJournal?segmentTxId=1&jid=" + JID));
+        "/getJournal?segmentTxId=1&jid=" + journalId));
     byte[] expected = Bytes.concat(
             Ints.toByteArray(HdfsConstants.LAYOUT_VERSION),
             EDITS_DATA);
@@ -196,7 +197,7 @@ public class TestJournalNode {
     
     // Attempt to fetch a non-existent file, check that we get an
     // error status code
-    URL badUrl = new URL(urlRoot + "/getJournal?segmentTxId=12345&jid=" + JID);
+    URL badUrl = new URL(urlRoot + "/getJournal?segmentTxId=12345&jid=" + journalId);
     HttpURLConnection connection = (HttpURLConnection)badUrl.openConnection();
     try {
       assertEquals(404, connection.getResponseCode());