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 to...@apache.org on 2012/10/04 03:40:12 UTC
svn commit: r1393870 - in
/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs:
CHANGES.HDFS-3077.txt
src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
Author: todd
Date: Thu Oct 4 01:40:12 2012
New Revision: 1393870
URL: http://svn.apache.org/viewvc?rev=1393870&view=rev
Log:
HDFS-4004. TestJournalNode#testJournal fails because of test case execution order. Contributed by Chao Shi.
Modified:
hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt
hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt?rev=1393870&r1=1393869&r2=1393870&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt Thu Oct 4 01:40:12 2012
@@ -82,3 +82,5 @@ HDFS-3950. QJM: misc TODO cleanup, impro
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)
Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java?rev=1393870&r1=1393869&r2=1393870&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java Thu Oct 4 01:40:12 2012
@@ -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());