You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ma...@apache.org on 2008/12/02 01:13:50 UTC

svn commit: r722308 - in /hadoop/zookeeper/trunk: ./ docs/ src/c/ src/c/src/ src/docs/src/documentation/content/xdocs/ src/java/main/org/apache/zookeeper/ src/java/main/org/apache/zookeeper/server/

Author: mahadev
Date: Mon Dec  1 16:13:50 2008
New Revision: 722308

URL: http://svn.apache.org/viewvc?rev=722308&view=rev
Log:
ZOOKEEPER-64. Log system env information when initializing server and client (pat via mahadev)

Added:
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Environment.java
Modified:
    hadoop/zookeeper/trunk/CHANGES.txt
    hadoop/zookeeper/trunk/docs/zookeeperAdmin.html
    hadoop/zookeeper/trunk/docs/zookeeperAdmin.pdf
    hadoop/zookeeper/trunk/src/c/configure.ac
    hadoop/zookeeper/trunk/src/c/src/zookeeper.c
    hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Mon Dec  1 16:13:50 2008
@@ -41,6 +41,12 @@
 
    ZOOKEEPER-232. fix apache licence header in TestableZookeeper (mahadev)
 
+IMPROVEMENTS:
+   
+   ZOOKEEPER-64. Log system env information when initializing server and
+client (pat via mahadev)
+
+
 Release 3.0.0 - 2008-10-21
 
 Non-backward compatible changes:

Modified: hadoop/zookeeper/trunk/docs/zookeeperAdmin.html
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/docs/zookeeperAdmin.html?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/docs/zookeeperAdmin.html (original)
+++ hadoop/zookeeper/trunk/docs/zookeeperAdmin.html Mon Dec  1 16:13:50 2008
@@ -945,6 +945,14 @@
 
         
 <dt>
+<term>envi</term>
+</dt>
+<dd>
+<p>Print details about serving environment</p>
+</dd>
+
+        
+<dt>
 <term>kill</term>
 </dt>
 <dd>
@@ -954,6 +962,14 @@
 
         
 <dt>
+<term>reqs</term>
+</dt>
+<dd>
+<p>List outstanding requests</p>
+</dd>
+
+        
+<dt>
 <term>ruok</term>
 </dt>
 <dd>
@@ -977,7 +993,7 @@
 <pre class="code">$ echo ruok | nc 127.0.0.1 5111
 imok
 </pre>
-<a name="N10307"></a><a name="sc_dataFileManagement"></a>
+<a name="N10315"></a><a name="sc_dataFileManagement"></a>
 <h3 class="h4">Data File Management</h3>
 <p>ZooKeeper stores its data in a data directory and its transaction
       log in a transaction log directory. By default these two directories are
@@ -985,7 +1001,7 @@
       transaction log files in a separate directory than the data files.
       Throughput increases and latency decreases when transaction logs reside
       on a dedicated log devices.</p>
-<a name="N10310"></a><a name="The+Data+Directory"></a>
+<a name="N1031E"></a><a name="The+Data+Directory"></a>
 <h4>The Data Directory</h4>
 <p>This directory has two files in it:</p>
 <ul>
@@ -1031,14 +1047,14 @@
         idempotent nature of its updates. By replaying the transaction log
         against fuzzy snapshots ZooKeeper gets the state of the system at the
         end of the log.</p>
-<a name="N1034C"></a><a name="The+Log+Directory"></a>
+<a name="N1035A"></a><a name="The+Log+Directory"></a>
 <h4>The Log Directory</h4>
 <p>The Log Directory contains the ZooKeeper transaction logs.
         Before any update takes place, ZooKeeper ensures that the transaction
         that represents the update is written to non-volatile storage. A new
         log file is started each time a snapshot is begun. The log file's
         suffix is the first zxid written to that log.</p>
-<a name="N10356"></a><a name="File+Management"></a>
+<a name="N10364"></a><a name="File+Management"></a>
 <h4>File Management</h4>
 <p>The format of snapshot and log files does not change between
         standalone ZooKeeper servers and different configurations of
