You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/11/14 09:42:53 UTC

svn commit: r1409115 - in /hbase/trunk/hbase-common/src: main/java/org/apache/hadoop/hbase/util/JVM.java main/java/org/apache/hadoop/hbase/util/OSMXBean.java test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java

Author: stack
Date: Wed Nov 14 08:42:52 2012
New Revision: 1409115

URL: http://svn.apache.org/viewvc?rev=1409115&view=rev
Log:
HBASE-6945 Compilation errors when using non-Sun JDKs to build HBase-0.94

Added:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JVM.java
Removed:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OSMXBean.java
Modified:
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JVM.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JVM.java?rev=1409115&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JVM.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JVM.java Wed Nov 14 08:42:52 2012
@@ -0,0 +1,176 @@
+/**
+ * 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.hadoop.hbase.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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.hadoop.util.Shell;
+
+/**
+ * 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.
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class JVM 
+{
+  static final Logger LOG = LoggerFactory.getLogger(JVM.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 JVM () {
+    this.osMbean = ManagementFactory.getOperatingSystemMXBean();
+  }
+ 
+  /**
+   * Check if the OS is unix. 
+   * 
+   * @return whether this is unix or not.
+   */
+  public boolean isUnix() {
+    if (windows) {
+      return false;
+    }
+    return (ibmvendor ? linux : true);
+  }
+
+  /**
+   * Load the implementation of UnixOperatingSystemMXBean for Oracle jvm
+   * and runs the desired method. 
+   * @param mBeanMethodName : method to run from the interface UnixOperatingSystemMXBean
+   * @return the method result
+   */
+  private Long runUnixMXBeanMethod (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 Oracle 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 = runUnixMXBeanMethod("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));
+
+      String openFileDesCount;
+      if ((openFileDesCount = output.readLine()) != null)      
+             return Long.parseLong(openFileDesCount);
+     } 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 Oracle 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 = runUnixMXBeanMethod("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));
+
+      String maxFileDesCount;
+      if ((maxFileDesCount = output.readLine()) != null)      
+        	return Long.parseLong(maxFileDesCount);
+    }   catch (IOException ie) {
+      		LOG.warn("Not able to get the max number of file descriptors", ie);
+    }
+    return -1;
+ }
+}

Modified: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java?rev=1409115&r1=1409114&r2=1409115&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java (original)
+++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java Wed Nov 14 08:42:52 2012
@@ -32,7 +32,7 @@ import java.util.concurrent.ConcurrentHa
 import org.apache.hadoop.hbase.ResourceChecker.Phase;
 import org.junit.runner.notification.RunListener;
 
-import com.sun.management.UnixOperatingSystemMXBean;
+import org.apache.hadoop.hbase.util.JVM;
 
 /**
  * Listen to the test progress and check the usage of:
@@ -85,32 +85,15 @@ public class ResourceCheckerJUnitListene
     }
   }
 
-  /**
-   * On unix, we know how to get the number of open file descriptor. This class allow to share
-   *  the MXBeans code.
-   */
-  abstract static class OSResourceAnalyzer extends ResourceChecker.ResourceAnalyzer {
-    protected static final OperatingSystemMXBean osStats;
-    protected static final UnixOperatingSystemMXBean unixOsStats;
-
-    static {
-      osStats = ManagementFactory.getOperatingSystemMXBean();
-      if (osStats instanceof UnixOperatingSystemMXBean) {
-        unixOsStats = (UnixOperatingSystemMXBean) osStats;
-      } else {
-        unixOsStats = null;
-      }
-    }
-  }
 
-  static class OpenFileDescriptorResourceAnalyzer extends OSResourceAnalyzer {
+  static class OpenFileDescriptorResourceAnalyzer extends ResourceChecker.ResourceAnalyzer {
     @Override
     public int getVal(Phase phase) {
-      if (unixOsStats == null) {
-        return 0;
-      } else {
-        return (int) unixOsStats.getOpenFileDescriptorCount();
-      }
+      JVM jvm = new JVM();
+      if (jvm != null && jvm.isUnix() == true)
+          return (int)jvm.getOpenFileDescriptorCount();
+      else
+           return 0;
     }
 
     @Override
@@ -119,16 +102,16 @@ public class ResourceCheckerJUnitListene
     }
   }
 
-  static class MaxFileDescriptorResourceAnalyzer extends OSResourceAnalyzer {
+  static class MaxFileDescriptorResourceAnalyzer extends ResourceChecker.ResourceAnalyzer {
     @Override
     public int getVal(Phase phase) {
-      if (unixOsStats == null) {
-        return 0;
-      } else {
-        return (int) unixOsStats.getMaxFileDescriptorCount();
-      }
-    }
-  }
+      JVM jvm = new JVM();
+      if (jvm != null && jvm.isUnix() == true)
+           return (int)jvm.getMaxFileDescriptorCount();
+      else
+           return 0;
+     } 
+   }
 
 
   /**