You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ha...@apache.org on 2018/07/30 04:22:26 UTC

zookeeper git commit: ZOOKEEPER-3082: Fix server snapshot behavior when out of disk space

Repository: zookeeper
Updated Branches:
  refs/heads/master 932fee861 -> 78e4a1047


ZOOKEEPER-3082: Fix server snapshot behavior when out of disk space

Author: Brian Nixon <ni...@fb.com>

Reviewers: Benjamin Reed <br...@apache.org>, Michale Han <ha...@apache.org>

Closes #560 from enixon/ZOOKEEPER-3082


Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/78e4a104
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/78e4a104
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/78e4a104

Branch: refs/heads/master
Commit: 78e4a1047c701006dd4ec8d09065eda0e7adedb5
Parents: 932fee8
Author: Brian Nixon <ni...@fb.com>
Authored: Sun Jul 29 21:22:23 2018 -0700
Committer: Michael Han <ha...@apache.org>
Committed: Sun Jul 29 21:22:23 2018 -0700

----------------------------------------------------------------------
 .../server/persistence/FileTxnSnapLog.java      | 34 ++++++--
 .../server/persistence/EmptySnapshotTest.java   | 84 ++++++++++++++++++++
 2 files changed, 112 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zookeeper/blob/78e4a104/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java b/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
index 76478c4..e6d2998 100644
--- a/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
+++ b/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
@@ -50,12 +50,12 @@ import org.slf4j.LoggerFactory;
 public class FileTxnSnapLog {
     //the directory containing the
     //the transaction logs
-    private final File dataDir;
+    final File dataDir;
     //the directory containing the
     //the snapshot directory
-    private final File snapDir;
-    private TxnLog txnLog;
-    private SnapShot snapLog;
+    final File snapDir;
+    TxnLog txnLog;
+    SnapShot snapLog;
     private final boolean autoCreateDB;
     public final static int VERSION = 2;
     public final static String version = "version-";
@@ -399,8 +399,30 @@ public class FileTxnSnapLog {
         File snapshotFile = new File(snapDir, Util.makeSnapshotName(lastZxid));
         LOG.info("Snapshotting: 0x{} to {}", Long.toHexString(lastZxid),
                 snapshotFile);
-        snapLog.serialize(dataTree, sessionsWithTimeouts, snapshotFile, syncSnap);
-
+        try {
+            snapLog.serialize(dataTree, sessionsWithTimeouts, snapshotFile, syncSnap);
+        } catch (IOException e) {
+            if (snapshotFile.length() == 0) {
+                /* This may be caused by a full disk. In such a case, the server
+                 * will get stuck in a loop where it tries to write a snapshot
+                 * out to disk, and ends up creating an empty file instead.
+                 * Doing so will eventually result in valid snapshots being
+                 * removed during cleanup. */
+                if (snapshotFile.delete()) {
+                    LOG.info("Deleted empty snapshot file: " +
+                             snapshotFile.getAbsolutePath());
+                } else {
+                    LOG.warn("Could not delete empty snapshot file: " +
+                             snapshotFile.getAbsolutePath());
+                }
+            } else {
+                /* Something else went wrong when writing the snapshot out to
+                 * disk. If this snapshot file is invalid, when restarting,
+                 * ZooKeeper will skip it, and find the last known good snapshot
+                 * instead. */
+            }
+            throw e;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/78e4a104/src/java/test/org/apache/zookeeper/server/persistence/EmptySnapshotTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/persistence/EmptySnapshotTest.java b/src/java/test/org/apache/zookeeper/server/persistence/EmptySnapshotTest.java
new file mode 100644
index 0000000..4aa8a7b
--- /dev/null
+++ b/src/java/test/org/apache/zookeeper/server/persistence/EmptySnapshotTest.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.persistence;
+
+import org.junit.Assert;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.zookeeper.server.DataTree;
+import org.junit.Test;
+
+/**
+ * This test checks that the server does not create empty snapshot files if the
+ * disk is full.
+ */
+public class EmptySnapshotTest {
+
+    static class MockFileSnap extends FileSnap {
+
+        MockFileSnap(File snapDir) {
+            super(snapDir);
+        }
+
+        public synchronized void serialize(DataTree dt, Map<Long, Integer> sessions, File snapShot,
+                                           boolean fsync) throws IOException {
+            // Create empty new file.
+            Assert.assertTrue(snapShot.createNewFile());
+            throw new IOException("Created empty snapshot file from " +
+                                  "MockFileSnap::serialize()");
+        }
+
+    }
+
+    @Test
+    public void testNoEmptySnapshot() throws Exception {
+        File tmpFile = File.createTempFile("empty-snapshot-test", ".junit",
+                 new File(System.getProperty("build.test.dir", "build")));
+        File tmpDataDir = new File(tmpFile + ".dir");
+        Assert.assertFalse(tmpDataDir.exists());
+        Assert.assertTrue(tmpDataDir.mkdirs());
+
+        FileTxnSnapLog snapLog = new FileTxnSnapLog(tmpDataDir, tmpDataDir);
+        snapLog.snapLog = new MockFileSnap(snapLog.dataDir);
+
+        Assert.assertEquals(0,
+                ((FileSnap) snapLog.snapLog).findNRecentSnapshots(10).size());
+
+        DataTree tree = new DataTree();
+        tree.createNode("/empty-snapshot-test-1", "data".getBytes(), null, -1,
+                       -1, 1, 1);
+        try {
+            snapLog.save(tree, new ConcurrentHashMap<Long, Integer>(), false);
+            Assert.fail("Should have thrown an IOException");
+        } catch (IOException e) {
+        }
+
+        Assert.assertEquals(0,
+                ((FileSnap) snapLog.snapLog).findNRecentSnapshots(10).size());
+
+        snapLog.snapLog = new FileSnap(snapLog.dataDir);
+        snapLog.save(tree, new ConcurrentHashMap<Long, Integer>(), false);
+        Assert.assertEquals(1,
+                ((FileSnap) snapLog.snapLog).findNRecentSnapshots(10).size());
+  }
+}