You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ph...@apache.org on 2011/09/17 02:42:10 UTC

svn commit: r1171881 - in /zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/server/ src/java/main/org/apache/zookeeper/server/quorum/ src/java/main/org/apache/zookeeper/server/util/ src/java/test/org/apache/zookeeper/server/ src/java/test/org/apa...

Author: phunt
Date: Sat Sep 17 00:42:10 2011
New Revision: 1171881

URL: http://svn.apache.org/viewvc?rev=1171881&view=rev
Log:
ZOOKEEPER-556. Startup messages should account for common error of missing leading slash in config files (Thomas Koch via phunt)

Added:
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/VerifyingFileFactory.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/util/
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1171881&r1=1171880&r2=1171881&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Sat Sep 17 00:42:10 2011
@@ -33,6 +33,9 @@ IMPROVEMENTS:
   ZOOKEEPER-1176. Remove dead code and basic cleanup in DataTree
   (Thomas Koch via phunt)
 
+  ZOOKEEPER-556. Startup messages should account for common error of
+  missing leading slash in config files (Thomas Koch via phunt)
+
 Release 3.4.0 - 
 
 Non-backward compatible changes:

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java?rev=1171881&r1=1171880&r2=1171881&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java Sat Sep 17 00:42:10 2011
@@ -46,9 +46,9 @@ public class DatadirCleanupManager {
 
     private PurgeTaskStatus purgeTaskStatus = PurgeTaskStatus.NOT_STARTED;
 
-    private final String snapDir;
+    private final File snapDir;
 
-    private final String dataLogDir;
+    private final File dataLogDir;
 
     private final int snapRetainCount;
 
@@ -69,7 +69,7 @@ public class DatadirCleanupManager {
      * @param purgeInterval
      *            purge interval in hours
      */
-    public DatadirCleanupManager(String snapDir, String dataLogDir, int snapRetainCount,
+    public DatadirCleanupManager(File snapDir, File dataLogDir, int snapRetainCount,
             int purgeInterval) {
         this.snapDir = snapDir;
         this.dataLogDir = dataLogDir;
@@ -123,11 +123,11 @@ public class DatadirCleanupManager {
     }
 
     static class PurgeTask extends TimerTask {
-        private String logsDir;
-        private String snapsDir;
+        private File logsDir;
+        private File snapsDir;
         private int snapRetainCount;
 
-        public PurgeTask(String dataDir, String snapDir, int count) {
+        public PurgeTask(File dataDir, File snapDir, int count) {
             logsDir = dataDir;
             snapsDir = snapDir;
             snapRetainCount = count;
@@ -137,7 +137,7 @@ public class DatadirCleanupManager {
         public void run() {
             LOG.info("Purge task started.");
             try {
-                PurgeTxnLog.purge(new File(logsDir), new File(snapsDir), snapRetainCount);
+                PurgeTxnLog.purge(logsDir, snapsDir, snapRetainCount);
             } catch (Exception e) {
                 LOG.error("Error occured while purging.", e);
             }
@@ -159,7 +159,7 @@ public class DatadirCleanupManager {
      * 
      * @return the snapshot directory.
      */
-    public String getSnapDir() {
+    public File getSnapDir() {
         return snapDir;
     }
 
@@ -168,7 +168,7 @@ public class DatadirCleanupManager {
      * 
      * @return the transaction log directory.
      */
-    public String getDataLogDir() {
+    public File getDataLogDir() {
         return dataLogDir;
     }
 

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java?rev=1171881&r1=1171880&r2=1171881&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java Sat Sep 17 00:42:10 2011
@@ -18,6 +18,7 @@
 
 package org.apache.zookeeper.server;
 
+import java.io.File;
 import java.net.InetSocketAddress;
 import java.util.Arrays;
 
@@ -36,8 +37,8 @@ public class ServerConfig {
     //// to update the "conf" 4letter word
     ////
     protected InetSocketAddress clientPortAddress;
-    protected String dataDir;
-    protected String dataLogDir;
+    protected File dataDir;
+    protected File dataLogDir;
     protected int tickTime = ZooKeeperServer.DEFAULT_TICK_TIME;
     protected int maxClientCnxns;
     /** defaults to -1 if not set explicitly */
@@ -58,7 +59,7 @@ public class ServerConfig {
         }
 
         clientPortAddress = new InetSocketAddress(Integer.parseInt(args[0]));
-        dataDir = args[1];
+        dataDir = new File(args[1]);
         dataLogDir = dataDir;
         if (args.length == 3) {
             tickTime = Integer.parseInt(args[2]);
@@ -100,8 +101,8 @@ public class ServerConfig {
     public InetSocketAddress getClientPortAddress() {
         return clientPortAddress;
     }
-    public String getDataDir() { return dataDir; }
-    public String getDataLogDir() { return dataLogDir; }
+    public File getDataDir() { return dataDir; }
+    public File getDataLogDir() { return dataLogDir; }
     public int getTickTime() { return tickTime; }
     public int getMaxClientCnxns() { return maxClientCnxns; }
     /** minimum session timeout in milliseconds, -1 if unset */

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=1171881&r1=1171880&r2=1171881&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 Sat Sep 17 00:42:10 2011
@@ -100,8 +100,7 @@ public class ZooKeeperServerMain {
             // create a file logger url from the command line args
             ZooKeeperServer zkServer = new ZooKeeperServer();
 
-            FileTxnSnapLog ftxn = new FileTxnSnapLog(new
-                   File(config.dataLogDir), new File(config.dataDir));
+            FileTxnSnapLog ftxn = new FileTxnSnapLog(config.dataLogDir, config.dataDir);
             zkServer.setTxnLogFactory(ftxn);
             zkServer.setTickTime(config.tickTime);
             zkServer.setMinSessionTimeout(config.minSessionTimeout);

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java?rev=1171881&r1=1171880&r2=1171881&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java Sat Sep 17 00:42:10 2011
@@ -41,13 +41,14 @@ import org.apache.zookeeper.server.quoru
 import org.apache.zookeeper.server.quorum.flexible.QuorumHierarchical;
 import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
 import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+import org.apache.zookeeper.server.util.VerifyingFileFactory;
 
 public class QuorumPeerConfig {
     private static final Logger LOG = LoggerFactory.getLogger(QuorumPeerConfig.class);
 
     protected InetSocketAddress clientPortAddress;
-    protected String dataDir;
-    protected String dataLogDir;
+    protected File dataDir;
+    protected File dataLogDir;
     protected int tickTime = ZooKeeperServer.DEFAULT_TICK_TIME;
     protected int maxClientCnxns = 60;
     /** defaults to -1 if not set explicitly */
@@ -73,7 +74,7 @@ public class QuorumPeerConfig {
     protected int purgeInterval = 0;
 
     protected LearnerType peerType = LearnerType.PARTICIPANT;
-    
+
     /**
      * Minimum snapshot retain count.
      * @see org.apache.zookeeper.server.PurgeTxnLog#purge(File, File, int)
@@ -96,15 +97,13 @@ public class QuorumPeerConfig {
      * @throws ConfigException error processing configuration
      */
     public void parse(String path) throws ConfigException {
-        File configFile = new File(path);
-
-        LOG.info("Reading configuration from: " + configFile);
+        LOG.info("Reading configuration from: " + path);
 
         try {
-            if (!configFile.exists()) {
-                throw new IllegalArgumentException(configFile.toString()
-                        + " file is missing");
-            }
+            File configFile = (new VerifyingFileFactory.Builder(LOG)
+                .warnForRelativePath()
+                .failForNonExistingPath()
+                .build()).create(path);
 
             Properties cfg = new Properties();
             FileInputStream in = new FileInputStream(configFile);
@@ -132,13 +131,14 @@ public class QuorumPeerConfig {
     throws IOException, ConfigException {
         int clientPort = 0;
         String clientPortAddress = null;
+        VerifyingFileFactory vff = new VerifyingFileFactory.Builder(LOG).warnForRelativePath().build();
         for (Entry<Object, Object> entry : zkProp.entrySet()) {
             String key = entry.getKey().toString().trim();
             String value = entry.getValue().toString().trim();
             if (key.equals("dataDir")) {
-                dataDir = value;
+                dataDir = vff.create(value);
             } else if (key.equals("dataLogDir")) {
-                dataLogDir = value;
+                dataLogDir = vff.create(value);
             } else if (key.equals("clientPort")) {
                 clientPort = Integer.parseInt(value);
             } else if (key.equals("clientPortAddress")) {
@@ -192,11 +192,11 @@ public class QuorumPeerConfig {
                     InetSocketAddress electionAddr = new InetSocketAddress(
                             parts[0], Integer.parseInt(parts[2]));
                     LearnerType type = LearnerType.PARTICIPANT;
-                    if (parts[3].toLowerCase().equals("observer")) {
+                    if (parts[3].equalsIgnoreCase("observer")) {
                         type = LearnerType.OBSERVER;
                         observers.put(Long.valueOf(sid), new QuorumServer(sid, addr,
                                 electionAddr,type));
-                    } else if (parts[3].toLowerCase().equals("participant")) {
+                    } else if (parts[3].equalsIgnoreCase("participant")) {
                         type = LearnerType.PARTICIPANT;
                         servers.put(Long.valueOf(sid), new QuorumServer(sid, addr,
                                 electionAddr,type));
@@ -227,7 +227,7 @@ public class QuorumPeerConfig {
                 System.setProperty("zookeeper." + key, value);
             }
         }
-        
+
         // Reset to MIN_SNAP_RETAIN_COUNT if invalid (less than 3)
         // PurgeTxnLog.purge(File, File, int) will not allow to purge less
         // than 3.
@@ -243,7 +243,7 @@ public class QuorumPeerConfig {
         if (dataLogDir == null) {
             dataLogDir = dataDir;
         } else {
-            if (!new File(dataLogDir).isDirectory()) {
+            if (!dataLogDir.isDirectory()) {
                 throw new IllegalArgumentException("dataLogDir " + dataLogDir
                         + " is missing.");
             }
@@ -251,11 +251,11 @@ public class QuorumPeerConfig {
         if (clientPort == 0) {
             throw new IllegalArgumentException("clientPort is not set");
         }
-        if (clientPortAddress != null) {
+        if (clientPortAddress == null) {
+            this.clientPortAddress = new InetSocketAddress(clientPort);
+        } else {
             this.clientPortAddress = new InetSocketAddress(
                     InetAddress.getByName(clientPortAddress), clientPort);
-        } else {
-            this.clientPortAddress = new InetSocketAddress(clientPort);
         }
 
         if (tickTime == 0) {
@@ -338,7 +338,7 @@ public class QuorumPeerConfig {
             // Now add observers to servers, once the quorums have been
             // figured out
             servers.putAll(observers);
-    
+
             File myIdFile = new File(dataDir, "myid");
             if (!myIdFile.exists()) {
                 throw new IllegalArgumentException(myIdFile.toString()
@@ -358,7 +358,7 @@ public class QuorumPeerConfig {
                 throw new IllegalArgumentException("serverid " + myIdString
                         + " is not a number");
             }
-            
+
             // Warn about inconsistent peer type
             LearnerType roleByServersList = observers.containsKey(serverId) ? LearnerType.OBSERVER
                     : LearnerType.PARTICIPANT;
@@ -366,15 +366,15 @@ public class QuorumPeerConfig {
                 LOG.warn("Peer type from servers list (" + roleByServersList
                         + ") doesn't match peerType (" + peerType
                         + "). Defaulting to servers list.");
-    
+
                 peerType = roleByServersList;
             }
         }
     }
 
     public InetSocketAddress getClientPortAddress() { return clientPortAddress; }
-    public String getDataDir() { return dataDir; }
-    public String getDataLogDir() { return dataLogDir; }
+    public File getDataDir() { return dataDir; }
+    public File getDataLogDir() { return dataLogDir; }
     public int getTickTime() { return tickTime; }
     public int getMaxClientCnxns() { return maxClientCnxns; }
     public int getMinSessionTimeout() { return minSessionTimeout; }

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=1171881&r1=1171880&r2=1171881&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 Sat Sep 17 00:42:10 2011
@@ -133,8 +133,8 @@ public class QuorumPeerMain {
           quorumPeer = new QuorumPeer();
           quorumPeer.setClientPortAddress(config.getClientPortAddress());
           quorumPeer.setTxnFactory(new FileTxnSnapLog(
-                      new File(config.getDataLogDir()),
-                      new File(config.getDataDir())));
+                      config.getDataLogDir(),
+                      config.getDataDir()));
           quorumPeer.setQuorumPeers(config.getServers());
           quorumPeer.setElectionType(config.getElectionAlg());
           quorumPeer.setMyid(config.getServerId());

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/VerifyingFileFactory.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/VerifyingFileFactory.java?rev=1171881&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/VerifyingFileFactory.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/VerifyingFileFactory.java Sat Sep 17 00:42:10 2011
@@ -0,0 +1,82 @@
+/**
+ * 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.util;
+
+import java.io.File;
+
+import org.slf4j.Logger;
+
+public final class VerifyingFileFactory {
+
+    private final boolean warnForRelativePath;
+    private final boolean failForNonExistingPath;
+    private final Logger log;
+
+    public VerifyingFileFactory(Builder builder){
+        warnForRelativePath = builder.warnForRelativePathOption;
+        failForNonExistingPath  = builder.failForNonExistingPathOption;
+        log = builder.log;
+        assert(log != null);
+    }
+
+    public File create(String path) {
+        File file = new File(path);
+        if(warnForRelativePath) doWarnForRelativePath(file);
+        if(failForNonExistingPath) doFailForNonExistingPath(file);
+        return file;
+    }
+
+    private void doFailForNonExistingPath(File file) {
+        if (!file.exists()) {
+            throw new IllegalArgumentException(file.toString()
+                    + " file is missing");
+        }
+    }
+
+    private void doWarnForRelativePath(File file) {
+        if(file.isAbsolute()) return;
+        if(file.getPath().substring(0, 2).equals("."+File.separator)) return;
+        log.warn(file.getPath()+" is relative. Prepend ."
+                +File.separator+" to indicate that you're sure!");
+    }
+
+    public static class Builder {
+        private boolean warnForRelativePathOption = false;
+        private boolean failForNonExistingPathOption = false;
+        private final Logger log;
+
+        public Builder(Logger log){
+            this.log = log;
+        }
+
+        public Builder warnForRelativePath() {
+            warnForRelativePathOption = true;
+            return this;
+        }
+
+        public Builder failForNonExistingPath() {
+            failForNonExistingPathOption = true;
+            return this;
+        }
+
+        public VerifyingFileFactory build() {
+            return new VerifyingFileFactory(this);
+        }
+    }
+}

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java?rev=1171881&r1=1171880&r2=1171881&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java Sat Sep 17 00:42:10 2011
@@ -34,14 +34,14 @@ import org.junit.Test;
 public class DatadirCleanupManagerTest {
 
     private DatadirCleanupManager purgeMgr;
-    private String snapDir;
-    private String dataLogDir;
+    private File snapDir;
+    private File dataLogDir;
 
     @Before
     public void setUp() throws Exception {
         File dataDir = ClientBase.createTmpDir();
-        snapDir = dataDir.getAbsolutePath();
-        dataLogDir = dataDir.getAbsolutePath();
+        snapDir = dataDir;
+        dataLogDir = dataDir;
     }
 
     @Test

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java?rev=1171881&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java Sat Sep 17 00:42:10 2011
@@ -0,0 +1,68 @@
+/**
+ * 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.util;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class VerifyingFileFactoryTest {
+
+    private Logger log;
+
+    @Before
+    public void setUp(){
+        log = LoggerFactory.getLogger("TODO: Mock Logging");
+    }
+
+    @Test
+    public void testForWarningOnRelativePath() {
+        VerifyingFileFactory vff = new VerifyingFileFactory.Builder(log)
+            .warnForRelativePath().build();
+        vff.create("a/relative/path");
+        // assertTrue(log.hasWarned);
+    }
+
+    @Test
+    public void testForNoWarningOnIntendedRelativePath() {
+        VerifyingFileFactory vff = new VerifyingFileFactory.Builder(log)
+            .warnForRelativePath().build();
+        vff.create("./an/intended/relative/path");
+        // assertFalse(log.hasWarned);
+    }
+
+    @Test(expected=IllegalArgumentException.class)
+    public void testForFailForNonExistingPath() {
+        VerifyingFileFactory vff = new VerifyingFileFactory.Builder(log)
+            .failForNonExistingPath().build();
+        vff.create("/I/H0p3/this/path/d035/n0t/ex15t");
+    }
+
+    @Test
+    public void testFileHasCorrectPath() {
+        File file = new File("/some/path");
+        VerifyingFileFactory vff = new VerifyingFileFactory.Builder(log).build();
+        assertEquals(file, vff.create(file.getPath()));
+    }
+}