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 2012/11/28 08:43:28 UTC

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

Author: phunt
Date: Wed Nov 28 07:43:26 2012
New Revision: 1414566

URL: http://svn.apache.org/viewvc?rev=1414566&view=rev
Log:
ZOOKEEPER-1474. Cannot build Zookeeper with IBM Java: use of Sun MXBean classes (Adalberto Medeiros via phunt)

Added:
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/OSMXBean.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OSMXBeanTest.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1414566&r1=1414565&r2=1414566&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Wed Nov 28 07:43:26 2012
@@ -267,6 +267,9 @@ BUGFIXES:
   ZOOKEEPER-1590. Patch to add zk.updateServerList(newServerList)
   broke the build (fpj via phunt)
 
+  ZOOKEEPER-1474. Cannot build Zookeeper with IBM Java: use of Sun
+  MXBean classes (Adalberto Medeiros via phunt)
+
 IMPROVEMENTS:
 
   ZOOKEEPER-1170. Fix compiler (eclipse) warnings: unused imports,

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java?rev=1414566&r1=1414565&r2=1414566&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java Wed Nov 28 07:43:26 2012
@@ -23,8 +23,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.Writer;
-import java.lang.management.ManagementFactory;
-import java.lang.management.OperatingSystemMXBean;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
@@ -51,7 +49,7 @@ import org.apache.zookeeper.proto.Watche
 import org.apache.zookeeper.server.quorum.Leader;
 import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
 import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer;
