|
@@ -25,6 +25,8 @@ import org.junit.BeforeClass;
|
|
import org.junit.AfterClass;
|
|
import org.junit.AfterClass;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.net.URI;
|
|
|
|
+
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
|
|
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
|
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
|
@@ -34,7 +36,9 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogTestUtil;
|
|
import org.apache.hadoop.hdfs.server.namenode.JournalManager;
|
|
import org.apache.hadoop.hdfs.server.namenode.JournalManager;
|
|
|
|
|
|
import org.apache.bookkeeper.proto.BookieServer;
|
|
import org.apache.bookkeeper.proto.BookieServer;
|
|
|
|
+import org.apache.zookeeper.CreateMode;
|
|
import org.apache.zookeeper.ZooKeeper;
|
|
import org.apache.zookeeper.ZooKeeper;
|
|
|
|
+import org.apache.zookeeper.ZooDefs.Ids;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -74,7 +78,6 @@ public class TestBookKeeperJournalManager {
|
|
public void testSimpleWrite() throws Exception {
|
|
public void testSimpleWrite() throws Exception {
|
|
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
|
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
|
BKJMUtil.createJournalURI("/hdfsjournal-simplewrite"));
|
|
BKJMUtil.createJournalURI("/hdfsjournal-simplewrite"));
|
|
- long txid = 1;
|
|
|
|
EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
for (long i = 1 ; i <= 100; i++) {
|
|
for (long i = 1 ; i <= 100; i++) {
|
|
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
@@ -94,7 +97,6 @@ public class TestBookKeeperJournalManager {
|
|
public void testNumberOfTransactions() throws Exception {
|
|
public void testNumberOfTransactions() throws Exception {
|
|
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
|
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
|
BKJMUtil.createJournalURI("/hdfsjournal-txncount"));
|
|
BKJMUtil.createJournalURI("/hdfsjournal-txncount"));
|
|
- long txid = 1;
|
|
|
|
EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
for (long i = 1 ; i <= 100; i++) {
|
|
for (long i = 1 ; i <= 100; i++) {
|
|
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
@@ -246,10 +248,12 @@ public class TestBookKeeperJournalManager {
|
|
|
|
|
|
EditLogOutputStream out1 = bkjm1.startLogSegment(start);
|
|
EditLogOutputStream out1 = bkjm1.startLogSegment(start);
|
|
try {
|
|
try {
|
|
- EditLogOutputStream out2 = bkjm2.startLogSegment(start);
|
|
|
|
|
|
+ bkjm2.startLogSegment(start);
|
|
fail("Shouldn't have been able to open the second writer");
|
|
fail("Shouldn't have been able to open the second writer");
|
|
} catch (IOException ioe) {
|
|
} catch (IOException ioe) {
|
|
LOG.info("Caught exception as expected", ioe);
|
|
LOG.info("Caught exception as expected", ioe);
|
|
|
|
+ }finally{
|
|
|
|
+ out1.close();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -257,7 +261,6 @@ public class TestBookKeeperJournalManager {
|
|
public void testSimpleRead() throws Exception {
|
|
public void testSimpleRead() throws Exception {
|
|
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
|
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
|
BKJMUtil.createJournalURI("/hdfsjournal-simpleread"));
|
|
BKJMUtil.createJournalURI("/hdfsjournal-simpleread"));
|
|
- long txid = 1;
|
|
|
|
final long numTransactions = 10000;
|
|
final long numTransactions = 10000;
|
|
EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
for (long i = 1 ; i <= numTransactions; i++) {
|
|
for (long i = 1 ; i <= numTransactions; i++) {
|
|
@@ -283,7 +286,6 @@ public class TestBookKeeperJournalManager {
|
|
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
|
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
|
BKJMUtil.createJournalURI("/hdfsjournal-simplerecovery"));
|
|
BKJMUtil.createJournalURI("/hdfsjournal-simplerecovery"));
|
|
EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
- long txid = 1;
|
|
|
|
for (long i = 1 ; i <= 100; i++) {
|
|
for (long i = 1 ; i <= 100; i++) {
|
|
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
op.setTransactionId(i);
|
|
op.setTransactionId(i);
|
|
@@ -449,5 +451,167 @@ public class TestBookKeeperJournalManager {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * If a journal manager has an empty inprogress node, ensure that we throw an
|
|
|
|
+ * error, as this should not be possible, and some third party has corrupted
|
|
|
|
+ * the zookeeper state
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testEmptyInprogressNode() throws Exception {
|
|
|
|
+ URI uri = BKJMUtil.createJournalURI("/hdfsjournal-emptyInprogress");
|
|
|
|
+ BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri);
|
|
|
|
+
|
|
|
|
+ EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
|
|
+ for (long i = 1; i <= 100; i++) {
|
|
|
|
+ FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
|
|
+ op.setTransactionId(i);
|
|
|
|
+ out.write(op);
|
|
|
|
+ }
|
|
|
|
+ out.close();
|
|
|
|
+ bkjm.finalizeLogSegment(1, 100);
|
|
|
|
+
|
|
|
|
+ out = bkjm.startLogSegment(101);
|
|
|
|
+ out.close();
|
|
|
|
+ bkjm.close();
|
|
|
|
+ String inprogressZNode = bkjm.inprogressZNode(101);
|
|
|
|
+ zkc.setData(inprogressZNode, new byte[0], -1);
|
|
|
|
+
|
|
|
|
+ bkjm = new BookKeeperJournalManager(conf, uri);
|
|
|
|
+ try {
|
|
|
|
+ bkjm.recoverUnfinalizedSegments();
|
|
|
|
+ fail("Should have failed. There should be no way of creating"
|
|
|
|
+ + " an empty inprogess znode");
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ // correct behaviour
|
|
|
|
+ assertTrue("Exception different than expected", e.getMessage().contains(
|
|
|
|
+ "Invalid ledger entry,"));
|
|
|
|
+ } finally {
|
|
|
|
+ bkjm.close();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * If a journal manager has an corrupt inprogress node, ensure that we throw
|
|
|
|
+ * an error, as this should not be possible, and some third party has
|
|
|
|
+ * corrupted the zookeeper state
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testCorruptInprogressNode() throws Exception {
|
|
|
|
+ URI uri = BKJMUtil.createJournalURI("/hdfsjournal-corruptInprogress");
|
|
|
|
+ BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri);
|
|
|
|
+
|
|
|
|
+ EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
|
|
+ for (long i = 1; i <= 100; i++) {
|
|
|
|
+ FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
|
|
+ op.setTransactionId(i);
|
|
|
|
+ out.write(op);
|
|
|
|
+ }
|
|
|
|
+ out.close();
|
|
|
|
+ bkjm.finalizeLogSegment(1, 100);
|
|
|
|
+
|
|
|
|
+ out = bkjm.startLogSegment(101);
|
|
|
|
+ out.close();
|
|
|
|
+ bkjm.close();
|
|
|
|
+
|
|
|
|
+ String inprogressZNode = bkjm.inprogressZNode(101);
|
|
|
|
+ zkc.setData(inprogressZNode, "WholeLottaJunk".getBytes(), -1);
|
|
|
|
+
|
|
|
|
+ bkjm = new BookKeeperJournalManager(conf, uri);
|
|
|
|
+ try {
|
|
|
|
+ bkjm.recoverUnfinalizedSegments();
|
|
|
|
+ fail("Should have failed. There should be no way of creating"
|
|
|
|
+ + " an empty inprogess znode");
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ // correct behaviour
|
|
|
|
+ assertTrue("Exception different than expected", e.getMessage().contains(
|
|
|
|
+ "Invalid ledger entry,"));
|
|
|
|
+
|
|
|
|
+ } finally {
|
|
|
|
+ bkjm.close();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Cases can occur where we create a segment but crash before we even have the
|
|
|
|
+ * chance to write the START_SEGMENT op. If this occurs we should warn, but
|
|
|
|
+ * load as normal
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testEmptyInprogressLedger() throws Exception {
|
|
|
|
+ URI uri = BKJMUtil.createJournalURI("/hdfsjournal-emptyInprogressLedger");
|
|
|
|
+ BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri);
|
|
|
|
+
|
|
|
|
+ EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
|
|
+ for (long i = 1; i <= 100; i++) {
|
|
|
|
+ FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
|
|
+ op.setTransactionId(i);
|
|
|
|
+ out.write(op);
|
|
|
|
+ }
|
|
|
|
+ out.close();
|
|
|
|
+ bkjm.finalizeLogSegment(1, 100);
|
|
|
|
+
|
|
|
|
+ out = bkjm.startLogSegment(101);
|
|
|
|
+ out.close();
|
|
|
|
+ bkjm.close();
|
|
|
|
+
|
|
|
|
+ bkjm = new BookKeeperJournalManager(conf, uri);
|
|
|
|
+ bkjm.recoverUnfinalizedSegments();
|
|
|
|
+ out = bkjm.startLogSegment(101);
|
|
|
|
+ for (long i = 1; i <= 100; i++) {
|
|
|
|
+ FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
|
|
+ op.setTransactionId(i);
|
|
|
|
+ out.write(op);
|
|
|
|
+ }
|
|
|
|
+ out.close();
|
|
|
|
+ bkjm.finalizeLogSegment(101, 200);
|
|
|
|
+
|
|
|
|
+ bkjm.close();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test that if we fail between finalizing an inprogress and deleting the
|
|
|
|
+ * corresponding inprogress znode.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testRefinalizeAlreadyFinalizedInprogress() throws Exception {
|
|
|
|
+ URI uri = BKJMUtil
|
|
|
|
+ .createJournalURI("/hdfsjournal-refinalizeInprogressLedger");
|
|
|
|
+ BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri);
|
|
|
|
+
|
|
|
|
+ EditLogOutputStream out = bkjm.startLogSegment(1);
|
|
|
|
+ for (long i = 1; i <= 100; i++) {
|
|
|
|
+ FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
|
|
|
+ op.setTransactionId(i);
|
|
|
|
+ out.write(op);
|
|
|
|
+ }
|
|
|
|
+ out.close();
|
|
|
|
+ bkjm.close();
|
|
|
|
+
|
|
|
|
+ String inprogressZNode = bkjm.inprogressZNode(1);
|
|
|
|
+ String finalizedZNode = bkjm.finalizedLedgerZNode(1, 100);
|
|
|
|
+ assertNotNull("inprogress znode doesn't exist", zkc.exists(inprogressZNode,
|
|
|
|
+ null));
|
|
|
|
+ assertNull("finalized znode exists", zkc.exists(finalizedZNode, null));
|
|
|
|
+
|
|
|
|
+ byte[] inprogressData = zkc.getData(inprogressZNode, false, null);
|
|
|
|
+
|
|
|
|
+ // finalize
|
|
|
|
+ bkjm = new BookKeeperJournalManager(conf, uri);
|
|
|
|
+ bkjm.recoverUnfinalizedSegments();
|
|
|
|
+ bkjm.close();
|
|
|
|
+
|
|
|
|
+ assertNull("inprogress znode exists", zkc.exists(inprogressZNode, null));
|
|
|
|
+ assertNotNull("finalized znode doesn't exist", zkc.exists(finalizedZNode,
|
|
|
|
+ null));
|
|
|
|
+
|
|
|
|
+ zkc.create(inprogressZNode, inprogressData, Ids.OPEN_ACL_UNSAFE,
|
|
|
|
+ CreateMode.PERSISTENT);
|
|
|
|
+
|
|
|
|
+ // should work fine
|
|
|
|
+ bkjm = new BookKeeperJournalManager(conf, uri);
|
|
|
|
+ bkjm.recoverUnfinalizedSegments();
|
|
|
|
+ bkjm.close();
|
|
|
|
+ }
|
|
|
|
|
|
}
|
|
}
|