Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ZOOKEEPER-3781: Create snapshots on followers when snapshot.trust.empty is true #111

Merged
merged 1 commit into from
Nov 2, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -544,6 +544,10 @@ public void takeSnapshot(boolean syncSnap) {
ServerMetrics.getMetrics().SNAPSHOT_TIME.add(elapsed);
}

public boolean shouldForceWriteInitialSnapshotAfterLeaderElection() {
return txnLogFactory.shouldForceWriteInitialSnapshotAfterLeaderElection();
}

@Override
public long getDataDirSize() {
if (zkDb == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -229,6 +229,15 @@ public SnapshotInfo getLastSnapshotInfo() {
return this.snapLog.getLastSnapshotInfo();
}

/**
* whether to force the write of an initial snapshot after a leader election,
* to address ZOOKEEPER-3781 after upgrading from Zookeeper 3.4.x.
* @return true if an initial snapshot should be written even if not otherwise required, false otherwise.
*/
public boolean shouldForceWriteInitialSnapshotAfterLeaderElection() {
return trustEmptySnapshot && getLastSnapshotInfo() == null;
}

/**
* this function restores the server
* database after reading from the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -561,7 +561,13 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception {
if (qp.getType() == Leader.DIFF) {
LOG.info("Getting a diff from the leader 0x{}", Long.toHexString(qp.getZxid()));
self.setSyncMode(QuorumPeer.SyncMode.DIFF);
snapshotNeeded = false;
if (zk.shouldForceWriteInitialSnapshotAfterLeaderElection()) {
LOG.info("Forcing a snapshot write as part of upgrading from an older Zookeeper. This should only happen while upgrading.");
snapshotNeeded = true;
syncSnapshot = true;
} else {
snapshotNeeded = false;
}
} else if (qp.getType() == Leader.SNAP) {
self.setSyncMode(QuorumPeer.SyncMode.SNAP);
LOG.info("Getting a snapshot from leader 0x{}", Long.toHexString(qp.getZxid()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,13 @@
package org.apache.zookeeper.test;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import java.io.File;
import java.io.IOException;
import java.io.PrintWriter;
import java.nio.file.Files;
import java.util.List;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.PortAssignment;
Expand Down Expand Up @@ -143,6 +145,54 @@ public void testRestoreWithTrustedEmptySnapFiles() throws Exception {
runTest(false, true);
}

@Test
public void testRestoreWithTrustedEmptySnapFilesWhenFollowing() throws Exception {
QuorumUtil qu = new QuorumUtil(1);
try {
qu.startAll();
String connString = qu.getConnectionStringForServer(1);
try (ZooKeeper zk = new ZooKeeper(connString, CONNECTION_TIMEOUT, this)) {
for (int i = 0; i < N_TRANSACTIONS; i++) {
zk.create("/node-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
}
}
int leaderIndex = qu.getLeaderServer();
//Shut down the cluster and delete the snapshots from the followers
for (int i = 1; i <= qu.ALL; i++) {
qu.shutdown(i);
if (i != leaderIndex) {
FileTxnSnapLog txnLogFactory = qu.getPeer(i).peer.getTxnFactory();
List<File> snapshots = txnLogFactory.findNRecentSnapshots(10);
assertTrue(snapshots.size() > 0, "We have a snapshot to corrupt");
for (File file : snapshots) {
Files.delete(file.toPath());
}
assertEquals(txnLogFactory.findNRecentSnapshots(10).size(), 0);
}
}
//Start while trusting empty snapshots, verify that the followers save snapshots
System.setProperty(FileTxnSnapLog.ZOOKEEPER_SNAPSHOT_TRUST_EMPTY, "true");
qu.start(leaderIndex);
for (int i = 1; i <= qu.ALL; i++) {
if (i != leaderIndex) {
qu.restart(i);
FileTxnSnapLog txnLogFactory = qu.getPeer(i).peer.getTxnFactory();
List<File> snapshots = txnLogFactory.findNRecentSnapshots(10);
assertTrue(snapshots.size() > 0, "A snapshot should have been created on follower " + i);
}
}
//Check that the created nodes are still there
try (ZooKeeper zk = new ZooKeeper(connString, CONNECTION_TIMEOUT, this)) {
for (int i = 0; i < N_TRANSACTIONS; i++) {
assertNotNull(zk.exists("/node-" + i, false));
}
}
} finally {
System.clearProperty(FileTxnSnapLog.ZOOKEEPER_SNAPSHOT_TRUST_EMPTY);
qu.tearDown();
}
}

public void process(WatchedEvent event) {
// do nothing
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -251,22 +251,22 @@ public void shutdownAll() {
public void shutdown(int id) {
QuorumPeer qp = getPeer(id).peer;
try {
LOG.info("Shutting down quorum peer {}", qp.getName());
LOG.info("Shutting down quorum peer {} with id {}", qp.getName(), id);
qp.shutdown();
Election e = qp.getElectionAlg();
if (e != null) {
LOG.info("Shutting down leader election {}", qp.getName());
LOG.info("Shutting down leader election {} with id {}", qp.getName(), id);
e.shutdown();
} else {
LOG.info("No election available to shutdown {}", qp.getName());
LOG.info("No election available to shutdown {} with id {}", qp.getName(), id);
}
LOG.info("Waiting for {} to exit thread", qp.getName());
LOG.info("Waiting for {} with id {} to exit thread", qp.getName(), id);
qp.join(30000);
if (qp.isAlive()) {
fail("QP failed to shutdown in 30 seconds: " + qp.getName());
fail("QP failed to shutdown in 30 seconds: " + qp.getName() + " " + id);
}
} catch (InterruptedException e) {
LOG.debug("QP interrupted: {}", qp.getName(), e);
LOG.debug("QP interrupted: {} {}", qp.getName(), id, e);
}
}

Expand Down