-import com.sun.management.UnixOperatingSystemMXBean;
+import org.apache.zookeeper.server.util.OSMXBean;
 
 /**
  * This class handles communication with clients using NIO. There is one per
@@ -760,12 +758,10 @@ public class NIOServerCnxn extends Serve
             print("ephemerals_count", zkdb.getDataTree().getEphemeralsCount());
             print("approximate_data_size", zkdb.getDataTree().approximateDataSize());
 
-            OperatingSystemMXBean osMbean = ManagementFactory.getOperatingSystemMXBean();
-            if(osMbean != null && osMbean instanceof UnixOperatingSystemMXBean) {
-                UnixOperatingSystemMXBean unixos = (UnixOperatingSystemMXBean)osMbean;
-
-                print("open_file_descriptor_count", unixos.getOpenFileDescriptorCount());
-                print("max_file_descriptor_count", unixos.getMaxFileDescriptorCount());
+            OSMXBean osMbean = new OSMXBean();
+            if (osMbean != null && osMbean.getUnix() == true) {
+                print("open_file_descriptor_count", osMbean.getOpenFileDescriptorCount());
+                print("max_file_descriptor_count", osMbean.getMaxFileDescriptorCount());
             }
 
             if(stats.getServerState().equals("leader")) {

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java?rev=1414566&r1=1414565&r2=1414566&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java Wed Nov 28 07:43:26 2012
@@ -25,8 +25,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.Writer;
-import java.lang.management.ManagementFactory;
-import java.lang.management.OperatingSystemMXBean;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.nio.ByteBuffer;
@@ -49,14 +47,13 @@ import org.apache.zookeeper.proto.Watche
 import org.apache.zookeeper.server.quorum.Leader;
 import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
 import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer;
+import org.apache.zookeeper.server.util.OSMXBean;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.Channel;
 import org.jboss.netty.channel.ChannelFuture;
 import org.jboss.netty.channel.MessageEvent;
 
-import com.sun.management.UnixOperatingSystemMXBean;
-
 public class NettyServerCnxn extends ServerCnxn {
     Logger LOG = LoggerFactory.getLogger(NettyServerCnxn.class);
     Channel channel;
@@ -571,14 +568,12 @@ public class NettyServerCnxn extends Ser
             print("ephemerals_count", zkdb.getDataTree().getEphemeralsCount());
             print("approximate_data_size", zkdb.getDataTree().approximateDataSize());
 
-            OperatingSystemMXBean osMbean = ManagementFactory.getOperatingSystemMXBean();
-            if(osMbean != null && osMbean instanceof UnixOperatingSystemMXBean) {
-                UnixOperatingSystemMXBean unixos = (UnixOperatingSystemMXBean)osMbean;
-
-                print("open_file_descriptor_count", unixos.getOpenFileDescriptorCount());
-                print("max_file_descriptor_count", unixos.getMaxFileDescriptorCount());
+            OSMXBean osMbean = new OSMXBean();
+            if (osMbean != null && osMbean.getUnix() == true) {
+                print("open_file_descriptor_count", osMbean.getOpenFileDescriptorCount());
+                print("max_file_descriptor_count", osMbean.getMaxFileDescriptorCount());
             }
-
+          
             if(stats.getServerState().equals("leader")) {
                 Leader leader = ((LeaderZooKeeperServer)zkServer).getLeader();
 

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/OSMXBean.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/OSMXBean.java?rev=1414566&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/OSMXBean.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/OSMXBean.java Wed Nov 28 07:43:26 2012
@@ -0,0 +1,182 @@
+/**
+ * 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.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.lang.management.RuntimeMXBean;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.lang.reflect.Method;
+
+/**
+ * This class is a wrapper for the implementation of
+ * com.sun.management.UnixOperatingSystemMXBean
+ * It will decide to use the sun api or its own implementation
+ * depending on the runtime (vendor) used.
+ */
+public class OSMXBean
+{
+    static final Logger LOG = LoggerFactory.getLogger(OSMXBean.class);
+
+    private OperatingSystemMXBean osMbean;
+
+    private static final boolean ibmvendor =
+        System.getProperty("java.vendor").contains("IBM");
+    private static final boolean windows = 
+        System.getProperty("os.name").startsWith("Windows");
+    private static final boolean linux =
+        System.getProperty("os.name").startsWith("Linux");
+
+    /**
+     * Constructor. Get the running Operating System instance
+     */
+    public OSMXBean () {
+        this.osMbean = ManagementFactory.getOperatingSystemMXBean();
+    }
+ 
+    /**
+     * Check if the OS is unix. If using the IBM java runtime, this
+     * will only work for linux.
+     * 
+     * @return whether this is unix or not.
+     */
+    public boolean getUnix() {
+        if (windows) {
+            return false;
+        }
+        return (ibmvendor ? linux : true);
+    }
+
+    /**
+     * Load the implementation of UnixOperatingSystemMXBean for sun jvm
+     * and runs the desired method. 
+     * @param mBeanMethodName : method to run from the interface UnixOperatingSystemMXBean
+     * @return the method result
+     */
+    private Long getOSUnixMXBeanMethod (String mBeanMethodName)
+    {
+        Object unixos;
+        Class<?> classRef;
+        Method mBeanMethod;
+
+        try {
+            classRef = Class.forName("com.sun.management.UnixOperatingSystemMXBean");
+            if (classRef.isInstance(osMbean)) {
+                mBeanMethod = classRef.getDeclaredMethod(mBeanMethodName,
+                new Class[0]);
+                unixos = classRef.cast(osMbean);
+                return (Long)mBeanMethod.invoke(unixos);
+            }
+        } catch(Exception e) {
+            LOG.warn("Not able to load class or method for com.sun.managment.UnixOperatingSystemMXBean.", e);
+        }
+        return null;
+    }
+
+    /**
+     * Get the number of opened filed descriptor for the runtime jvm.
+     * If sun java, it will use the com.sun.management interfaces.
+     * Otherwise, this methods implements it (linux only).  
+     * @return number of open file descriptors for the jvm
+     */
+    public long getOpenFileDescriptorCount() 
+    {
+        Long ofdc;
+    
+        if (!ibmvendor) {
+            ofdc = getOSUnixMXBeanMethod("getOpenFileDescriptorCount");
+            return (ofdc != null ? ofdc.longValue () : -1);
+        }
+        
+        try {
+            //need to get the PID number of the process first
+            RuntimeMXBean rtmbean = ManagementFactory.getRuntimeMXBean();
+            String rtname = rtmbean.getName();
+            String[] pidhost = rtname.split("@");
+
+            //using linux bash commands to retrieve info
+            Process p = Runtime.getRuntime().exec(
+                    new String[] { "bash", "-c",
+                    "ls /proc/" + pidhost[0] + "/fdinfo | wc -l" });
+            InputStream in = p.getInputStream();
+            BufferedReader output = new BufferedReader(
+                    new InputStreamReader(in));
+
+            try {
+                String openFileDesCount;
+                if ((openFileDesCount = output.readLine()) != null) {
+                    return Long.parseLong(openFileDesCount);
+                }
+            } finally {
+                if (output != null) {
+                    output.close();
+                }
+            }
+        } catch (IOException ie) {
+            LOG.warn("Not able to get the number of open file descriptors", ie);
+        }
+        return -1;
+    }
+
+    /**
+     * Get the number of the maximum file descriptors the system can use.
+     * If sun java, it will use the com.sun.management interfaces.
+     * Otherwise, this methods implements it (linux only).  
+     * @return max number of file descriptors the operating system can use.
+     */
+    public long getMaxFileDescriptorCount()
+    {
+        Long mfdc;
+
+        if (!ibmvendor) {
+            mfdc = getOSUnixMXBeanMethod("getMaxFileDescriptorCount");
+            return (mfdc != null ? mfdc.longValue () : -1);
+        }
+        
+        try {
+            //using linux bash commands to retrieve info
+            Process p = Runtime.getRuntime().exec(
+                    new String[] { "bash", "-c", "ulimit -n" });
+            InputStream in = p.getInputStream();
+            BufferedReader output = new BufferedReader(
+                    new InputStreamReader(in));
+
+            try {
+                String maxFileDesCount;
+                if ((maxFileDesCount = output.readLine()) != null) {
+                    return Long.parseLong(maxFileDesCount);
+                }
+            } finally {
+                if (output != null) {
+                    output.close();
+                }
+            }
+        } catch (IOException ie) {
+            LOG.warn("Not able to get the max number of file descriptors", ie);
+        }
+        return -1;
+    }  
+}

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OSMXBeanTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OSMXBeanTest.java?rev=1414566&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OSMXBeanTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OSMXBeanTest.java Wed Nov 28 07:43:26 2012
@@ -0,0 +1,70 @@
+/**
+ * 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.test;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.zookeeper.server.util.OSMXBean;
+
+public class OSMXBeanTest {
+    
+    private OSMXBean osMbean;
+    private Long ofdc = 0L;
+    private Long mfdc = 0L;
+    protected static final Logger LOG = LoggerFactory.getLogger(OSMXBeanTest.class);
+
+    @Before
+    public void initialize() {
+        this.osMbean = new OSMXBean();
+        Assert.assertNotNull("Could not initialize OSMXBean object!", osMbean);
+    }
+    
+    @Test
+    public final void testGetUnix() {
+        boolean isUnix = osMbean.getUnix();
+        if (!isUnix) {
+        	LOG.info("Running in a Windows system! Output won't be printed!");
+        } else {
+        	LOG.info("Running in a Unix or Linux system!");
+        }
+    }
+
+    @Test
+    public final void testGetOpenFileDescriptorCount() {
+        if (osMbean != null && osMbean.getUnix() == true) {
+            ofdc = osMbean.getOpenFileDescriptorCount();
+            LOG.info("open fdcount is: " + ofdc);
+        }   
+        Assert.assertFalse("The number of open file descriptor is negative",(ofdc < 0));
+    }
+
+    @Test
+    public final void testGetMaxFileDescriptorCount() {
+        if (osMbean != null && osMbean.getUnix() == true) {
+            mfdc = osMbean.getMaxFileDescriptorCount();
+            LOG.info("max fdcount is: " + mfdc);
+        }
+        Assert.assertFalse("The max file descriptor number is negative",(mfdc < 0));
+    }
+
+}