Skip to content

ZOOKEEPER-4925: Fix data loss due to propagation of discontinuous committedLog #2254

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

Merged
merged 2 commits into from
Jun 10, 2025
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -78,6 +78,19 @@ public Request(long sessionId, int xid, int type, TxnHeader hdr, Record txn, lon
this.authInfo = null;
}

public Request(TxnHeader hdr, Record txn, TxnDigest digest) {
this.sessionId = hdr.getClientId();
this.cxid = hdr.getCxid();
this.type = hdr.getType();
this.hdr = hdr;
this.txn = txn;
this.zxid = hdr.getZxid();
this.request = null;
this.cnxn = null;
this.authInfo = null;
this.txnDigest = digest;
}

public final long sessionId;

public final int cxid;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,4 +47,8 @@ public TxnHeader getHeader() {
public TxnDigest getDigest() {
return digest;
}

public Request toRequest() {
return new Request(header, txn, digest);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import org.apache.zookeeper.server.quorum.Leader.PureRequestProposal;
import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
import org.apache.zookeeper.server.util.SerializeUtils;
import org.apache.zookeeper.server.util.ZxidUtils;
import org.apache.zookeeper.txn.TxnDigest;
import org.apache.zookeeper.txn.TxnHeader;
import org.slf4j.Logger;
Expand All @@ -82,6 +83,8 @@ public class ZKDatabase {
protected FileTxnSnapLog snapLog;
protected long minCommittedLog, maxCommittedLog;

private final boolean allowDiscontinuousProposals = Boolean.getBoolean("zookeeper.test.allowDiscontinuousProposals");

/**
* Default value is to use snapshot if txnlog size exceeds 1/3 the size of snapshot
*/
Expand Down Expand Up @@ -170,8 +173,6 @@ public boolean isInitialized() {
* data structures in zkdatabase.
*/
public void clear() {
minCommittedLog = 0;
maxCommittedLog = 0;
/* to be safe we just create a new
* datatree.
*/
Expand All @@ -182,6 +183,8 @@ public void clear() {
try {
lock.lock();
committedLog.clear();
minCommittedLog = 0;
maxCommittedLog = 0;
} finally {
lock.unlock();
}
Expand Down Expand Up @@ -320,17 +323,30 @@ public void addCommittedProposal(Request request) {
WriteLock wl = logLock.writeLock();
try {
wl.lock();
if (committedLog.size() > commitLogCount) {
committedLog.remove();
minCommittedLog = committedLog.peek().getZxid();
}
if (committedLog.isEmpty()) {
minCommittedLog = request.zxid;
maxCommittedLog = request.zxid;
} else if (request.zxid <= maxCommittedLog) {
// This could happen if lastProcessedZxid is rewinded and database is re-synced.
// Currently, it only happens in test codes, but it should also be safe for production path.
return;
} else if (!allowDiscontinuousProposals
&& request.zxid != maxCommittedLog + 1
&& ZxidUtils.getEpochFromZxid(request.zxid) <= ZxidUtils.getEpochFromZxid(maxCommittedLog)) {
String msg = String.format(
"Committed proposal cached out of order: 0x%s is not the next proposal of 0x%s",
ZxidUtils.zxidToString(request.zxid),
ZxidUtils.zxidToString(maxCommittedLog));
LOG.error(msg);
throw new IllegalStateException(msg);
}
PureRequestProposal p = new PureRequestProposal(request);
committedLog.add(p);
maxCommittedLog = p.getZxid();
if (committedLog.size() > commitLogCount) {
committedLog.remove();
minCommittedLog = committedLog.peek().getZxid();
}
} finally {
wl.unlock();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1846,13 +1846,6 @@ private void processSasl(RequestRecord request, ServerCnxn cnxn, RequestHeader r
cnxn.sendResponse(replyHeader, record, "response");
}

// entry point for quorum/Learner.java
public ProcessTxnResult processTxn(TxnHeader hdr, Record txn) {
processTxnForSessionEvents(null, hdr, txn);
return processTxnInDB(hdr, txn, null);
}

// entry point for FinalRequestProcessor.java
public ProcessTxnResult processTxn(Request request) {
TxnHeader hdr = request.getHdr();
processTxnForSessionEvents(request, hdr, request.getTxn());
Expand All @@ -1864,8 +1857,10 @@ public ProcessTxnResult processTxn(Request request) {
if (!writeRequest && !quorumRequest) {
return new ProcessTxnResult();
}

ProcessTxnResult rc;
synchronized (outstandingChanges) {
ProcessTxnResult rc = processTxnInDB(hdr, request.getTxn(), request.getTxnDigest());
rc = processTxnInDB(hdr, request.getTxn(), request.getTxnDigest());

// request.hdr is set for write requests, which are the only ones
// that add to outstandingChanges.
Expand All @@ -1886,13 +1881,13 @@ public ProcessTxnResult processTxn(Request request) {
}
}
}
}

// do not add non quorum packets to the queue.
if (quorumRequest) {
getZKDatabase().addCommittedProposal(request);
}
return rc;
// do not add non quorum packets to the queue.
if (quorumRequest) {
getZKDatabase().addCommittedProposal(request);
}
return rc;
}

private void processTxnForSessionEvents(Request request, TxnHeader hdr, Record txn) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
import org.apache.zookeeper.server.util.SerializeUtils;
import org.apache.zookeeper.server.util.ZxidUtils;
import org.apache.zookeeper.txn.SetDataTxn;
import org.apache.zookeeper.txn.TxnDigest;
import org.apache.zookeeper.txn.TxnHeader;

/**
Expand Down Expand Up @@ -164,7 +163,6 @@ protected void processPacket(QuorumPacket qp) throws Exception {
TxnLogEntry logEntry = SerializeUtils.deserializeTxn(qp.getData());
TxnHeader hdr = logEntry.getHeader();
Record txn = logEntry.getTxn();
TxnDigest digest = logEntry.getDigest();
if (hdr.getZxid() != lastQueued + 1) {
LOG.warn(
"Got zxid 0x{} expected 0x{}",
Expand All @@ -179,7 +177,7 @@ protected void processPacket(QuorumPacket qp) throws Exception {
self.setLastSeenQuorumVerifier(qv, true);
}

fzk.logRequest(hdr, txn, digest);
fzk.logRequest(logEntry.toRequest());
if (hdr != null) {
/*
* Request header is created only by the leader, so this is only set
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.LinkedBlockingQueue;
import javax.management.JMException;
import org.apache.jute.Record;
import org.apache.zookeeper.jmx.MBeanRegistry;
import org.apache.zookeeper.metrics.MetricsContext;
import org.apache.zookeeper.server.ExitCode;
Expand All @@ -33,8 +32,6 @@
import org.apache.zookeeper.server.SyncRequestProcessor;
import org.apache.zookeeper.server.ZKDatabase;
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
import org.apache.zookeeper.txn.TxnDigest;
import org.apache.zookeeper.txn.TxnHeader;
import org.apache.zookeeper.util.ServiceUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -79,20 +76,17 @@ protected void setupRequestProcessors() {

LinkedBlockingQueue<Request> pendingTxns = new LinkedBlockingQueue<>();

public void logRequest(TxnHeader hdr, Record txn, TxnDigest digest) {
final Request request = buildRequestToProcess(hdr, txn, digest);
public void logRequest(Request request) {
if ((request.zxid & 0xffffffffL) != 0) {
pendingTxns.add(request);
}
syncProcessor.processRequest(request);
}

/**
* Build a request for the txn and append it to the transaction log
* @param hdr the txn header
* @param txn the txn
* @param digest the digest of txn
* Append txn request to the transaction log directly without go through request processors.
*/
public void appendRequest(final TxnHeader hdr, final Record txn, final TxnDigest digest) throws IOException {
final Request request = new Request(hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, hdr.getZxid());
request.setTxnDigest(digest);
public void appendRequest(Request request) throws IOException {
getZKDatabase().append(request);
}

Expand Down Expand Up @@ -188,20 +182,4 @@ protected void unregisterMetrics() {
rootContext.unregisterGauge("synced_observers");

}

/**
* Build a request for the txn
* @param hdr the txn header
* @param txn the txn
* @param digest the digest of txn
* @return a request moving through a chain of RequestProcessors
*/
private Request buildRequestToProcess(final TxnHeader hdr, final Record txn, final TxnDigest digest) {
final Request request = new Request(hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, hdr.getZxid());
request.setTxnDigest(digest);
if ((request.zxid & 0xffffffffL) != 0) {
pendingTxns.add(request);
}
return request;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,10 @@ static class PacketInFlight {
Record rec;
TxnDigest digest;

Request toRequest() {
return new Request(hdr, rec, digest);
}

}

QuorumPeer self;
Expand Down Expand Up @@ -630,11 +634,21 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception {
pif.hdr = logEntry.getHeader();
pif.rec = logEntry.getTxn();
pif.digest = logEntry.getDigest();
if (pif.hdr.getZxid() != lastQueued + 1) {
LOG.warn(
"Got zxid 0x{} expected 0x{}",
if (lastQueued == 0) {
LOG.info("DIFF sync got first proposal 0x{}", Long.toHexString(pif.hdr.getZxid()));
} else if (pif.hdr.getZxid() != lastQueued + 1) {
if (ZxidUtils.getEpochFromZxid(pif.hdr.getZxid()) <= ZxidUtils.getEpochFromZxid(lastQueued)) {
String msg = String.format("DIFF sync got proposal 0x%s which is not next of last proposal 0x%s",
Long.toHexString(pif.hdr.getZxid()),
Long.toHexString(lastQueued));
LOG.error(msg);
throw new Exception(msg);
}
// We can't tell whether it is a data loss. Given that new epoch is rare,
// log at warn should not be too verbose.
LOG.warn("DIFF sync got new epoch proposal 0x{}, last proposal 0x{}",
Long.toHexString(pif.hdr.getZxid()),
Long.toHexString(lastQueued + 1));
Long.toHexString(lastQueued));
}
lastQueued = pif.hdr.getZxid();

Expand Down Expand Up @@ -666,7 +680,7 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception {
Long.toHexString(qp.getZxid()),
Long.toHexString(pif.hdr.getZxid()));
} else {
zk.processTxn(pif.hdr, pif.rec);
zk.processTxn(pif.toRequest());
packetsNotLogged.remove();
}
} else {
Expand Down Expand Up @@ -696,18 +710,27 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception {
packet.rec = logEntry.getTxn();
packet.hdr = logEntry.getHeader();
packet.digest = logEntry.getDigest();
// Log warning message if txn comes out-of-order
if (packet.hdr.getZxid() != lastQueued + 1) {
LOG.warn(
"Got zxid 0x{} expected 0x{}",
Long.toHexString(packet.hdr.getZxid()),
Long.toHexString(lastQueued + 1));
if (lastQueued == 0) {
LOG.info("DIFF sync got first proposal 0x{}", Long.toHexString(packet.hdr.getZxid()));
} else if (packet.hdr.getZxid() != lastQueued + 1) {
if (ZxidUtils.getEpochFromZxid(packet.hdr.getZxid()) <= ZxidUtils.getEpochFromZxid(lastQueued)) {
String msg = String.format("DIFF sync got proposal 0x%s which is not next of last proposal 0x%s",
Long.toHexString(packet.hdr.getZxid()),
Long.toHexString(lastQueued));
LOG.error(msg);
throw new Exception(msg);
}
// We can't tell whether it is a data loss. Given that new epoch is rare,
// log at warn should not be too verbose.
LOG.warn("DIFF sync got new epoch proposal 0x{}, last proposal 0x{}",
Long.toHexString(packet.hdr.getZxid()),
Long.toHexString(lastQueued));
}
lastQueued = packet.hdr.getZxid();
}
if (!writeToTxnLog) {
// Apply to db directly if we haven't taken the snapshot
zk.processTxn(packet.hdr, packet.rec);
zk.processTxn(packet.toRequest());
} else {
packetsNotLogged.add(packet);
packetsCommitted.add(qp.getZxid());
Expand Down Expand Up @@ -780,8 +803,8 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception {
continue;
}
packetsNotLogged.removeFirst();
fzk.appendRequest(pif.hdr, pif.rec, pif.digest);
fzk.processTxn(pif.hdr, pif.rec);
fzk.appendRequest(pif.toRequest());
fzk.processTxn(pif.toRequest());
}

// @see https://issues.apache.org/jira/browse/ZOOKEEPER-4646
Expand Down Expand Up @@ -823,7 +846,7 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception {
if (zk instanceof FollowerZooKeeperServer) {
FollowerZooKeeperServer fzk = (FollowerZooKeeperServer) zk;
for (PacketInFlight p : packetsNotLogged) {
fzk.logRequest(p.hdr, p.rec, p.digest);
fzk.logRequest(p.toRequest());
}
LOG.info("{} txns have been logged asynchronously", packetsNotLogged.size());

Expand All @@ -847,8 +870,7 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception {
continue;
}
packetsCommitted.remove();
Request request = new Request(p.hdr.getClientId(), p.hdr.getCxid(), p.hdr.getType(), p.hdr, p.rec, -1);
request.setTxnDigest(p.digest);
Request request = p.toRequest();
ozk.commitRequest(request);
}
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -202,12 +202,8 @@ protected void processPacket(QuorumPacket qp) throws Exception {
case Leader.INFORM:
ServerMetrics.getMetrics().LEARNER_COMMIT_RECEIVED_COUNT.add(1);
logEntry = SerializeUtils.deserializeTxn(qp.getData());
hdr = logEntry.getHeader();
txn = logEntry.getTxn();
digest = logEntry.getDigest();
Request request = new Request(hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, 0);
Request request = logEntry.toRequest();
request.logLatency(ServerMetrics.getMetrics().COMMIT_PROPAGATION_LATENCY);
request.setTxnDigest(digest);
ObserverZooKeeperServer obs = (ObserverZooKeeperServer) zk;
obs.commitRequest(request);
break;
Expand All @@ -219,13 +215,10 @@ protected void processPacket(QuorumPacket qp) throws Exception {
byte[] remainingdata = new byte[buffer.remaining()];
buffer.get(remainingdata);
logEntry = SerializeUtils.deserializeTxn(remainingdata);
hdr = logEntry.getHeader();
txn = logEntry.getTxn();
digest = logEntry.getDigest();
QuorumVerifier qv = self.configFromString(new String(((SetDataTxn) txn).getData(), UTF_8));

request = new Request(hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, 0);
request.setTxnDigest(digest);
request = logEntry.toRequest();
obs = (ObserverZooKeeperServer) zk;

boolean majorChange = self.processReconfig(qv, suggestedLeaderId, qp.getZxid(), true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,13 +60,15 @@ public class TxnLogDigestTest extends ClientBase {

@BeforeEach
public void setUp() throws Exception {
System.setProperty("zookeeper.test.allowDiscontinuousProposals", "true");
super.setUp();
server = serverFactory.getZooKeeperServer();
zk = createClient();
}

@AfterEach
public void tearDown() throws Exception {
System.clearProperty("zookeeper.test.allowDiscontinuousProposals");
// server will be closed in super.tearDown
super.tearDown();

Expand Down
Loading