[
https://issues.apache.org/jira/browse/ZOOKEEPER-1621?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15560526#comment-15560526
]
Abhishek Rai commented on ZOOKEEPER-1621:
-----------------------------------------
Reviving this old thread. [~shralex] has a valid concern about trading off
consistency for availability. However, for the specific issue being addressed
here, we can have both.
The patch skips transaction logs with an incomplete header (the first 16
bytes). Skipping such files should not cause any loss of data as the header is
an internal bookkeeping write from Zookeeper and does not contain any user
data. This avoids the current behavior of Zookeeper crashing on encountering
an incomplete header, which compromises availability.
This has been a recurring problem for us in production because our app's
operating environment occasionally causes a Zookeeper server's disk to become
full. After that, the server invariably runs into this problem - perhaps
because there's something else that deterministically triggers a log rotation
when the previous txn log throws an IOException due to disk full?
That said, we can tighten the exception being caught in [~michim]'s patch to
EOFException instead of IOException to make sure that the log we are skipping
indeed only has a partially written header and nothing else (in
FileTxnLog.goToNextLog).
Additionally, I have written a test to verify that EOFException is thrown if
and only if the header is truncated. Zookeeper already ignores any other
partially written transactions in the txn log. If that's useful, I can upload
the test, thanks.
> ZooKeeper does not recover from crash when disk was full
> --------------------------------------------------------
>
> Key: ZOOKEEPER-1621
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1621
> Project: ZooKeeper
> Issue Type: Bug
> Components: server
> Affects Versions: 3.4.3
> Environment: Ubuntu 12.04, Amazon EC2 instance
> Reporter: David Arthur
> Assignee: Michi Mutsuzaki
> Fix For: 3.5.3, 3.6.0
>
> Attachments: ZOOKEEPER-1621.patch, zookeeper.log.gz
>
>
> The disk that ZooKeeper was using filled up. During a snapshot write, I got
> the following exception
> 2013-01-16 03:11:14,098 - ERROR [SyncThread:0:SyncRequestProcessor@151] -
> Severe unrecoverable error, exiting
> java.io.IOException: No space left on device
> at java.io.FileOutputStream.writeBytes(Native Method)
> at java.io.FileOutputStream.write(FileOutputStream.java:282)
> at
> java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
> at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:123)
> at
> org.apache.zookeeper.server.persistence.FileTxnLog.commit(FileTxnLog.java:309)
> at
> org.apache.zookeeper.server.persistence.FileTxnSnapLog.commit(FileTxnSnapLog.java:306)
> at org.apache.zookeeper.server.ZKDatabase.commit(ZKDatabase.java:484)
> at
> org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:162)
> at
> org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:101)
> Then many subsequent exceptions like:
> 2013-01-16 15:02:23,984 - ERROR [main:Util@239] - Last transaction was
> partial.
> 2013-01-16 15:02:23,985 - ERROR [main:ZooKeeperServerMain@63] - Unexpected
> exception, exiting abnormally
> java.io.EOFException
> at java.io.DataInputStream.readInt(DataInputStream.java:375)
> at
> org.apache.jute.BinaryInputArchive.readInt(BinaryInputArchive.java:63)
> at
> org.apache.zookeeper.server.persistence.FileHeader.deserialize(FileHeader.java:64)
> at
> org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.inStreamCreated(FileTxnLog.java:558)
> at
> org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.createInputArchive(FileTxnLog.java:577)
> at
> org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.goToNextLog(FileTxnLog.java:543)
> at
> org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.next(FileTxnLog.java:625)
> at
> org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.init(FileTxnLog.java:529)
> at
> org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.<init>(FileTxnLog.java:504)
> at
> org.apache.zookeeper.server.persistence.FileTxnLog.read(FileTxnLog.java:341)
> at
> org.apache.zookeeper.server.persistence.FileTxnSnapLog.restore(FileTxnSnapLog.java:130)
> at
> org.apache.zookeeper.server.ZKDatabase.loadDataBase(ZKDatabase.java:223)
> at
> org.apache.zookeeper.server.ZooKeeperServer.loadData(ZooKeeperServer.java:259)
> at
> org.apache.zookeeper.server.ZooKeeperServer.startdata(ZooKeeperServer.java:386)
> at
> org.apache.zookeeper.server.NIOServerCnxnFactory.startup(NIOServerCnxnFactory.java:138)
> at
> org.apache.zookeeper.server.ZooKeeperServerMain.runFromConfig(ZooKeeperServerMain.java:112)
> at
> org.apache.zookeeper.server.ZooKeeperServerMain.initializeAndRun(ZooKeeperServerMain.java:86)
> at
> org.apache.zookeeper.server.ZooKeeperServerMain.main(ZooKeeperServerMain.java:52)
> at
> org.apache.zookeeper.server.quorum.QuorumPeerMain.initializeAndRun(QuorumPeerMain.java:116)
> at
> org.apache.zookeeper.server.quorum.QuorumPeerMain.main(QuorumPeerMain.java:78)
> It seems to me that writing the transaction log should be fully atomic to
> avoid such situations. Is this not the case?
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)