Skip to content

Commit

Permalink
ZOOKEEPER-3781: Create snapshots on followers when snapshot.trust.emp…
Browse files Browse the repository at this point in the history
…ty is true

snapshot.trust.empty is an escape hatch for users upgrading from 3.4.x to later Zookeeper
versions, allowing nodes to start with a non-empty transaction log but no snapshot.

The intent is for this setting to be enabled for a short while during the upgrade,
and then disabled again, as the check it disables is a safety feature.

Prior to this PR, a node would only write a snapshot locally if it became leader,
or if it had fallen so far behind the leader that the leader sent a SNAP message instead
of a DIFF. This made the upgrade process inconvenient, as not all nodes would create
a snapshot when snapshot.trust.empty was true, meaning that the safety check could
not be flipped back on.

This PR makes follower nodes write a local snapshot when they receive NEWLEADER,
if they have no local snapshot and snapshot.trust.empty is true.

Author: Stig Rohde Døssing <stig@humio.com>

Reviewers: Enrico Olivelli <eolivelli@apache.org>, Damien Diederen <ddiederen@apache.org>

Closes #1581 from srdo/zookeeper-3781

(cherry picked from commit 1214d3b)
Signed-off-by: Damien Diederen <ddiederen@apache.org>
(cherry picked from commit 679cc2b)
  • Loading branch information
Stig Rohde Døssing authored and Mate Szalay-Beko committed May 16, 2022
1 parent 22d0c85 commit d1ec2f3
Show file tree
Hide file tree
Showing 5 changed files with 82 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -353,6 +353,10 @@ public void takeSnapshot(boolean syncSnap){
}
}

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 @@ -217,6 +217,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 @@ -390,7 +390,13 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception{
synchronized (zk) {
if (qp.getType() == Leader.DIFF) {
LOG.info("Getting a diff from the leader 0x{}", Long.toHexString(qp.getZxid()));
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) {
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 @@ -22,6 +22,7 @@
import java.io.File;
import java.io.IOException;
import java.io.PrintWriter;
import java.nio.file.Files;
import java.util.List;

import org.apache.log4j.Logger;
Expand All @@ -39,7 +40,7 @@
import org.junit.Assert;
import org.junit.Test;

/** If snapshots are corrupted to the empty file or deleted, Zookeeper should
/** If snapshots are corrupted to the empty file or deleted, Zookeeper should
* not proceed to read its transaction log files
* Test that zxid == -1 in the presence of emptied/deleted snapshots
*/
Expand Down Expand Up @@ -145,6 +146,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);
Assert.assertTrue("We have a snapshot to corrupt", snapshots.size() > 0);
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);
Assert.assertTrue("A snapshot should have been created on follower " + i, snapshots.size() > 0);
}
}
//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++) {
Assert.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 @@ -107,7 +107,7 @@ public QuorumUtil(int n, int syncLimit) throws RuntimeException {
ps.clientPort = PortAssignment.unique();
peers.put(i, ps);

peersView.put(Long.valueOf(i), new QuorumServer(i,
peersView.put(Long.valueOf(i), new QuorumServer(i,
new InetSocketAddress("127.0.0.1", PortAssignment.unique()),
new InetSocketAddress("127.0.0.1", PortAssignment.unique()),
new InetSocketAddress("127.0.0.1", ps.clientPort),
Expand Down Expand Up @@ -136,7 +136,7 @@ public PeerStruct getPeer(int id) {

// This was added to avoid running into the problem of ZOOKEEPER-1539
public boolean disableJMXTest = false;


public void enableLocalSession(boolean localSessionEnabled) {
this.localSessionEnabled = localSessionEnabled;
Expand All @@ -158,7 +158,7 @@ public void startAll() throws IOException {

// This was added to avoid running into the problem of ZOOKEEPER-1539
if (disableJMXTest) return;

// interesting to see what's there...
try {
JMXEnv.dump();
Expand Down Expand Up @@ -250,22 +250,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 " + qp.getName() + " to exit thread");
LOG.info("Waiting for {} with id {} to exit thread", qp.getName(), id);
qp.join(30000);
if (qp.isAlive()) {
Assert.fail("QP failed to shutdown in 30 seconds: " + qp.getName());
Assert.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 Expand Up @@ -293,11 +293,11 @@ public QuorumPeer getLeaderQuorumPeer() {
}

public List<QuorumPeer> getFollowerQuorumPeers() {
List<QuorumPeer> peerList = new ArrayList<QuorumPeer>(ALL - 1);
List<QuorumPeer> peerList = new ArrayList<QuorumPeer>(ALL - 1);

for (PeerStruct ps: peers.values()) {
if (ps.peer.leader == null) {
peerList.add(ps.peer);
peerList.add(ps.peer);
}
}

Expand All @@ -308,7 +308,7 @@ public void tearDown() throws Exception {
LOG.info("TearDown started");

OSMXBean osMbean = new OSMXBean();
if (osMbean.getUnix() == true) {
if (osMbean.getUnix() == true) {
LOG.info("fdcount after test is: " + osMbean.getOpenFileDescriptorCount());
}

Expand Down

0 comments on commit d1ec2f3

Please sign in to comment.