|
@@ -29,6 +29,7 @@ import java.util.concurrent.CountDownLatch;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
+import org.apache.zookeeper.data.Stat;
|
|
|
|
|
|
import org.apache.zookeeper.CreateMode;
|
|
|
import org.apache.zookeeper.PortAssignment;
|
|
@@ -40,6 +41,7 @@ import org.apache.zookeeper.server.ServerCnxnFactory;
|
|
|
import org.apache.zookeeper.server.SyncRequestProcessor;
|
|
|
import org.apache.zookeeper.server.ZooKeeperServer;
|
|
|
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
|
|
|
+import org.apache.zookeeper.server.persistence.Util;
|
|
|
import org.apache.zookeeper.test.ClientBase;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
@@ -172,10 +174,17 @@ public class PurgeTxnTest extends ZKTestCase {
|
|
|
int nRecentSnap = 4; // n recent snap shots
|
|
|
int nRecentCount = 30;
|
|
|
int offset = 0;
|
|
|
+
|
|
|
tmpDir = ClientBase.createTmpDir();
|
|
|
File version2 = new File(tmpDir.toString(), "version-2");
|
|
|
Assert.assertTrue("Failed to create version_2 dir:" + version2.toString(),
|
|
|
version2.mkdir());
|
|
|
+
|
|
|
+ // Test that with no snaps, findNRecentSnapshots returns empty list
|
|
|
+ FileTxnSnapLog txnLog = new FileTxnSnapLog(tmpDir, tmpDir);
|
|
|
+ List<File> foundSnaps = txnLog.findNRecentSnapshots(1);
|
|
|
+ assertEquals(0, foundSnaps.size());
|
|
|
+
|
|
|
List<File> expectedNRecentSnapFiles = new ArrayList<File>();
|
|
|
int counter = offset + (2 * nRecentCount);
|
|
|
for (int i = 0; i < nRecentCount; i++) {
|
|
@@ -194,14 +203,25 @@ public class PurgeTxnTest extends ZKTestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- FileTxnSnapLog txnLog = new FileTxnSnapLog(tmpDir, tmpDir);
|
|
|
+ // Test that when we ask for recent snaps we get the number we asked for and
|
|
|
+ // the files we expected
|
|
|
List<File> nRecentSnapFiles = txnLog.findNRecentSnapshots(nRecentSnap);
|
|
|
- txnLog.close();
|
|
|
Assert.assertEquals("exactly 4 snapshots ", 4,
|
|
|
nRecentSnapFiles.size());
|
|
|
expectedNRecentSnapFiles.removeAll(nRecentSnapFiles);
|
|
|
Assert.assertEquals("Didn't get the recent snap files", 0,
|
|
|
expectedNRecentSnapFiles.size());
|
|
|
+
|
|
|
+ // Test that when asking for more snaps than we created, we still only get snaps
|
|
|
+ // not logs or anything else (per ZOOKEEPER-2420)
|
|
|
+ nRecentSnapFiles = txnLog.findNRecentSnapshots(nRecentCount + 5);
|
|
|
+ assertEquals(nRecentCount, nRecentSnapFiles.size());
|
|
|
+ for (File f: nRecentSnapFiles) {
|
|
|
+ Assert.assertTrue("findNRecentSnapshots() returned a non-snapshot: " + f.getPath(),
|
|
|
+ (Util.getZxidFromName(f.getName(), "snapshot") != -1));
|
|
|
+ }
|
|
|
+
|
|
|
+ txnLog.close();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -225,14 +245,21 @@ public class PurgeTxnTest extends ZKTestCase {
|
|
|
List<File> logs = new ArrayList<File>();
|
|
|
List<File> snapsAboveRecentFiles = new ArrayList<File>();
|
|
|
List<File> logsAboveRecentFiles = new ArrayList<File>();
|
|
|
- createDataDirFiles(offset, fileToPurgeCount, version2, snapsToPurge,
|
|
|
+ createDataDirFiles(offset, fileToPurgeCount, false, version2, snapsToPurge,
|
|
|
logsToPurge);
|
|
|
- createDataDirFiles(offset, nRecentCount, version2, snaps, logs);
|
|
|
- createDataDirFiles(offset, fileAboveRecentCount, version2,
|
|
|
+ createDataDirFiles(offset, nRecentCount, false, version2, snaps, logs);
|
|
|
+ logs.add(logsToPurge.remove(0)); // log that precedes first retained snapshot is also retained
|
|
|
+ createDataDirFiles(offset, fileAboveRecentCount, false, version2,
|
|
|
snapsAboveRecentFiles, logsAboveRecentFiles);
|
|
|
|
|
|
+ /**
|
|
|
+ * The newest log file preceding the oldest retained snapshot is not removed as it may
|
|
|
+ * contain transactions newer than the oldest snapshot.
|
|
|
+ */
|
|
|
+ logsToPurge.remove(0);
|
|
|
+
|
|
|
FileTxnSnapLog txnLog = new FileTxnSnapLog(tmpDir, tmpDir);
|
|
|
- PurgeTxnLog.retainNRecentSnapshots(txnLog, snaps);
|
|
|
+ PurgeTxnLog.purgeOlderSnapshots(txnLog, snaps.get(snaps.size() - 1));
|
|
|
txnLog.close();
|
|
|
verifyFilesAfterPurge(snapsToPurge, false);
|
|
|
verifyFilesAfterPurge(logsToPurge, false);
|
|
@@ -243,11 +270,24 @@ public class PurgeTxnTest extends ZKTestCase {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Tests purge where the data directory contains snap files equals to the
|
|
|
+ * Tests purge where the data directory contains snap files and log files equals to the
|
|
|
* number of files to be retained
|
|
|
*/
|
|
|
@Test
|
|
|
public void testSnapFilesEqualsToRetain() throws Exception {
|
|
|
+ internalTestSnapFilesEqualsToRetain(false);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests purge where the data directory contains snap files equals to the
|
|
|
+ * number of files to be retained, and a log file that precedes the earliest snapshot
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testSnapFilesEqualsToRetainWithPrecedingLog() throws Exception {
|
|
|
+ internalTestSnapFilesEqualsToRetain(true);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void internalTestSnapFilesEqualsToRetain(boolean testWithPrecedingLogFile) throws Exception {
|
|
|
int nRecentCount = 3;
|
|
|
AtomicInteger offset = new AtomicInteger(0);
|
|
|
tmpDir = ClientBase.createTmpDir();
|
|
@@ -256,10 +296,10 @@ public class PurgeTxnTest extends ZKTestCase {
|
|
|
version2.mkdir());
|
|
|
List<File> snaps = new ArrayList<File>();
|
|
|
List<File> logs = new ArrayList<File>();
|
|
|
- createDataDirFiles(offset, nRecentCount, version2, snaps, logs);
|
|
|
+ createDataDirFiles(offset, nRecentCount, testWithPrecedingLogFile, version2, snaps, logs);
|
|
|
|
|
|
FileTxnSnapLog txnLog = new FileTxnSnapLog(tmpDir, tmpDir);
|
|
|
- PurgeTxnLog.retainNRecentSnapshots(txnLog, snaps);
|
|
|
+ PurgeTxnLog.purgeOlderSnapshots(txnLog, snaps.get(snaps.size() - 1));
|
|
|
txnLog.close();
|
|
|
verifyFilesAfterPurge(snaps, true);
|
|
|
verifyFilesAfterPurge(logs, true);
|
|
@@ -282,12 +322,19 @@ public class PurgeTxnTest extends ZKTestCase {
|
|
|
List<File> logsToPurge = new ArrayList<File>();
|
|
|
List<File> snaps = new ArrayList<File>();
|
|
|
List<File> logs = new ArrayList<File>();
|
|
|
- createDataDirFiles(offset, fileToPurgeCount, version2, snapsToPurge,
|
|
|
+ createDataDirFiles(offset, fileToPurgeCount, false, version2, snapsToPurge,
|
|
|
logsToPurge);
|
|
|
- createDataDirFiles(offset, nRecentCount, version2, snaps, logs);
|
|
|
+ createDataDirFiles(offset, nRecentCount, false, version2, snaps, logs);
|
|
|
+ logs.add(logsToPurge.remove(0)); // log that precedes first retained snapshot is also retained
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The newest log file preceding the oldest retained snapshot is not removed as it may
|
|
|
+ * contain transactions newer than the oldest snapshot.
|
|
|
+ */
|
|
|
+ logsToPurge.remove(0);
|
|
|
|
|
|
FileTxnSnapLog txnLog = new FileTxnSnapLog(tmpDir, tmpDir);
|
|
|
- PurgeTxnLog.retainNRecentSnapshots(txnLog, snaps);
|
|
|
+ PurgeTxnLog.purgeOlderSnapshots(txnLog, snaps.get(snaps.size() - 1));
|
|
|
txnLog.close();
|
|
|
verifyFilesAfterPurge(snapsToPurge, false);
|
|
|
verifyFilesAfterPurge(logsToPurge, false);
|
|
@@ -327,15 +374,16 @@ public class PurgeTxnTest extends ZKTestCase {
|
|
|
snapFile.createNewFile();
|
|
|
}
|
|
|
|
|
|
- int numberOfFilesToKeep = 10;
|
|
|
+ int numberOfSnapFilesToKeep = 10;
|
|
|
// scenario where four parameter are passed
|
|
|
String[] args = new String[] { dataLogDir.getAbsolutePath(),
|
|
|
dataDir.getAbsolutePath(), "-n",
|
|
|
- Integer.toString(numberOfFilesToKeep) };
|
|
|
+ Integer.toString(numberOfSnapFilesToKeep) };
|
|
|
PurgeTxnLog.main(args);
|
|
|
|
|
|
- assertEquals(numberOfFilesToKeep, dataDirVersion2.listFiles().length);
|
|
|
- assertEquals(numberOfFilesToKeep, dataLogDirVersion2.listFiles().length);
|
|
|
+ assertEquals(numberOfSnapFilesToKeep, dataDirVersion2.listFiles().length);
|
|
|
+ // Since for each snapshot we have a log file with same zxid, expect same # logs as snaps to be kept
|
|
|
+ assertEquals(numberOfSnapFilesToKeep, dataLogDirVersion2.listFiles().length);
|
|
|
ClientBase.recursiveDelete(tmpDir);
|
|
|
|
|
|
}
|
|
@@ -371,28 +419,121 @@ public class PurgeTxnTest extends ZKTestCase {
|
|
|
snapFile.createNewFile();
|
|
|
}
|
|
|
|
|
|
- int numberOfFilesToKeep = 10;
|
|
|
+ int numberOfSnapFilesToKeep = 10;
|
|
|
// scenario where only three parameter are passed
|
|
|
String[] args = new String[] { dataLogDir.getAbsolutePath(), "-n",
|
|
|
- Integer.toString(numberOfFilesToKeep) };
|
|
|
+ Integer.toString(numberOfSnapFilesToKeep) };
|
|
|
PurgeTxnLog.main(args);
|
|
|
- assertEquals(numberOfFilesToKeep + numberOfFilesToKeep,
|
|
|
+ assertEquals(numberOfSnapFilesToKeep * 2, // Since for each snapshot we have a log file with same zxid, expect same # logs as snaps to be kept
|
|
|
dataLogDirVersion2.listFiles().length);
|
|
|
ClientBase.recursiveDelete(tmpDir);
|
|
|
|
|
|
}
|
|
|
|
|
|
- private void createDataDirFiles(AtomicInteger offset, int limit,
|
|
|
+ /**
|
|
|
+ * Verifies that purge does not delete any log files which started before the oldest retained
|
|
|
+ * snapshot but which might extend beyond it.
|
|
|
+ * @throws Exception an exception might be thrown here
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testPurgeDoesNotDeleteOverlappingLogFile() throws Exception {
|
|
|
+ // Setting used for snapRetainCount in this test.
|
|
|
+ final int SNAP_RETAIN_COUNT = 3;
|
|
|
+ // Number of znodes this test creates in each snapshot.
|
|
|
+ final int NUM_ZNODES_PER_SNAPSHOT = 100;
|
|
|
+ /**
|
|
|
+ * Set a sufficiently high snapCount to ensure that we don't rollover the log. Normally,
|
|
|
+ * the default value (100K at time of this writing) would ensure this, but we make that
|
|
|
+ * dependence explicit here to make the test future-proof. Not rolling over the log is
|
|
|
+ * important for this test since we are testing retention of the one and only log file which
|
|
|
+ * predates each retained snapshot.
|
|
|
+ */
|
|
|
+ SyncRequestProcessor.setSnapCount(SNAP_RETAIN_COUNT * NUM_ZNODES_PER_SNAPSHOT * 10);
|
|
|
+
|
|
|
+ // Create Zookeeper and connect to it.
|
|
|
+ tmpDir = ClientBase.createTmpDir();
|
|
|
+ ClientBase.setupTestEnv();
|
|
|
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
|
|
|
+ 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 = ClientBase.createZKClient(HOSTPORT);
|
|
|
+
|
|
|
+ // Unique identifier for each znode that we create.
|
|
|
+ int unique = 0;
|
|
|
+ try {
|
|
|
+ /**
|
|
|
+ * Create some znodes and take a snapshot. Repeat this until we have SNAP_RETAIN_COUNT
|
|
|
+ * snapshots. Do not rollover the log.
|
|
|
+ */
|
|
|
+ for (int snapshotCount = 0; snapshotCount < SNAP_RETAIN_COUNT; snapshotCount++) {
|
|
|
+ for (int i = 0; i< 100; i++, unique++) {
|
|
|
+ zk.create("/snap-" + unique, new byte[0], Ids.OPEN_ACL_UNSAFE,
|
|
|
+ CreateMode.PERSISTENT);
|
|
|
+ }
|
|
|
+ zks.takeSnapshot();
|
|
|
+ }
|
|
|
+ // Create some additional znodes without taking a snapshot afterwards.
|
|
|
+ for (int i = 0; i< 100; i++, unique++) {
|
|
|
+ zk.create("/snap-" + unique, new byte[0], Ids.OPEN_ACL_UNSAFE,
|
|
|
+ CreateMode.PERSISTENT);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ zk.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ // Shutdown Zookeeper.
|
|
|
+ f.shutdown();
|
|
|
+ zks.getTxnLogFactory().close();
|
|
|
+ zks.shutdown();
|
|
|
+ Assert.assertTrue("waiting for server to shutdown",
|
|
|
+ ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
|
|
|
+
|
|
|
+ // Purge snapshot and log files.
|
|
|
+ PurgeTxnLog.purge(tmpDir, tmpDir, SNAP_RETAIN_COUNT);
|
|
|
+
|
|
|
+ // Initialize Zookeeper again from the same dataDir.
|
|
|
+ zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
|
|
|
+ f = ServerCnxnFactory.createFactory(PORT, -1);
|
|
|
+ f.startup(zks);
|
|
|
+ zk = ClientBase.createZKClient(HOSTPORT);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Verify that the last znode that was created above exists. This znode's creation was
|
|
|
+ * captured by the transaction log which was created before any of the above
|
|
|
+ * SNAP_RETAIN_COUNT snapshots were created, but it's not captured in any of these
|
|
|
+ * snapshots. So for it it exist, the (only) existing log file should not have been purged.
|
|
|
+ */
|
|
|
+ final String lastZnode = "/snap-" + (unique - 1);
|
|
|
+ final Stat stat = zk.exists(lastZnode, false);
|
|
|
+ Assert.assertNotNull("Last znode does not exist: " + lastZnode, stat);
|
|
|
+
|
|
|
+ // Shutdown for the last time.
|
|
|
+ f.shutdown();
|
|
|
+ zks.getTxnLogFactory().close();
|
|
|
+ zks.shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
+ private File createDataDirLogFile(File version_2, int Zxid) throws IOException {
|
|
|
+ File logFile = new File(version_2 + "/log." + Long.toHexString(Zxid));
|
|
|
+ Assert.assertTrue("Failed to create log File:" + logFile.toString(),
|
|
|
+ logFile.createNewFile());
|
|
|
+ return logFile;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void createDataDirFiles(AtomicInteger offset, int limit, boolean createPrecedingLogFile,
|
|
|
File version_2, List<File> snaps, List<File> logs)
|
|
|
throws IOException {
|
|
|
int counter = offset.get() + (2 * limit);
|
|
|
+ if (createPrecedingLogFile) {
|
|
|
+ counter++;
|
|
|
+ }
|
|
|
offset.set(counter);
|
|
|
for (int i = 0; i < limit; i++) {
|
|
|
// simulate log file
|
|
|
- File logFile = new File(version_2 + "/log." + Long.toHexString(--counter));
|
|
|
- Assert.assertTrue("Failed to create log File:" + logFile.toString(),
|
|
|
- logFile.createNewFile());
|
|
|
- logs.add(logFile);
|
|
|
+ logs.add(createDataDirLogFile(version_2, --counter));
|
|
|
// simulate snapshot file
|
|
|
File snapFile = new File(version_2 + "/snapshot."
|
|
|
+ Long.toHexString(--counter));
|
|
@@ -400,6 +541,9 @@ public class PurgeTxnTest extends ZKTestCase {
|
|
|
snapFile.createNewFile());
|
|
|
snaps.add(snapFile);
|
|
|
}
|
|
|
+ if (createPrecedingLogFile) {
|
|
|
+ logs.add(createDataDirLogFile(version_2, --counter));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private void verifyFilesAfterPurge(List<File> logs, boolean exists) {
|