zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ZOOKEEPER-1621) ZooKeeper does not recover from crash when disk was full
Date Mon, 08 Jan 2018 20:16:01 GMT

    [ https://issues.apache.org/jira/browse/ZOOKEEPER-1621?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16316970#comment-16316970
] 

ASF GitHub Bot commented on ZOOKEEPER-1621:
-------------------------------------------

Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/439#discussion_r160244035
  
    --- Diff: src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java ---
    @@ -307,4 +315,104 @@ public void testReloadSnapshotWithMissingParent() throws Exception
{
     
             startServer();
         }
    +
    +    /**
    +     * Verify that FileTxnIterator doesn't throw an EOFException when the
    +     * transaction log header is incomplete.
    +     */
    +    @Test
    +    public void testIncompleteHeader() throws Exception {
    +        ClientBase.setupTestEnv();
    +        File dataDir = ClientBase.createTmpDir();
    +        loadDatabase(dataDir, NUM_MESSAGES);
    +
    +        File logDir = new File(dataDir, FileTxnSnapLog.version +
    +                                        FileTxnSnapLog.VERSION);
    +        FileTxnLog.FileTxnIterator fileItr = new FileTxnLog.FileTxnIterator(logDir, 0);
    +        List<File> logFiles = fileItr.getStoredFiles();
    +        int numTransactions = 0;
    +        while (fileItr.next()) {
    +            numTransactions++;
    +        }
    +        Assert.assertTrue("Verify the number of log files",
    +                          logFiles.size() > 0);
    +        Assert.assertTrue("Verify the number of transactions",
    +                          numTransactions >= NUM_MESSAGES);
    +
    +        // Truncate the last log file.
    +        File lastLogFile = logFiles.get(logFiles.size() - 1);
    +        FileChannel channel = new FileOutputStream(lastLogFile).getChannel();
    +        channel.truncate(0);
    +        channel.close();
    +
    +        // This shouldn't thow Exception.
    +        fileItr = new FileTxnLog.FileTxnIterator(logDir, 0);
    +        logFiles = fileItr.getStoredFiles();
    +        numTransactions = 0;
    +        while (fileItr.next()) {
    +        }
    +
    +        // Verify that the truncated log file does not exist anymore.
    +        Assert.assertFalse("Verify truncated log file has been deleted",
    +                           lastLogFile.exists());
    +    }
    +
    +    /**
    +     * Verifies that FileTxnIterator throws CorruptedStreamException if the
    +     * magic number is corrupted.
    +     */
    +    @Test(expected = StreamCorruptedException.class)
    +    public void testCorruptMagicNumber() throws Exception {
    +        ClientBase.setupTestEnv();
    +        File dataDir = ClientBase.createTmpDir();
    +        loadDatabase(dataDir, NUM_MESSAGES);
    +
    +        File logDir = new File(dataDir, FileTxnSnapLog.version +
    +                                        FileTxnSnapLog.VERSION);
    +        FileTxnLog.FileTxnIterator fileItr = new FileTxnLog.FileTxnIterator(logDir, 0);
    +        List<File> logFiles = fileItr.getStoredFiles();
    +        Assert.assertTrue("Verify the number of log files",
    +                          logFiles.size() > 0);
    +
    +        // Corrupt the magic number.
    +        File lastLogFile = logFiles.get(logFiles.size() - 1);
    +        RandomAccessFile file = new RandomAccessFile(lastLogFile, "rw");
    +        file.seek(0);
    +        file.writeByte(123);
    +        file.close();
    +
    +        // This should throw CorruptedStreamException.
    +        while (fileItr.next()) {
    +        }
    +    }
    +
    +    /**
    +     * Starts a standalone server and create znodes.
    +     */
    +    public void loadDatabase(File dataDir, int numEntries) throws Exception {
    +        final String hostPort = HOST + PortAssignment.unique();
    +        // setup a single server cluster
    +        ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
    +        SyncRequestProcessor.setSnapCount(100);
    +        final int PORT = Integer.parseInt(hostPort.split(":")[1]);
    +        ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
    +        f.startup(zks);
    +        Assert.assertTrue("waiting for server being up ",
    +                ClientBase.waitForServerUp(hostPort,CONNECTION_TIMEOUT));
    +        ZooKeeper zk = ClientBase.createZKClient(hostPort);
    +
    +        // Generate some transactions that will get logged.
    +        try {
    +            for (int i = 0; i < numEntries; i++) {
    +                zk.create("/load-database-" + i, new byte[0],
    +                          Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +            }
    +        } finally {
    +            zk.close();
    +        }
    +        f.shutdown();
    --- End diff --
    
    Starting the server is already implemented in base class' setUp() method and shutdown
is already in tearDown(). It's safer to let junit's infrastructure to deal with startup /
shutdown methods which makes sure the server(s) gets shutdown even if some error occurs.


> 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.4, 3.6.0
>
>         Attachments: ZOOKEEPER-1621.2.patch, 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.4.14#64029)

Mime
View raw message