You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by he...@apache.org on 2012/03/06 09:20:50 UTC

svn commit: r1297377 - in /zookeeper/trunk: ./ bin/ src/docs/src/documentation/content/xdocs/ src/java/main/org/apache/zookeeper/server/ src/java/main/org/apache/zookeeper/server/persistence/ src/java/main/org/apache/zookeeper/server/quorum/ src/java/t...

Author: henry
Date: Tue Mar  6 08:20:49 2012
New Revision: 1297377

URL: http://svn.apache.org/viewvc?rev=1297377&view=rev
Log:
ZOOKEEPER-1161. Provide an option for disabling auto-creation of the data directory (phunt via henry)

Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/bin/zkServer.sh
    zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1297377&r1=1297376&r2=1297377&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Tue Mar  6 08:20:49 2012
@@ -232,6 +232,9 @@ IMPROVEMENTS:
   in order to get rid of the intermediate shell process
   (Roman Shaposhnik via phunt)
 
+  ZOOKEEPER-1161. Provide an option for disabling auto-creation of the data directory 
+  (phunt via henry)
+
 
 Release 3.4.0 - 
 

Modified: zookeeper/trunk/bin/zkServer.sh
URL: http://svn.apache.org/viewvc/zookeeper/trunk/bin/zkServer.sh?rev=1297377&r1=1297376&r2=1297377&view=diff
==============================================================================
--- zookeeper/trunk/bin/zkServer.sh (original)
+++ zookeeper/trunk/bin/zkServer.sh Tue Mar  6 08:20:49 2012
@@ -53,7 +53,7 @@ else
   . "$ZOOBINDIR"/zkEnv.sh
 fi
 
-if [ "x$SERVER_JVMFLAGS" ]
+if [ "x$SERVER_JVMFLAGS" != "x" ]
 then
     JVMFLAGS="$SERVER_JVMFLAGS $JVMFLAGS"
 fi
@@ -80,8 +80,25 @@ fi
 
 echo "Using config: $ZOOCFG" >&2
 
-if [ -z $ZOOPIDFILE ]; then
-    ZOO_DATADIR=$(grep "^[[:space:]]*dataDir" "$ZOOCFG" | sed -e 's/.*=//')
+ZOO_DATADIR=$(grep "^[[:space:]]*dataDir" "$ZOOCFG" | sed -e 's/.*=//')
+ZOO_DATALOGDIR=$(grep "^[[:space:]]*dataLogDir" "$ZOOCFG" | sed -e 's/.*=//')
+
+# iff autocreate is turned off and the datadirs don't exist fail
+# immediately as we can't create the PID file, etc..., anyway.
+if [ -n "$ZOO_DATADIR_AUTOCREATE_DISABLE" ]; then
+    if [ ! -d "$ZOO_DATADIR/version-2" ]; then
+        echo "ZooKeeper data directory is missing at $ZOO_DATADIR fix the path or run initialize"
+        exit 1
+    fi
+
+    if [ -n "$ZOO_DATALOGDIR" ] && [ ! -d "$ZOO_DATALOGDIR/version-2" ]; then
+        echo "ZooKeeper txnlog directory is missing at $ZOO_DATALOGDIR fix the path or run initialize"
+        exit 1
+    fi
+    ZOO_DATADIR_AUTOCREATE="-Dzookeeper.datadir.autocreate=false"
+fi
+
+if [ -z "$ZOOPIDFILE" ]; then
     if [ ! -d "$ZOO_DATADIR" ]; then
         mkdir -p "$ZOO_DATADIR"
     fi
@@ -102,7 +119,8 @@ start)
          exit 0
       fi
     fi
-    nohup $JAVA "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \
+    nohup $JAVA $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" \
+    "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \
     -cp "$CLASSPATH" $JVMFLAGS $ZOOMAIN "$ZOOCFG" > "$_ZOO_DAEMON_OUT" 2>&1 < /dev/null &
     if [ $? -eq 0 ]
     then