@@ -1055,7 +1071,7 @@
         needs the latest complete fuzzy snapshot and the log files from the
         start of that snapshot. The PurgeTxnLog utility implements a simple
         retention policy that administrators can use.</p>
-<a name="N10367"></a><a name="sc_commonProblems"></a>
+<a name="N10375"></a><a name="sc_commonProblems"></a>
 <h3 class="h4">Things to Avoid</h3>
 <p>Here are some common problems you can avoid by configuring
       ZooKeeper correctly:</p>
@@ -1109,7 +1125,7 @@
 </dd>
       
 </dl>
-<a name="N1038B"></a><a name="sc_bestPractices"></a>
+<a name="N10399"></a><a name="sc_bestPractices"></a>
 <h3 class="h4">Best Practices</h3>
 <p>For best results, take note of the following list of good
       Zookeeper practices. <em>[tbd...]</em>

Modified: hadoop/zookeeper/trunk/docs/zookeeperAdmin.pdf
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/docs/zookeeperAdmin.pdf?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
Binary files - no diff available.

Modified: hadoop/zookeeper/trunk/src/c/configure.ac
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/configure.ac?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/configure.ac (original)
+++ hadoop/zookeeper/trunk/src/c/configure.ac Mon Dec  1 16:13:50 2008
@@ -60,7 +60,7 @@
 
 # Checks for header files.
 AC_HEADER_STDC
-AC_CHECK_HEADERS([arpa/inet.h fcntl.h netdb.h netinet/in.h stdlib.h string.h sys/socket.h sys/time.h unistd.h])
+AC_CHECK_HEADERS([arpa/inet.h fcntl.h netdb.h netinet/in.h stdlib.h string.h sys/socket.h sys/time.h unistd.h sys/utsname.h])
 
 # Checks for typedefs, structures, and compiler characteristics.
 AC_C_CONST
@@ -72,7 +72,7 @@
     [#include <poll.h>])
 
 # Checks for library functions.
-AC_CHECK_FUNCS([gethostbyname gettimeofday memmove memset poll socket strchr strdup strerror strtol])
+AC_CHECK_FUNCS([getcwd gethostbyname gethostname getlogin getpwuid_r gettimeofday getuid memmove memset poll socket strchr strdup strerror strtol])
 
 AC_CONFIG_FILES([Makefile])
 AC_OUTPUT

Modified: hadoop/zookeeper/trunk/src/c/src/zookeeper.c
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/src/zookeeper.c?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/src/zookeeper.c (original)
+++ hadoop/zookeeper/trunk/src/c/src/zookeeper.c Mon Dec  1 16:13:50 2008
@@ -49,6 +49,16 @@
 #include <stdarg.h>
 #include <limits.h>
 
+#include "config.h"
+
+#ifdef HAVE_SYS_UTSNAME_H
+#include <sys/utsname.h>
+#endif
+
+#ifdef HAVE_GETPWUID_R
+#include <pwd.h>
+#endif
+
 #define IF_DEBUG(x) if(logLevel==ZOO_LOG_LEVEL_DEBUG) {x;}
 
 const int ZOOKEEPER_WRITE = 1 << 0;
@@ -415,12 +425,68 @@
     return oldWatcher;
 }
 
