You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by gr...@apache.org on 2014/05/13 22:19:27 UTC

git commit: PHOENIX-962 Load FilterList with clean classloader

Repository: incubator-phoenix
Updated Branches:
  refs/heads/3.0 f5a42a622 -> 57f030f63


PHOENIX-962 Load FilterList with clean classloader

Ensure that the HBase FilterList class is loaded while a
"standard" context classloader (i.e. the classloader used to load
Phoenix classes) is set for the current thread. This is to avoid
issues with hbase config resources not being able to be found if
the FilterList class is loaded from within another thread in a
client that has a custom context classloader.


Project: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/commit/57f030f6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/tree/57f030f6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/diff/57f030f6

Branch: refs/heads/3.0
Commit: 57f030f63651d8728350d96c3ea77518e148b506
Parents: f5a42a6
Author: Gabriel Reid <ga...@ngdata.com>
Authored: Wed Apr 30 22:30:00 2014 +0200
Committer: Gabriel Reid <gr...@apache.org>
Committed: Tue May 13 22:14:27 2014 +0200

----------------------------------------------------------------------
 .../phoenix/end2end/ContextClassloaderIT.java   | 144 +++++++++++++++++++
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  19 ++-
 2 files changed, 162 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/57f030f6/phoenix-core/src/it/java/org/apache/phoenix/end2end/ContextClassloaderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ContextClassloaderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ContextClassloaderIT.java
new file mode 100644
index 0000000..a2ee18a
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ContextClassloaderIT.java
@@ -0,0 +1,144 @@
+/*
+ * 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.phoenix.end2end;
+
+import java.io.File;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.phoenix.query.QueryServices;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+
+public class ContextClassloaderIT  {
+
+    private static HBaseTestingUtility hbaseTestUtil;
+    private static ClassLoader badContextClassloader;
+
+    @BeforeClass
+    public static void setUpBeforeClass() throws Exception {
+        hbaseTestUtil = new HBaseTestingUtility();
+        hbaseTestUtil.getConfiguration().setInt(QueryServices.MASTER_INFO_PORT_ATTRIB, -1);
+        hbaseTestUtil.getConfiguration().setInt(QueryServices.REGIONSERVER_INFO_PORT_ATTRIB, -1);
+        hbaseTestUtil.startMiniCluster();
+        Connection conn = DriverManager.getConnection(getUrl());
+        Statement stmt = conn.createStatement();
+        stmt.execute("CREATE TABLE test (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR)");
+        stmt.execute("UPSERT INTO test VALUES (1, 'name1')");
+        stmt.execute("UPSERT INTO test VALUES (2, 'name2')");
+        stmt.close();
+        conn.commit();
+        conn.close();
+        badContextClassloader = new URLClassLoader(new URL[] {
+                File.createTempFile("invalid", ".jar").toURI().toURL() }, null);
+    }
+
+    private static String getUrl() {
+        return "jdbc:phoenix:localhost:" + hbaseTestUtil.getZkCluster
+                ().getClientPort();
+    }
+
+    @AfterClass
+    public static void tearDownAfterClass() throws Exception {
+        hbaseTestUtil.shutdownMiniCluster();
+    }
+
+    @Test
+    public void testQueryWithDifferentContextClassloader() throws SQLException, InterruptedException {
+        Runnable target = new Runnable() {
+
+
+            @Override
+            public void run() {
+                try {
+                    Connection conn = DriverManager.getConnection(getUrl());
+                    Statement stmt = conn.createStatement();
+                    ResultSet rs = stmt.executeQuery("select * from test where name = 'name2'");
+                    while (rs.next()) {
+                        // Just make sure we run over all records
+                    }
+                    rs.close();
+                    stmt.close();
+                    conn.close();
+                } catch (SQLException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        };
+        BadContextClassloaderThread t = new BadContextClassloaderThread(target);
+        t.start();
+        t.join();
+        assertFalse(t.failed);
+    }
+
+    @Test
+    public void testGetDatabaseMetadataWithDifferentContextClassloader() throws InterruptedException {
+        Runnable target = new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    Connection conn = DriverManager.getConnection(getUrl());
+                    ResultSet tablesRs = conn.getMetaData().getTables(null, null, null, null);
+                    while (tablesRs.next()) {
+                        // Just make sure we run over all records
+                    }
+                    tablesRs.close();
+                    conn.close();
+                } catch (SQLException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        };
+        BadContextClassloaderThread t = new BadContextClassloaderThread(target);
+        t.start();
+        t.join();
+        assertFalse(t.failed);
+    }
+
+    static class BadContextClassloaderThread extends Thread {
+
+        private final Runnable target;
+        boolean failed = false;
+
+        public BadContextClassloaderThread(Runnable target) {
+            super("BadContextClassloaderThread");
+            this.target = target;
+            setContextClassLoader(badContextClassloader);
+        }
+
+        @Override
+        public void run() {
+            try {
+                target.run();
+            } catch (Throwable t) {
+                failed = true;
+                throw new RuntimeException(t);
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/57f030f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 2368211..4209c0f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -47,6 +47,7 @@ import java.util.concurrent.Executor;
 import javax.annotation.Nullable;
 
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.MutationState;
@@ -110,7 +111,7 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
     private final String datePattern;
     
     private boolean isClosed = false;
-    
+
     private static Properties newPropsWithSCN(long scn, Properties props) {
         props = new Properties(props);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(scn));
@@ -702,4 +703,20 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
     public KeyValueBuilder getKeyValueBuilder() {
         return this.services.getKeyValueBuilder();
     }
+
+    // We create a FilterList while manipulating the context classloader here to get around the
+    // fact that the FilterList class calls HBaseConfiguration.create, which is in turn reliant
+    // on the current thread's context classloader. Some JDBC clients do questionable things with
+    // the context classloader, so by ensuring that we create a FilterList here with a context
+    // classloader that is under our control, we avoid these kinds of errors in client tools.
+    // Note: this is only needed when using HBase 0.94.x.
+    static {
+        ClassLoader saveContextClassloader = Thread.currentThread().getContextClassLoader();
+        try {
+            Thread.currentThread().setContextClassLoader(PhoenixConnection.class.getClassLoader());
+            new FilterList();
+        } finally {
+            Thread.currentThread().setContextClassLoader(saveContextClassloader);
+        }
+    }
 }