@@ -124,11 +142,12 @@ start-foreground)
     if [ "${ZOO_NOEXEC}" != "" ]; then
       ZOO_CMD="$JAVA"
     fi
-    $ZOO_CMD "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \
+    $ZOO_CMD $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" \
+    "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \
     -cp "$CLASSPATH" $JVMFLAGS $ZOOMAIN "$ZOOCFG"
     ;;
 print-cmd)
-    echo "$JAVA -Dzookeeper.log.dir=\"${ZOO_LOG_DIR}\" -Dzookeeper.root.logger=\"${ZOO_LOG4J_PROP}\" -cp \"$CLASSPATH\" $JVMFLAGS $ZOOMAIN \"$ZOOCFG\" > \"$_ZOO_DAEMON_OUT\" 2>&1 < /dev/null"
+    echo "$JAVA $ZOO_DATADIR_AUTOCREATE -Dzookeeper.log.dir=\"${ZOO_LOG_DIR}\" -Dzookeeper.root.logger=\"${ZOO_LOG4J_PROP}\" -cp \"$CLASSPATH\" $JVMFLAGS $ZOOMAIN \"$ZOOCFG\" > \"$_ZOO_DAEMON_OUT\" 2>&1 < /dev/null"
     ;;
 stop)
     echo -n "Stopping zookeeper ... "

Modified: zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml?rev=1297377&r1=1297376&r2=1297377&view=diff
==============================================================================
--- zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml (original)
+++ zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml Tue Mar  6 08:20:49 2012
@@ -1113,6 +1113,64 @@ server.3=zoo3:2888:3888</programlisting>
       </section>
 
       <section>
+        <title>Disabling data directory autocreation</title>
+
+        <para><emphasis role="bold">New in 3.5:</emphasis> The default
+        behavior of a ZooKeeper server is to automatically create the
+        data directory (specified in the configuration file) when
+        started if that directory does not already exist. This can be
+        inconvenient and even dangerous in some cases. Take the case
+        where a configuration change is made to a running server,
+        wherein the <emphasis role="bold">dataDir</emphasis> parameter
+        is accidentally changed. When the ZooKeeper server is
+        restarted it will create this non-existent directory and begin
+        serving - with an empty znode namespace. This scenario can
+        result in an effective "split brain" situation (i.e. data in
+        both the new invalid directory and the original valid data
+        store). As such is would be good to have an option to turn off
+        this autocreate behavior. In general for production
+        environments this should be done, unfortunately however the
+        default legacy behavior cannot be changed at this point and
+        therefore this must be done on a case by case basis. This is
+        left to users and to packagers of ZooKeeper distributions.
+        </para>
+
+        <para>When running <emphasis
+        role="bold">zkServer.sh</emphasis> autocreate can be disabled
+        by setting the environment variable <emphasis
+        role="bold">ZOO_DATADIR_AUTOCREATE_DISABLE</emphasis> to 1.
+        When running ZooKeeper servers directly from class files this
+        can be accomplished by setting <emphasis
+        role="bold">zookeeper.datadir.autocreate=false</emphasis> on
+        the java command line, i.e. <emphasis
+        role="bold">-Dzookeeper.datadir.autocreate=false</emphasis>
+        </para>
+
+        <para>When this feature is disabled, and the ZooKeeper server
+        determines that the required directories do not exist it will
+        generate an error and refuse to start.
+        </para>
+
+        <para>A new script <emphasis
+        role="bold">zkServer-initialize.sh</emphasis> is provided to
+        support this new feature. If autocreate is disabled it is
+        necessary for the user to first install ZooKeeper, then create
+        the data directory (and potentially txnlog directory), and
+        then start the server. Otherwise as mentioned in the previous
+        paragraph the server will not start. Running <emphasis
+        role="bold">zkServer-initialize.sh</emphasis> will create the
+        required directories, and optionally setup the myid file
+        (optional command line parameter). This script can be used
+        even if the autocreate feature itself is not used, and will
+        likely be of use to users as this (setup, including creation
+        of the myid file) has been an issue for users in the past.
+        Note that this script ensures the data directories exist only,
+        it does not create a config file, but rather requires a config
+        file to be available in order to execute.
+        </para>
+      </section>
+
+      <section>
         <title>Communication using the Netty framework</title>
 
         <para><emphasis role="bold">New in

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java?rev=1297377&r1=1297376&r2=1297377&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java Tue Mar  6 08:20:49 2012
@@ -22,11 +22,12 @@ import java.io.IOException;
 
 import javax.management.JMException;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.jmx.ManagedUtil;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog.DatadirException;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class starts and runs a standalone ZooKeeperServer.
