You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@zookeeper.apache.org by "Damien Diederen (Jira)" <ji...@apache.org> on 2021/01/06 11:33:00 UTC

[jira] [Updated] (ZOOKEEPER-3701) Split brain on log disk full

     [ https://issues.apache.org/jira/browse/ZOOKEEPER-3701?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Damien Diederen updated ZOOKEEPER-3701:
---------------------------------------
    Fix Version/s: 3.6.0
                   3.5.7
                   3.7.0

> Split brain on log disk full
> ----------------------------
>
>                 Key: ZOOKEEPER-3701
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3701
>             Project: ZooKeeper
>          Issue Type: Bug
>    Affects Versions: 3.4.13, 3.5.6
>            Reporter: Ivan Kelly
>            Assignee: Andor Molnar
>            Priority: Blocker
>              Labels: pull-request-available
>             Fix For: 3.6.0, 3.5.7, 3.7.0
>
>          Time Spent: 6h 20m
>  Remaining Estimate: 0h
>
>  We ran into a situation where the cluster ended up with split brain when the log disk filled up on a node.
> The ZK cluster(3 node) in question was being used as the metadata store for Pulsar. There was an outage in the Pulsar cluster, where two the ZK nodes filled up there log disks, causing the cluster to lose quorum. Once we rectified the full disk situation and restarted the nodes everything seemed to work, but we started getting a lot of log messages about UpdateMetadataLoop retrying. UpdateMetadataLoop is used to update bookkeeper ledger metadata. If it sees a write conflict it rereads the znode, checks whether the update needs to happen, applies it and writes. These retries were flooding the log on a subset of the brokers. It turned out that it was reading a znode with version 0, but when it tried the setData with version set to 0 it was failing because the znode had a version of 2 (there were many instances of this). After investigating this, we saw that the znode had a different stat and value on ZK-1 to that on ZK-0 & ZK-2.
> We resolved the situation by deleting the log and snapshots from ZK-1 and restarting, at which point everything went back to normal. Had ZK-1 managed to become leader we would have been in a lot of trouble, but thankfully this didn't happen.
> For the sequence of events that led to split brain, I'll refer to the following code.
> {code}
> public class FileTxnSnapLog {
>     ...
>     public boolean truncateLog(long zxid) throws IOException {
>         // close the existing txnLog and snapLog
>         close();
>         // truncate it
>         FileTxnLog truncLog = new FileTxnLog(dataDir);
>         boolean truncated = truncLog.truncate(zxid);
>         truncLog.close();
>         // re-open the txnLog and snapLog
>         // I'd rather just close/reopen this object itself, however that 
>         // would have a big impact outside ZKDatabase as there are other
>         // objects holding a reference to this object.
>         txnLog = new FileTxnLog(dataDir);
>         snapLog = new FileSnap(snapDir);
>         return truncated;
>     }
>     public void close() throws IOException {
>         txnLog.close();
>         snapLog.close();
>     }
> }
> public class FileSnap implements SnapShot {
>     ...
>     public synchronized void serialize(DataTree dt, Map<Long, Integer> sessions, File snapShot)
>             throws IOException {
>         if (!close) {
>             // actual snapshot code
>         }
>     }
>     @Override
>     public synchronized void close() throws IOException {
>         close = true;
>     }
> }
> {code}
> The sequence of events that lead to the failure are:
> | 2020-01-04 01:56:56Z | ZK-2 fails to write to its transaction log due to disk full. ZK-2 is still participating in leader election. ZK-2 becomes a follower of ZK-1. ZK-1 sends TRUNC to ZK-2. truncLog.truncate on ZK-2 throws an exception because of the disk being full, and leaves the process in a broken state. |
> |2020-01-04 02:35:23Z | ZK-2 removes 9 transaction logs from disk (bringing it from 100% to 19%). It doesn't recover because its in a broken state. |
> |2020-01-09 08:57:33Z| ZK-1 fails to write to its transaction log due to disk full. Restarts as follower. Goes into loop of dropping from quorum (because it can't update transaction log)|
> |2020-01-09 08:59:33Z |ZK-1 receives snapshot from leader (ZK-0) (at 1e00000000). ZK-1 persists snapshot, but fails to add subsequent transations to log due to lack of space. ZK-1 drops from quorum.|
> |2020-01-09 09:00:12Z |ZK-1 joins quorum as follower. 1e00000000 is close enough to leader to receive TRUNC(1d0000001d). TRUNC fails because txnLog can't flush on close() in trunateLog. ZK-1 goes into loop, dropping and joining quorum.|
> |2020-01-09 09:39:00Z |ZK-1 runs purgeTxnLog. Process doesn't recover due to truncation exception having broken FileTxnSnapLog.|
> |2020-01-09 19:28:37Z |ZK-1 is restarted. ZK-1 joins quorum as follower. ZK-1 receives TRUNC(1d0000001d). In this case, txnLog.close() can succeed because there's nothing to flush. snapLog is closed. truncLog.truncate fails with "java.io.IOException: No log files found to truncate! This could happen if you still have snapshots from an old setup or log files were deleted accidentally or dataLogDir was changed in zoo.cfg.". It's true that there are no log files to truncate because the snapshot is at 1e00000000 which was received from the leader at 08:59 and nothing has been logged since. In any case, FileTxnSnapLog is in another inconsistent state. snapLog is closed. txnLog is closed, but nothing was ever written to it, so it looks like brand new.|
> |2020-01-09 19:29:04Z| ZK-2 is restarted. ZK-2 & ZK-0 are now in a good state, so they can make progress. Transactions start to be logged.|
> |2020-01-09 19:33:16Z| ZK-1 joins the quorum. As progress has been made, it receives a SNAP from the leader at 6b30001183a. It writes a snapshot, which ultimately calls FileSnap#serialize. Nothing hits the snapshot disk, because FileSnap is in closed state since 19:28. ZK-1 starts logging transactions to its log disk.|
> |2020-01-09 19:42:00Z |We do a rolling restart of the cluster.|
> |2020-01-09 19:45:11Z |ZK-1 loads the last snapshot that has been persisted to disk (1e00000000), and applies all log entries with zxid greater than the snapshot (6b30001183a onwards). |
> |2020-01-09 19:47:35Z |ZK-2 & ZK-1 form a quorum, ZK-2 leading. ZK-1 reports its lastZxid as 6b30001b32f and gets a DIFF from ZK-2.|
> From this point on, the cluster has split brain. ZK-1 is missing all transaction between 1e00000000 and 6bf0001183a. 
> There's a couple of failures in the code that could stop this problem. 
> - An exception in truncateLog should nuke the process. Even without the split brain occurring, the processes limped on in a broken state for days and required human intervention to get going again.
> - snapLog and txnLog should be defensively nulled after they're closed. 
> - FileSnap#serialize should not fail silently if close=true. This is really bad. It should at least throw an exception.
> The issue occurred with 3.4.13 running on a kubernetes cluster. The bad code paths still exist on master.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)