Author: phunt
Date: Mon Feb 10 20:52:25 2014
New Revision: 1566740
URL: http://svn.apache.org/r1566740
Log:
ZOOKEEPER-1573. Unable to load database due to missing parent node (Vinayakumar B via phunt,
fpj)
Modified:
zookeeper/branches/branch-3.4/CHANGES.txt
zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java
Modified: zookeeper/branches/branch-3.4/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/CHANGES.txt?rev=1566740&r1=1566739&r2=1566740&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/CHANGES.txt (original)
+++ zookeeper/branches/branch-3.4/CHANGES.txt Mon Feb 10 20:52:25 2014
@@ -235,6 +235,9 @@ BUGFIXES:
ZOOKEEPER-1872. QuorumPeer is not shutdown in few cases
(Rakesh R via fpj)
+ ZOOKEEPER-1573. Unable to load database due to missing parent node
+ (Vinayakumar B via phunt, fpj)
+
IMPROVEMENTS:
ZOOKEEPER-1564. Allow JUnit test build with IBM Java
Modified: zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java?rev=1566740&r1=1566739&r2=1566740&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
(original)
+++ zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
Mon Feb 10 20:52:25 2014
@@ -199,23 +199,17 @@ public class FileTxnSnapLog {
}
/**
- * This should never happen. A NONODE can never show up in the
- * transaction logs. This is more indicative of a corrupt transaction
- * log. Refer ZOOKEEPER-1333 for more info.
+ * Snapshots are lazily created. So when a snapshot is in progress,
+ * there is a chance for later transactions to make into the
+ * snapshot. Then when the snapshot is restored, NONODE/NODEEXISTS
+ * errors could occur. It should be safe to ignore these.
*/
- if (rc.err != Code.OK.intValue()) {
- if (hdr.getType() == OpCode.create && rc.err == Code.NONODE.intValue())
{
- int lastSlash = rc.path.lastIndexOf('/');
- String parentName = rc.path.substring(0, lastSlash);
- LOG.error("Parent {} missing for {}", parentName, rc.path);
- throw new KeeperException.NoNodeException(parentName);
- } else {
- LOG.debug("Ignoring processTxn failure hdr: " + hdr.getType() +
- " : error: " + rc.err);
- }
+ if (rc.err != Code.OK.intValue()) {
+ LOG.debug("Ignoring processTxn failure hdr:" + hdr.getType()
+ + ", error: " + rc.err + ", path: " + rc.path);
}
}
-
+
/**
* the last logged zxid on the transaction logs
* @return the last logged zxid
Modified: zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java?rev=1566740&r1=1566739&r2=1566740&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java
(original)
+++ zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java
Mon Feb 10 20:52:25 2014
@@ -21,6 +21,7 @@ package org.apache.zookeeper.test;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileInputStream;
+import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
@@ -37,6 +38,7 @@ import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooDefs.OpCode;
import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.server.DataNode;
import org.apache.zookeeper.server.DataTree;
import org.apache.zookeeper.server.ServerCnxnFactory;
@@ -278,22 +280,7 @@ public class LoadFromLogTest extends ZKT
f.startup(zks);
Assert.assertTrue("waiting for server being up ", ClientBase
.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
- ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
-
- long start = System.currentTimeMillis();
- while (!connected) {
- long end = System.currentTimeMillis();
- if (end - start > 5000) {
- Assert.assertTrue("Could not connect with server in 5 seconds",
- false);
- }
- try {
- Thread.sleep(200);
- } catch (Exception e) {
- LOG.warn("Intrrupted");
- }
-
- }
+ ZooKeeper zk = getConnectedZkClient();
// generate some transactions
String lastPath = null;
try {
@@ -333,21 +320,7 @@ public class LoadFromLogTest extends ZKT
// Verify lastProcessedZxid is set correctly
Assert.assertTrue("Restore failed expected zxid=" + eZxid + " found="
+ fZxid, fZxid == eZxid);
- zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
- start = System.currentTimeMillis();
- while (!connected) {
- long end = System.currentTimeMillis();
- if (end - start > 5000) {
- Assert.assertTrue("Could not connect with server in 5 seconds",
- false);
- }
- try {
- Thread.sleep(200);
- } catch (Exception e) {
- LOG.warn("Intrrupted");
- }
-
- }
+ zk = getConnectedZkClient();
// Verify correctness of data and whether sequential znode creation
// proceeds correctly after this point
String[] children;
@@ -386,22 +359,7 @@ public class LoadFromLogTest extends ZKT
f.startup(zks);
Assert.assertTrue("waiting for server being up ", ClientBase
.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
- ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
-
- long start = System.currentTimeMillis();
- while (!connected) {
- long end = System.currentTimeMillis();
- if (end - start > 5000) {
- Assert.assertTrue("Could not connect with server in 5 seconds",
- false);
- }
- try {
- Thread.sleep(200);
- } catch (Exception e) {
- LOG.warn("Intrrupted");
- }
-
- }
+ ZooKeeper zk = getConnectedZkClient();
// generate some transactions
try {
for (int i = 0; i < NUM_MESSAGES; i++) {
@@ -437,4 +395,68 @@ public class LoadFromLogTest extends ZKT
f.shutdown();
zks.shutdown();
}
+
+ /**
+ * ZOOKEEPER-1573: test restoring a snapshot with deleted txns ahead of the
+ * snapshot file's zxid.
+ */
+ @Test
+ public void testReloadSnapshotWithMissingParent() throws Exception {
+ // setup a single server cluster
+ File tmpDir = ClientBase.createTmpDir();
+ ClientBase.setupTestEnv();
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ SyncRequestProcessor.setSnapCount(10000);
+ final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
+ ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
+ f.startup(zks);
+ Assert.assertTrue("waiting for server being up ",
+ ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
+ ZooKeeper zk = getConnectedZkClient();
+
+ // create transactions to create the snapshot with create/delete pattern
+ zk.create("/a", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ Stat stat = zk.exists("/a", false);
+ long createZxId = stat.getMzxid();
+ zk.create("/a/b", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ zk.delete("/a/b", -1);
+ zk.delete("/a", -1);
+ // force the zxid to be behind the content
+ zks.getZKDatabase().setlastProcessedZxid(createZxId);
+ LOG.info("Set lastProcessedZxid to {}", zks.getZKDatabase()
+ .getDataTreeLastProcessedZxid());
+ // Force snapshot and restore
+ zks.takeSnapshot();
+ zks.shutdown();
+ f.shutdown();
+
+ zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ SyncRequestProcessor.setSnapCount(10000);
+ f = ServerCnxnFactory.createFactory(PORT, -1);
+ f.startup(zks);
+ Assert.assertTrue("waiting for server being up ",
+ ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
+ f.shutdown();
+ }
+
+ private ZooKeeper getConnectedZkClient() throws IOException {
+ ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
+
+ long start = System.currentTimeMillis();
+ while (!connected) {
+ long end = System.currentTimeMillis();
+ if (end - start > 5000) {
+ Assert.assertTrue("Could not connect with server in 5 seconds",
+ false);
+ }
+ try {
+ Thread.sleep(200);
+ } catch (Exception e) {
+ LOG.warn("Interrupted");
+ }
+ }
+ return zk;
+ }
}
\ No newline at end of file
|