@@ -58,6 +59,10 @@ public class ZooKeeperServerMain {
             LOG.error("Invalid config, exiting abnormally", e);
             System.err.println("Invalid config, exiting abnormally");
             System.exit(2);
+        } catch (DatadirException e) {
+            LOG.error("Unable to access datadir, exiting abnormally", e);
+            System.err.println("Unable to access datadir, exiting abnormally");
+            System.exit(3);
         } catch (Exception e) {
             LOG.error("Unexpected exception, exiting abnormally", e);
             System.exit(1);

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java?rev=1297377&r1=1297376&r2=1297377&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java Tue Mar  6 08:20:49 2012
@@ -58,6 +58,11 @@ public class FileTxnSnapLog {
 
     private static final Logger LOG = LoggerFactory.getLogger(FileTxnSnapLog.class);
 
+    public static final String ZOOKEEPER_DATADIR_AUTOCREATE =
+            "zookeeper.datadir.autocreate";
+
+    public static final String ZOOKEEPER_DATADIR_AUTOCREATE_DEFAULT = "true";
+
     /**
      * This listener helps
      * the external apis calling
@@ -78,15 +83,40 @@ public class FileTxnSnapLog {
     public FileTxnSnapLog(File dataDir, File snapDir) throws IOException {
         this.dataDir = new File(dataDir, version + VERSION);
         this.snapDir = new File(snapDir, version + VERSION);
+
+        // by default create snap/log dirs, but otherwise complain instead
+        // See ZOOKEEPER-1161 for more details
+        boolean enableAutocreate = Boolean.valueOf(
+                System.getProperty(ZOOKEEPER_DATADIR_AUTOCREATE,
+                        ZOOKEEPER_DATADIR_AUTOCREATE_DEFAULT));
+
         if (!this.dataDir.exists()) {
+            if (!enableAutocreate) {
+                throw new DatadirException("Missing data directory "
+                        + this.dataDir
+                        + ", automatic data directory creation is disabled ("
+                        + ZOOKEEPER_DATADIR_AUTOCREATE
+                        + " is false). Please create this directory manually.");
+            }
+
             if (!this.dataDir.mkdirs()) {
-                throw new IOException("Unable to create data directory "
+                throw new DatadirException("Unable to create data directory "
                         + this.dataDir);
             }
         }
         if (!this.snapDir.exists()) {
+            // by default create this directory, but otherwise complain instead
+            // See ZOOKEEPER-1161 for more details
+            if (!enableAutocreate) {
+                throw new DatadirException("Missing snap directory "
+                        + this.snapDir
+                        + ", automatic data directory creation is disabled ("
+                        + ZOOKEEPER_DATADIR_AUTOCREATE
+                        + " is false). Please create this directory manually.");
+            }
+
             if (!this.snapDir.mkdirs()) {
-                throw new IOException("Unable to create snap directory "
+                throw new DatadirException("Unable to create snap directory "
                         + this.snapDir);
             }
         }
@@ -197,11 +227,11 @@ public class FileTxnSnapLog {
         }
 
         /**
-         * This should never happen. A NONODE can never show up in the 
+         * This should never happen. A NONODE can never show up in the
          * transaction logs. This is more indicative of a corrupt transaction
          * log. Refer ZOOKEEPER-1333 for more info.
          */
-        if (rc.err != Code.OK.intValue()) {          
+        if (rc.err != Code.OK.intValue()) {
             if (hdr.getType() == OpCode.create && rc.err == Code.NONODE.intValue()) {
                 int lastSlash = rc.path.lastIndexOf('/');
                 String parentName = rc.path.substring(0, lastSlash);
@@ -238,7 +268,7 @@ public class FileTxnSnapLog {
         LOG.info("Snapshotting: 0x{} to {}", Long.toHexString(lastZxid),
                 snapshotFile);
         snapLog.serialize(dataTree, sessionsWithTimeouts, snapshotFile);
-        
+
     }
 
     /**
@@ -322,4 +352,14 @@ public class FileTxnSnapLog {
         txnLog.close();
         snapLog.close();
     }
+
+    @SuppressWarnings("serial")
+    public static class DatadirException extends IOException {
+        public DatadirException(String msg) {
+            super(msg);
+        }
+        public DatadirException(String msg, Exception e) {
+            super(msg, e);
+        }
+    }
 }

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java?rev=1297377&r1=1297376&r2=1297377&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java Tue Mar  6 08:20:49 2012
@@ -29,6 +29,7 @@ import org.apache.zookeeper.server.ZKDat
 import org.apache.zookeeper.server.DatadirCleanupManager;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog.DatadirException;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 
 /**
@@ -84,6 +85,10 @@ public class QuorumPeerMain {
             LOG.error("Invalid config, exiting abnormally", e);
             System.err.println("Invalid config, exiting abnormally");
             System.exit(2);
+        } catch (DatadirException e) {
+            LOG.error("Unable to access datadir, exiting abnormally", e);
+            System.err.println("Unable to access datadir, exiting abnormally");
+            System.exit(3);
         } catch (Exception e) {
             LOG.error("Unexpected exception, exiting abnormally", e);
             System.exit(1);

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java?rev=1297377&r1=1297376&r2=1297377&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java Tue Mar  6 08:20:49 2012
@@ -21,6 +21,7 @@ package org.apache.zookeeper.test;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
@@ -63,7 +64,7 @@ public class LoadFromLogTest extends ZKT
     protected static final Logger LOG = LoggerFactory.getLogger(LoadFromLogTest.class);
 
     // setting up the quorum has a transaction overhead for creating and closing the session
-    private static final int TRANSACTION_OVERHEAD = 2;	
+    private static final int TRANSACTION_OVERHEAD = 2;
     private static final int TOTAL_TRANSACTIONS = NUM_MESSAGES + TRANSACTION_OVERHEAD;
     private volatile boolean connected;
 
@@ -113,7 +114,7 @@ public class LoadFromLogTest extends ZKT
             Assert.assertTrue("excepting next transaction. expected=" + expectedZxid + ", retreived=" + hdr.getZxid(), (hdr.getZxid() == expectedZxid));
             lastZxid = hdr.getZxid();
         }while(itr.next());
-	
+
         Assert.assertTrue("processed all transactions. " + expectedZxid + " == " + TOTAL_TRANSACTIONS, (expectedZxid == TOTAL_TRANSACTIONS));
     }
 
@@ -122,7 +123,7 @@ public class LoadFromLogTest extends ZKT
 
     public void process(WatchedEvent event) {
     	switch (event.getType()) {
-    	case None:   
+    	case None:
     		switch (event.getState()) {
     		case SyncConnected:
     			connected = true;
@@ -130,7 +131,7 @@ public class LoadFromLogTest extends ZKT
     		case Disconnected:
     			connected = false;
     			break;
-    		default:   
+    		default:
     			break;
     		}
         	break;
@@ -163,11 +164,11 @@ public class LoadFromLogTest extends ZKT
         LOG.info("Attempting to create " + "/test/" + (count - 1));
         doOp(logFile, OpCode.create, "/test/" + (count - 1), dt, zk,
                 zk.stat.getCversion() + 1);
-        
+
         LOG.info("Attempting to create " + "/test/" + (count - 1));
         doOp(logFile, OpCode.multi, "/test/" + (count - 1), dt, zk,
                 zk.stat.getCversion() + 1);
-        
+
         LOG.info("Attempting to create " + "/test/" + (count - 1));
         doOp(logFile, OpCode.multi, "/test/" + (count - 1), dt, zk,
                 -1);
@@ -210,7 +211,7 @@ public class LoadFromLogTest extends ZKT
         else if (type == OpCode.multi) {
             txnHeader = new TxnHeader(0xabcd, 0x123, prevPzxid + 1,
                     System.currentTimeMillis(), OpCode.create);
-            txn = new CreateTxn(path, new byte[0], null, false, cversion);                       
+            txn = new CreateTxn(path, new byte[0], null, false, cversion);
             ArrayList txnList = new ArrayList();
             ByteArrayOutputStream baos = new ByteArrayOutputStream();
             BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
@@ -260,10 +261,10 @@ public class LoadFromLogTest extends ZKT
         Assert.assertTrue("Missing magic number ",
               header.getMagic() == FileTxnLog.TXNLOG_MAGIC);
     }
-    
+
     /**
      * Test we can restore the snapshot that has data ahead of the zxid
-     * of the snapshot file. 
+     * of the snapshot file.
      */
     @Test
     public void testRestore() throws Exception {
@@ -347,7 +348,7 @@ public class LoadFromLogTest extends ZKT
 			}
 
 		}
-		// Verify correctness of data and whether sequential znode creation 
+		// Verify correctness of data and whether sequential znode creation
 		// proceeds correctly after this point
 		String[] children;
 		String path;
@@ -367,10 +368,10 @@ public class LoadFromLogTest extends ZKT
 				(children.length == NUM_MESSAGES));
 		f.shutdown();
 	}
-    
+
     /**
      * Test we can restore a snapshot that has errors and data ahead of the zxid
-     * of the snapshot file. 
+     * of the snapshot file.
      */
     @Test
     public void testRestoreWithTransactionErrors() throws Exception {
@@ -419,7 +420,7 @@ public class LoadFromLogTest extends ZKT
                 zks.getZKDatabase().getDataTreeLastProcessedZxid() - 10);
         LOG.info("Set lastProcessedZxid to "
                 + zks.getZKDatabase().getDataTreeLastProcessedZxid());
-        
+
         // Force snapshot and restore
         zks.takeSnapshot();
         zks.shutdown();
@@ -431,7 +432,48 @@ public class LoadFromLogTest extends ZKT
         f.startup(zks);
         Assert.assertTrue("waiting for server being up ", ClientBase
                 .waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
-        
+
+        f.shutdown();
+    }
+
+    /**
+     * Verify snap/log dir create with/without autocreate enabled.
+     */
+    @Test
+    public void testDatadirAutocreate() throws Exception {
+        ClientBase.setupTestEnv();
+
+        // first verify the default (autocreate on) works
+        File tmpDir = ClientBase.createTmpDir();
+        ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+        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));
+        zks.shutdown();
         f.shutdown();
+        Assert.assertTrue("waiting for server being down ", ClientBase
+                .waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
+
+        try {
+            // now verify autocreate off works
+            System.setProperty(FileTxnSnapLog.ZOOKEEPER_DATADIR_AUTOCREATE, "false");
+
+            tmpDir = ClientBase.createTmpDir();
+            zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+            f = ServerCnxnFactory.createFactory(PORT, -1);
+            f.startup(zks);
+            Assert.assertTrue("waiting for server being up ", ClientBase
+                    .waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
+
+            Assert.fail("Server should not have started without datadir");
+        } catch (IOException e) {
+            LOG.info("Server failed to start - correct behavior " + e);
+        } finally {
+            System.setProperty(FileTxnSnapLog.ZOOKEEPER_DATADIR_AUTOCREATE,
+                FileTxnSnapLog.ZOOKEEPER_DATADIR_AUTOCREATE_DEFAULT);
+        }
     }
-}
\ No newline at end of file
+
+}