You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/04/26 20:18:32 UTC

svn commit: r1590272 - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/coprocessor/endpoints/ main/java/org/apache/hadoop/hbase/thrift/ main/java/org/apache/hadoop/hbase/util/ test/java/org/apache/hadoop/hbase/thrift/

Author: liyin
Date: Sat Apr 26 18:18:32 2014
New Revision: 1590272

URL: http://svn.apache.org/r1590272
Log:
[master] Change of CuttingClassLoader and ThrfitResultSerializer/2 for loading different version of Thrift

Author: daviddeng

Summary:
`instance` in `ThrfitResultSerializer2` is declared as Object and used with reflection
`protocolClass` is loaded with same classloader of `ThrfitResultSerializer`
Add some logs

Test Plan: `TestThrfitResultSerializer`

Reviewers: rshroff, manukranthk, liyintang, gauravm, adela

Reviewed By: adela

CC: hbase-eng@, ssl-diffs@, arice

Differential Revision: https://phabricator.fb.com/D1294940

Task ID: 3547474

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/HTableEndpointClient.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/CuttingClassLoader.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer2.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtils.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftResultSerializer.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/HTableEndpointClient.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/HTableEndpointClient.java?rev=1590272&r1=1590271&r2=1590272&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/HTableEndpointClient.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/HTableEndpointClient.java Sat Apr 26 18:18:32 2014
@@ -104,7 +104,7 @@ public class HTableEndpointClient implem
         results.put(region, caller.call(ep));
       }
     } catch (UndeclaredThrowableException e) {
-      ExceptionUtils.throwIOExcetion(e.getUndeclaredThrowable());
+      throw ExceptionUtils.toIOException(e.getUndeclaredThrowable());
     }
 
     return results;

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/CuttingClassLoader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/CuttingClassLoader.java?rev=1590272&r1=1590271&r2=1590272&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/CuttingClassLoader.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/CuttingClassLoader.java Sat Apr 26 18:18:32 2014
@@ -21,32 +21,63 @@ package org.apache.hadoop.hbase.thrift;
 
 import java.net.URL;
 import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 /**
  * The classloader allows the searching firstly in some path, if not found
  * search in parent.
+ *
+ * The class is useful for loading classes whose dependency is incompatible with
+ * the current classloader.
+ *
+ * skipClasses are set to make those classes to be loaded with parent loader
+ * and thus can be used by classes loaded by both classloaders. They are
+ * commonly classes used in the interface. If the class doesn't exist in the
+ * cutting path, it is not necessary to put in skipClasses.
+ *
+ * Class loaded with different classloader is not assignable to each other, use
+ * reflection to call its methods.
  */
 public class CuttingClassLoader extends URLClassLoader {
-  public CuttingClassLoader(URL[] urls, ClassLoader parent) {
+  private static Log LOG = LogFactory.getLog(CuttingClassLoader.class);
+  private Set<String> skipClasses = new HashSet<>();
+
+  public CuttingClassLoader(URL[] urls, ClassLoader parent,
+      String... skipClasses) {
     super(urls, parent);
+    Collections.addAll(this.skipClasses, skipClasses);
   }
 
   @Override
   protected synchronized Class<?> loadClass(String name, boolean resolve)
       throws ClassNotFoundException {
+    if (skipClasses.contains(name)) {
+      // Find in parent
+      LOG.debug("Finding " + name + " in parent path!");
+      return super.loadClass(name, resolve);
+    }
+
     // Find in loaded classes
     Class<?> clazz = findLoadedClass(name);
     if (clazz != null) {
+      LOG.debug("Found " + name + " in loaded class!");
       return clazz;
     }
 
     try {
       // Find in cutting paths
+      LOG.debug("Finding " + name + " in cutting path!");
       return findClass(name);
     } catch (ClassNotFoundException e) {
     }
 
     // Find in parent
+    LOG.debug("Finding " + name + " in parent path!");
     return super.loadClass(name, resolve);
   }
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer.java?rev=1590272&r1=1590271&r2=1590272&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer.java Sat Apr 26 18:18:32 2014
@@ -28,8 +28,8 @@ import org.apache.hadoop.conf.Configurab
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapred.TableInputFormat;
 import org.apache.hadoop.hbase.thrift.generated.TRowResult;
+import org.apache.hadoop.hbase.util.ExceptionUtils;
 import org.apache.hadoop.io.serializer.Serializer;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TCompactProtocol;
@@ -38,18 +38,18 @@ import org.apache.thrift.transport.TIOSt
 import org.apache.thrift.transport.TTransport;
 
 /**
- * A serializer for use with {@link TableInputFormat}. Serializes results as TResult.
+ * A serializer for use with TableInputFormat. Serializes results as TResult.
  */
 public class ThriftResultSerializer implements Serializer<Object>, Configurable {
 
   private static final Log LOG = LogFactory.getLog(ThriftResultSerializer.class);
 
-  public static final String PROTOCOL_CONF_KEY = "hbase.thrift.result.serializer.protocol.class";
-  
+  public static final String PROTOCOL_CONF_KEY =
+      "hbase.thrift.result.serializer.protocol.class";
+
   private OutputStream out;
-  
-  @SuppressWarnings("rawtypes")
-  private Class protocolClass = TCompactProtocol.class;
+
+  private Class<? extends TProtocol> protocolClass = TCompactProtocol.class;
 
   private TProtocol prot;
   private DataOutput dataOut;
@@ -63,15 +63,15 @@ public class ThriftResultSerializer impl
     transport = new TIOStreamTransport(out);
 
     LOG.info("Using Thrift protocol: " + protocolClass.getName());
-    
+
     try {
-      Constructor<TProtocol> constructor =
-          protocolClass.getConstructor(new Class[] { TTransport.class });
+      Constructor<? extends TProtocol> constructor =
+          protocolClass.getConstructor(TTransport.class);
       prot = constructor.newInstance(transport);
     } catch (Exception ex) {
-      throw new RuntimeException(ex);
+      throw ExceptionUtils.toIOException(ex);
     }
-    
+
     if (out instanceof DataOutput) {
       dataOut = (DataOutput) out;
     } else {
@@ -79,10 +79,9 @@ public class ThriftResultSerializer impl
     }
   }
 
-  @SuppressWarnings("rawtypes")
   @Override
   public void serialize(Object t) throws IOException {
-    Class klass = t.getClass();
+    Class<?> klass = t.getClass();
     if (klass == Result.class) {
       Result result = (Result) t;
       TRowResult tResult = ThriftUtilities.oneRowResult(result);
@@ -105,10 +104,21 @@ public class ThriftResultSerializer impl
     out.close();
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   public void setConf(Configuration conf) {
     this.conf = conf;
-    protocolClass = conf.getClass(PROTOCOL_CONF_KEY, protocolClass);
+    String protoclCLassName =
+        conf.get(PROTOCOL_CONF_KEY, protocolClass.getName());
+    try {
+      protocolClass =
+          (Class<? extends TProtocol>) Class.forName(protoclCLassName);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+
+    LOG.info("Classloader of " + protocolClass.getName() + " is "
+        + protocolClass.getClassLoader());
   }
 
   @Override

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer2.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer2.java?rev=1590272&r1=1590271&r2=1590272&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer2.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer2.java Sat Apr 26 18:18:32 2014
@@ -22,12 +22,20 @@ package org.apache.hadoop.hbase.thrift;
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.lang.reflect.Method;
 import java.net.URL;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.ExceptionUtils;
 import org.apache.hadoop.io.serializer.Serializer;
 
+
 /**
  * A wrapper class which loads <code>ThriftResultSerializer</code> class with a
  * <code>CuttingClassLoader</code>.
@@ -37,16 +45,25 @@ import org.apache.hadoop.io.serializer.S
  */
 public class ThriftResultSerializer2 implements Serializer<Object>,
     Configurable {
-  ThriftResultSerializer instance;
+  private static Log LOG = LogFactory.getLog(ThriftResultSerializer2.class);
+  // ThriftResultSerializer instance;
+  Object instance;
   /**
    * Key to the configuration of classpath to cut
    */
   public static final String CUTTING_CLASSPATH_KEY =
       "hbase.thrift.result.serializer.cutting.classpath";
 
+  private Method getConfMethod;
+  private Method openMethod;
+  private Method serializeMethod;
+  private Method closeMethod;
+
+  @SuppressWarnings("resource")
   @Override
   public void setConf(Configuration conf) {
     String classPath = conf.get(CUTTING_CLASSPATH_KEY, "");
+    LOG.info(CUTTING_CLASSPATH_KEY + " is set to " + classPath);
     try {
       // make the URL array of cutting paths
       String[] parts = classPath.split(File.pathSeparator);
@@ -56,37 +73,69 @@ public class ThriftResultSerializer2 imp
       }
 
       // Create the ClassLoader
+      // Skip all classes in the interface so that both side can use it.
       ClassLoader classLoader = new CuttingClassLoader(urls,
- this.getClass().getClassLoader());
+              ThriftResultSerializer.class.getClassLoader(),
+              Result.class.getName(), ImmutableBytesWritable.class.getName(),
+              KeyValue.class.getName(), Configuration.class.getName());
       // and load the class.
-      Class<?> cls = Class.forName(ThriftResultSerializer.class.getName(),
-          true, classLoader);
+      Class<?> cls =
+          classLoader.loadClass(ThriftResultSerializer.class.getName());
       // Then create the new instance.
-      instance = (ThriftResultSerializer) cls.newInstance();
+      instance = cls.newInstance();
     } catch (Exception e) {
-      throw new RuntimeException(e);
+      throw ExceptionUtils.toRuntimeException(e);
     }
 
-    instance.setConf(conf);
+    try {
+      getConfMethod = instance.getClass().getMethod("getConf");
+      openMethod = instance.getClass().getMethod("open", OutputStream.class);
+      serializeMethod =
+          instance.getClass().getMethod("serialize", Object.class);
+      closeMethod = instance.getClass().getMethod("close");
+
+      Method setConfMethod =
+          instance.getClass().getMethod("setConf", Configuration.class);
+      setConfMethod.invoke(instance, conf);
+    } catch (Exception e) {
+      throw ExceptionUtils.toRuntimeException(e);
+    }
   }
 
   @Override
   public Configuration getConf() {
-    return instance.getConf();
+    try {
+      return (Configuration) getConfMethod.invoke(instance);
+    } catch (Exception e) {
+      throw ExceptionUtils.toRuntimeException(e);
+    }
   }
 
   @Override
   public void open(OutputStream out) throws IOException {
-    instance.open(out);
+    try {
+      openMethod.invoke(instance, out);
+    } catch (Exception e) {
+      throw ExceptionUtils.toRuntimeException(e);
+    }
   }
 
   @Override
   public void serialize(Object t) throws IOException {
-    instance.serialize(t);
+    try {
+      serializeMethod.invoke(instance, t);
+    } catch (Exception e) {
+      throw ExceptionUtils.toRuntimeException(e);
+    }
   }
 
   @Override
   public void close() throws IOException {
-    instance.close();
+    try {
+      closeMethod.invoke(instance);
+    } catch (Exception e) {
+      throw ExceptionUtils.toRuntimeException(e);
+    }
   }
+
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtils.java?rev=1590272&r1=1590271&r2=1590272&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtils.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtils.java Sat Apr 26 18:18:32 2014
@@ -20,25 +20,70 @@
 package org.apache.hadoop.hbase.util;
 
 import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.UndeclaredThrowableException;
 
 /**
  * Some utility methods for handling exception.
  */
 public class ExceptionUtils {
   /**
-   * Throws an arbitrary exception to an IOException. Wrap the exception with
-   * an IOException if the exception cannot be thrown directly.
+   * If t is an RuntimeException throw it directly, or
+   * If t is an InvocationTargetException, recursively call toIOException
+   * with target exception, or
+   * If t is an UndeclaredThrowableException, recursively call toIOException
+   * with undeclared throwable, or
+   * Otherwise convert it into an IOException. Wrap the exception with an
+   * IOException if necessary.
+   *
+   * DO NOT use it if you want to wrap all exception including RuntimeException
+   * into an IOException.
    */
-  public static void throwIOExcetion(Throwable t)
-      throws IOException {
+  public static IOException toIOException(Throwable t) {
+    if (t instanceof InvocationTargetException) {
+      return toIOException(((InvocationTargetException) t)
+          .getTargetException());
+    }
+
+    if (t instanceof UndeclaredThrowableException) {
+      return toIOException(((UndeclaredThrowableException) t)
+          .getUndeclaredThrowable());
+    }
+
     if (t instanceof IOException) {
-      throw (IOException) t;
+      return (IOException) t;
     }
 
     if (t instanceof RuntimeException) {
       throw (RuntimeException) t;
     }
 
-    throw new IOException(t);
+    return new IOException(t);
+  }
+
+  /**
+   * If t is an RuntimeException return it directly, or
+   * If t is an InvocationTargetException, recursively call toRuntimeException
+   * with target exception, or
+   * If t is an UndeclaredThrowableException, recursively call
+   * toRuntimeException with undeclared throwable, or
+   * Otherwise wrap it into an toRuntimeException.
+   */
+  public static RuntimeException toRuntimeException(Throwable t) {
+    if (t instanceof InvocationTargetException) {
+      return toRuntimeException(((InvocationTargetException) t)
+          .getTargetException());
+    }
+
+    if (t instanceof UndeclaredThrowableException) {
+      return toRuntimeException(((UndeclaredThrowableException) t)
+          .getUndeclaredThrowable());
+    }
+
+    if (t instanceof RuntimeException) {
+      return (RuntimeException) t;
+    }
+
+    return new RuntimeException(t);
   }
 }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftResultSerializer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftResultSerializer.java?rev=1590272&r1=1590271&r2=1590272&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftResultSerializer.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftResultSerializer.java Sat Apr 26 18:18:32 2014
@@ -20,9 +20,6 @@
 package org.apache.hadoop.hbase.thrift;
 
 import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -38,15 +35,15 @@ import org.junit.Test;
  */
 public class TestThriftResultSerializer {
 
-  private KeyValue[] constuctKvList(int n) {
-    List<KeyValue> list = new ArrayList<KeyValue>();
+  private static KeyValue[] constuctKvList(int n) {
+    KeyValue[] kvs = new KeyValue[n];
     for (int i = 0; i < n; i++) {
       KeyValue kv = new KeyValue(Bytes.toBytes("myRow" + i),
           Bytes.toBytes("myCF"), Bytes.toBytes("myQualifier"), 12345L,
           Bytes.toBytes("myValue"));
-      list.add(kv);
+      kvs[i] = kv;
     }
-    return list.toArray(new KeyValue[list.size()]);
+    return kvs;
   }
 
   @Test
@@ -69,28 +66,19 @@ public class TestThriftResultSerializer 
     Assert.assertTrue("output is empty", out.size() > 0);
   }
 
-  private static String findThriftPath() {
-    String[] paths = System.getProperty("java.class.path").split(
-        File.pathSeparator);
-    for (String path : paths) {
-      if (path.contains("/libthrift-")) {
-        return path;
-      }
-    }
-    return "";
-  }
-
   @Test
   public void testWithClassPath() throws Exception {
     ByteArrayOutputStream out = new ByteArrayOutputStream();
 
     ThriftResultSerializer2 trs = new ThriftResultSerializer2();
 
-    String cp = findThriftPath();
+    String cp = System.getProperty("java.class.path");
     System.out.println("Cutting classpath: " + cp);
 
     Configuration conf = HBaseConfiguration.create();
     conf.set(ThriftResultSerializer2.CUTTING_CLASSPATH_KEY, cp);
+    conf.set(ThriftResultSerializer.PROTOCOL_CONF_KEY,
+        "org.apache.thrift.protocol.TCompactProtocol");
     trs.setConf(conf);
 
     trs.open(out);
@@ -103,4 +91,33 @@ public class TestThriftResultSerializer 
 
     Assert.assertTrue("output is empty", out.size() > 0);
   }
+
+  public static void main(String[] args) throws Exception {
+    System.out.println("java.class.path: "
+        + System.getProperty("java.class.path"));
+
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    ThriftResultSerializer2 trs = new ThriftResultSerializer2();
+
+    Configuration conf = HBaseConfiguration.create();
+    if (args.length >= 1) {
+      System.out.println("Setting "
+          + ThriftResultSerializer2.CUTTING_CLASSPATH_KEY + " to " + args[0]);
+      conf.set(ThriftResultSerializer2.CUTTING_CLASSPATH_KEY, args[0]);
+    }
+    if (args.length >= 2) {
+      System.out.println("Setting " + ThriftResultSerializer.PROTOCOL_CONF_KEY
+          + " to " + args[1]);
+      conf.set(ThriftResultSerializer.PROTOCOL_CONF_KEY, args[1]);
+    }
+    trs.setConf(conf);
+
+    trs.open(out);
+    try {
+      trs.serialize(new ImmutableBytesWritable(Bytes.toBytes("row")));
+      trs.serialize(new Result(constuctKvList(10)));
+    } finally {
+      trs.close();
+    }
+  }
 }