+static void log_env() {
+  char buf[2048];
+
+  LOG_INFO(("Client environment:zookeeper.version=%s", PACKAGE_STRING));
+
+#ifdef HAVE_GETHOSTNAME
+  gethostname(buf, sizeof(buf));
+  LOG_INFO(("Client environment:host.name=%s", buf));
+#else
+  LOG_INFO(("Client environment:host.name=<not implemented>"));
+#endif
+
+#ifdef HAVE_SYS_UTSNAME_H
+  struct utsname utsname;
+  uname(&utsname);
+  LOG_INFO(("Client environment:os.name=%s", utsname.sysname));
+  LOG_INFO(("Client environment:os.arch=%s", utsname.release));
+  LOG_INFO(("Client environment:os.version=%s", utsname.version));
+#else
+  LOG_INFO(("Client environment:os.name=<not implemented>"));
+  LOG_INFO(("Client environment:os.arch=<not implemented>"));
+  LOG_INFO(("Client environment:os.version=<not implemented>"));
+#endif
+
+#ifdef HAVE_GETLOGIN
+  LOG_INFO(("Client environment:user.name=%s", getlogin()));
+#else
+  LOG_INFO(("Client environment:user.name=<not implemented>"));
+#endif
+
+#if defined(HAVE_GETUID) && defined(HAVE_GETPWUID_R)
+  uid_t uid = getuid();
+  struct passwd pw;
+  struct passwd *pwp;
+  if (!getpwuid_r(uid, &pw, buf, sizeof(buf), &pwp)) {
+    LOG_INFO(("Client environment:user.home=%s", pw.pw_dir));
+  } else {
+    LOG_INFO(("Client environment:user.home=<NA>"));
+  }
+#else
+  LOG_INFO(("Client environment:user.home=<not implemented>"));
+#endif
+
+#ifdef HAVE_GETCWD
+  if (!getcwd(buf, sizeof(buf))) {
+    LOG_INFO(("Client environment:user.dir=<toolong>"));
+  } else {
+    LOG_INFO(("Client environment:user.dir=%s", buf));
+  }
+#else
+  LOG_INFO(("Client environment:user.dir=<not implemented>"));
+#endif
+}
+
 /**
  * Create a zookeeper handle associated with the given host and port.
  */
 zhandle_t *zookeeper_init(const char *host, watcher_fn watcher,
   int recv_timeout, const clientid_t *clientid, void *context, int flags)
 {
+    log_env();
+
     int errnosave;
     zhandle_t *zh = calloc(1, sizeof(*zh));
     if (!zh) {

Modified: hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml (original)
+++ hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml Mon Dec  1 16:13:50 2008
@@ -663,6 +663,14 @@
         </varlistentry>
 
         <varlistentry>
+          <term>envi</term>
+
+          <listitem>
+            <para>Print details about serving environment</para>
+          </listitem>
+        </varlistentry>
+
+        <varlistentry>
           <term>kill</term>
 
           <listitem>
@@ -672,6 +680,14 @@
         </varlistentry>
 
         <varlistentry>
+          <term>reqs</term>
+
+          <listitem>
+            <para>List outstanding requests</para>
+          </listitem>
+        </varlistentry>
+
+        <varlistentry>
           <term>ruok</term>
 
           <listitem>

Added: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Environment.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Environment.java?rev=722308&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Environment.java (added)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Environment.java Mon Dec  1 16:13:50 2008
@@ -0,0 +1,100 @@
+/**
+ * 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;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.log4j.Logger;
+
+/**
+ * Provide insight into the runtime environment.
+ *
+ */
+public class Environment {
+    public static class Entry {
+        private String k;
+        private String v;
+        public Entry(String k, String v) {
+            this.k = k;
+            this.v = v;
+        }
+        public String getKey() { return k; }
+        public String getValue() { return v; }
+        
+        @Override
+        public String toString() {
+            return k + "=" + v;
+        }
+    }
+
+    private static void put(ArrayList<Entry> l, String k, String v) {
+        l.add(new Entry(k,v));
+    }
+
+    public static List<Entry> list() {
+        ArrayList<Entry> l = new ArrayList<Entry>();
+        put(l, "zookeeper.version", Version.getFullVersion());
+
+        try {
+            put(l, "host.name",
+                InetAddress.getLocalHost().getCanonicalHostName().toString());
+        } catch (UnknownHostException e) {
+            put(l, "host.name", "<NA>");
+        }
+
+        put(l, "java.version",
+                System.getProperty("java.version", "<NA>"));
+        put(l, "java.vendor",
+                System.getProperty("java.vendor", "<NA>"));
+        put(l, "java.home",
+                System.getProperty("java.home", "<NA>"));
+        put(l, "java.class.path",
+                System.getProperty("java.class.path", "<NA>"));
+        put(l, "java.library.path",
+                System.getProperty("java.library.path", "<NA>"));
+        put(l, "java.io.tmpdir",
+                System.getProperty("java.io.tmpdir", "<NA>"));
+        put(l, "java.compiler",
+                System.getProperty("java.compiler", "<NA>"));
+        put(l, "os.name",
+                System.getProperty("os.name", "<NA>"));
+        put(l, "os.arch",
+                System.getProperty("os.arch", "<NA>"));
+        put(l, "os.version",
+                System.getProperty("os.version", "<NA>"));
+        put(l, "user.name",
+                System.getProperty("user.name", "<NA>"));
+        put(l, "user.home",
+                System.getProperty("user.home", "<NA>"));
+        put(l, "user.dir",
+                System.getProperty("user.dir", "<NA>"));
+        
+        return l;
+    }
+    
+    public static void logEnv(String msg, Logger log) {
+        List<Entry> env = Environment.list();
+        for (Entry e : env) {
+            log.info(msg + e.toString());
+        }
+    }
+}

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Mon Dec  1 16:13:50 2008
@@ -96,7 +96,14 @@
  *
  */
 public class ZooKeeper {
-    private static final Logger LOG = Logger.getLogger(ZooKeeper.class);
+    private static final Logger LOG;
+    
+    static {
+        LOG = Logger.getLogger(ZooKeeper.class);
+        
+        Environment.logEnv("Client environment:", LOG);
+    }
+
 
     private final ZKWatchManager watchManager = new ZKWatchManager();
 

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java Mon Dec  1 16:13:50 2008
@@ -35,18 +35,19 @@
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
 
-import org.apache.log4j.Logger;
-
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.BinaryOutputArchive;
 import org.apache.jute.Record;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.Environment;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Version;
-import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.proto.AuthPacket;
@@ -604,6 +605,20 @@
                 sendBuffer(ByteBuffer.wrap(sb.toString().getBytes()));
                 k.interestOps(SelectionKey.OP_WRITE);
                 return;
+            } else if (len == enviCmd) {
+                StringBuffer sb = new StringBuffer();
+                
+                List<Environment.Entry> env = Environment.list();
+                
+                sb.append("Environment:\n");
+                for(Environment.Entry e : env) {
+                    sb.append(e.getKey()).append("=").append(e.getValue())
+                        .append("\n");
+                }
+
+                sendBuffer(ByteBuffer.wrap(sb.toString().getBytes()));
+                k.interestOps(SelectionKey.OP_WRITE);
+                return;
             }
         }
         if (len < 0 || len > BinaryInputArchive.maxBuffer) {

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java Mon Dec  1 16:13:50 2008
@@ -47,6 +47,8 @@
     final static int getTraceMaskCmd = ByteBuffer.wrap("gtmk".getBytes())
             .getInt();
 
+    final static int enviCmd = ByteBuffer.wrap("envi".getBytes()).getInt();
+
     final static ByteBuffer imok = ByteBuffer.wrap("imok".getBytes());
 
     public abstract int getSessionTimeout();

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java?rev=722308&r1=722307&r2=722308&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java Mon Dec  1 16:13:50 2008
@@ -36,10 +36,10 @@
 import org.apache.jute.OutputArchive;
 import org.apache.jute.Record;
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.Environment;
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.StatPersisted;
 import org.apache.zookeeper.proto.RequestHeader;
 import org.apache.zookeeper.server.SessionTracker.SessionExpirer;
@@ -57,7 +57,13 @@
  * PrepRequestProcessor -> SyncRequestProcessor -> FinalRequestProcessor
  */
 public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
-    private static final Logger LOG = Logger.getLogger(ZooKeeperServer.class);
+    private static final Logger LOG;
+    
+    static {
+        LOG = Logger.getLogger(ZooKeeperServer.class);
+        
+        Environment.logEnv("Server environment:", LOG);
+    }
 
     /**
      * Create an instance of Zookeeper server