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/02 22:49:41 UTC

svn commit: r1584173 - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/thrift/ test/java/org/apache/hadoop/hbase/thrift/

Author: liyin
Date: Wed Apr  2 20:49:41 2014
New Revision: 1584173

URL: http://svn.apache.org/r1584173
Log:
[master] Make ThriftResultSerializer use the correct library

Author: daviddeng

Summary:
Add `CuttingClassLoader` which allow cutting paths for loading clases
Add `ThriftResultSerializer` which loads

Test Plan: TestThriftResultSerializer

Reviewers: rshroff, manukranthk, liyintang, aaiyer, adela

Reviewed By: rshroff

CC: hbase-eng@, arice, chaoyc

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

Task ID: 3547474, 3948196

Added:
    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/ThriftResultSerializer2.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftResultSerializer.java

Added: 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=1584173&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/CuttingClassLoader.java (added)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/CuttingClassLoader.java Wed Apr  2 20:49:41 2014
@@ -0,0 +1,52 @@
+/**
+ * Copyright 2014 The Apache Software Foundation
+ *
+ * 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.thrift;
+
+import java.net.URL;
+import java.net.URLClassLoader;
+
+/**
+ * The classloader allows the searching firstly in some path, if not found
+ * search in parent.
+ */
+public class CuttingClassLoader extends URLClassLoader {
+  public CuttingClassLoader(URL[] urls, ClassLoader parent) {
+    super(urls, parent);
+  }
+
+  @Override
+  protected synchronized Class<?> loadClass(String name, boolean resolve)
+      throws ClassNotFoundException {
+    // Find in loaded classes
+    Class<?> clazz = findLoadedClass(name);
+    if (clazz != null) {
+      return clazz;
+    }
+
+    try {
+      // Find in cutting paths
+      return findClass(name);
+    } catch (ClassNotFoundException e) {
+    }
+
+    // Find in parent
+    return super.loadClass(name, resolve);
+  }
+}

Added: 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=1584173&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer2.java (added)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftResultSerializer2.java Wed Apr  2 20:49:41 2014
@@ -0,0 +1,92 @@
+/**
+ * Copyright 2014 The Apache Software Foundation
+ *
+ * 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.thrift;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URL;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.serializer.Serializer;
+
+/**
+ * A wrapper class which loads <code>ThriftResultSerializer</code> class with a
+ * <code>CuttingClassLoader</code>.
+ *
+ * Using PROTOCOL_CLASSPATH_KEY in configuration to specify paths to cut in
+ * queue.
+ */
+public class ThriftResultSerializer2 implements Serializer<Object>,
+    Configurable {
+  ThriftResultSerializer instance;
+  /**
+   * Key to the configuration of classpath to cut
+   */
+  public static final String CUTTING_CLASSPATH_KEY =
+      "hbase.thrift.result.serializer.cutting.classpath";
+
+  @Override
+  public void setConf(Configuration conf) {
+    String classPath = conf.get(CUTTING_CLASSPATH_KEY, "");
+    try {
+      // make the URL array of cutting paths
+      String[] parts = classPath.split(File.pathSeparator);
+      URL[] urls = new URL[parts.length];
+      for (int i = 0; i < parts.length; i++) {
+        urls[i] = new File(parts[i]).toURI().toURL();
+      }
+
+      // Create the ClassLoader
+      ClassLoader classLoader = new CuttingClassLoader(urls,
+ this.getClass().getClassLoader());
+      // and load the class.
+      Class<?> cls = Class.forName(ThriftResultSerializer.class.getName(),
+          true, classLoader);
+      // Then create the new instance.
+      instance = (ThriftResultSerializer) cls.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    instance.setConf(conf);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return instance.getConf();
+  }
+
+  @Override
+  public void open(OutputStream out) throws IOException {
+    instance.open(out);
+  }
+
+  @Override
+  public void serialize(Object t) throws IOException {
+    instance.serialize(t);
+  }
+
+  @Override
+  public void close() throws IOException {
+    instance.close();
+  }
+}

Added: 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=1584173&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftResultSerializer.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftResultSerializer.java Wed Apr  2 20:49:41 2014
@@ -0,0 +1,106 @@
+/**
+ * Copyright 2014 The Apache Software Foundation
+ *
+ * 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.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;
+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.Bytes;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Testcase for ThriftResultSerializer and ThriftResultSerializer2
+ */
+public class TestThriftResultSerializer {
+
+  private KeyValue[] constuctKvList(int n) {
+    List<KeyValue> list = new ArrayList<KeyValue>();
+    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);
+    }
+    return list.toArray(new KeyValue[list.size()]);
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+    ThriftResultSerializer trs = new ThriftResultSerializer();
+
+    Configuration conf = HBaseConfiguration.create();
+    trs.setConf(conf);
+
+    trs.open(out);
+    try {
+      trs.serialize(new ImmutableBytesWritable(Bytes.toBytes("row")));
+      trs.serialize(new Result(constuctKvList(10)));
+    } finally {
+      trs.close();
+    }
+
+    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();
+    System.out.println("Cutting classpath: " + cp);
+
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(ThriftResultSerializer2.CUTTING_CLASSPATH_KEY, cp);
+    trs.setConf(conf);
+
+    trs.open(out);
+    try {
+      trs.serialize(new ImmutableBytesWritable(Bytes.toBytes("row")));
+      trs.serialize(new Result(constuctKvList(10)));
+    } finally {
+      trs.close();
+    }
+
+    Assert.assertTrue("output is empty", out.size() > 0);
+  }
+}