You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ma...@apache.org on 2016/08/17 07:20:24 UTC

[1/2] kylin git commit: KYLIN-1963 Delegate the loading of certain package (like slf4j) to tomcat's parent classloader

Repository: kylin
Updated Branches:
  refs/heads/master 224b07cd4 -> 0badf49fd


KYLIN-1963 Delegate the loading of certain package (like slf4j) to tomcat's parent classloader


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/360c8ee0
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/360c8ee0
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/360c8ee0

Branch: refs/heads/master
Commit: 360c8ee03892ec40428bdee1a76f36dd618715b1
Parents: 224b07c
Author: Hongbin Ma <ma...@apache.org>
Authored: Wed Aug 17 15:16:21 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Wed Aug 17 15:18:14 2016 +0800

----------------------------------------------------------------------
 assembly/pom.xml                                | 16 +++++
 build/deploy/context.xml                        | 38 +++++++++++
 build/script/download-tomcat.sh                 | 16 +++++
 build/script/package.sh                         |  2 +-
 pom.xml                                         |  7 +-
 .../kylin/rest/controller/QueryController.java  |  5 +-
 server/pom.xml                                  |  8 ++-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     |  2 +-
 .../hbase/cube/v2/ExpectedSizeIterator.java     |  7 +-
 tomcat-ext/README.md                            |  5 ++
 tomcat-ext/pom.xml                              | 23 +++++++
 .../kylin/ext/CustomizedWebappClassloader.java  | 69 ++++++++++++++++++++
 12 files changed, 186 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 76ca0d8..36076d2 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -52,6 +52,22 @@
             <artifactId>kylin-engine-streaming</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        
         <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.kylin</groupId>

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/build/deploy/context.xml
----------------------------------------------------------------------
diff --git a/build/deploy/context.xml b/build/deploy/context.xml
new file mode 100644
index 0000000..5d1bedf
--- /dev/null
+++ b/build/deploy/context.xml
@@ -0,0 +1,38 @@
+<?xml version='1.0' encoding='utf-8'?>
+<!--
+  ~ 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.
+  -->
+<!-- The contents of this file will be loaded for each web application -->
+<Context>
+
+    <!-- Default set of monitored resources -->
+    <WatchedResource>WEB-INF/web.xml</WatchedResource>
+
+    <!-- Uncomment this to disable session persistence across Tomcat restarts -->
+    <!--
+    <Manager pathname="" />
+    -->
+
+    <!-- Uncomment this to enable Comet connection tacking (provides events
+         on session expiration as well as webapp lifecycle) -->
+    <!--
+    <Valve className="org.apache.catalina.valves.CometConnectionManagerValve" />
+    -->
+    
+    <Loader loaderClass="org.apache.kylin.ext.CustomizedWebappClassloader"/>
+
+</Context>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/build/script/download-tomcat.sh
----------------------------------------------------------------------
diff --git a/build/script/download-tomcat.sh b/build/script/download-tomcat.sh
index 87467ce..9dd26f0 100755
--- a/build/script/download-tomcat.sh
+++ b/build/script/download-tomcat.sh
@@ -46,5 +46,21 @@ mv build/apache-tomcat-7.0.69 build/tomcat
 rm -rf build/tomcat/webapps/*
 
 mv build/tomcat/conf/server.xml build/tomcat/conf/server.xml.bak
+mv build/tomcat/conf/context.xml build/tomcat/conf/context.xml.bak
 cp build/deploy/server.xml build/tomcat/conf/server.xml
 echo "server.xml overwritten..."
+cp build/deploy/context.xml build/tomcat/conf/context.xml
+echo "context.xml overwritten..."
+
+
+if [ -z "$version" ]
+then
+    echo 'version not set'
+    version=`mvn org.apache.maven.plugins:maven-help-plugin:2.1.1:evaluate -Dexpression=project.version | grep -v '\['`
+fi
+echo "version ${version}"
+export version
+
+cp tomcat-ext/target/kylin-tomcat-ext-${version}.jar build/tomcat/lib/kylin-tomcat-ext-${version}.jar
+chmod 644 build/tomcat/lib/kylin-tomcat-ext-${version}.jar
+

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/build/script/package.sh
----------------------------------------------------------------------
diff --git a/build/script/package.sh b/build/script/package.sh
index a077be5..1f9fbbd 100755
--- a/build/script/package.sh
+++ b/build/script/package.sh
@@ -74,8 +74,8 @@ cat << EOF > build/commit_SHA1
 EOF
 git rev-parse HEAD >> build/commit_SHA1
 
-sh build/script/download-tomcat.sh || { exit 1; }
 sh build/script/build.sh || { exit 1; }
+sh build/script/download-tomcat.sh || { exit 1; }
 sh build/script/prepare.sh || { exit 1; }
 sh build/script/compress.sh || { exit 1; }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index fe0d8cf..b3c3c6b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -16,7 +16,6 @@
  See the License for the specific language governing permissions and
  limitations under the License.
 -->
-
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
     <modelVersion>4.0.0</modelVersion>
@@ -454,6 +453,7 @@
                 <groupId>org.apache.calcite.avatica</groupId>
                 <artifactId>avatica</artifactId>
                 <version>${calcite.version}</version>
+
             </dependency>
             <!-- Workaround for hive 0.14 avatica dependency -->
             <dependency>
@@ -945,7 +945,7 @@
                                         </goals>
                                     </pluginExecutionFilter>
                                     <action>
-                                        <ignore></ignore>
+                                        <ignore/>
                                     </action>
                                 </pluginExecution>
                             </pluginExecutions>
@@ -993,6 +993,7 @@
         <module>assembly</module>
         <module>tool</module>
         <module>kylin-it</module>
+        <module>tomcat-ext</module>
     </modules>
 
     <profiles>
@@ -1219,4 +1220,4 @@
             </build>
         </profile>
     </profiles>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
index cf5358e..f61a90e 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
@@ -148,7 +148,7 @@ public class QueryController extends BasicController {
                 csvWriter.write(row);
             }
         } catch (IOException e) {
-            logger.error("", e);
+            throw new InternalErrorException(e);
         } finally {
             IOUtils.closeQuietly(csvWriter);
         }
@@ -160,7 +160,6 @@ public class QueryController extends BasicController {
         try {
             return queryService.getMetadata(metaRequest.getProject());
         } catch (SQLException e) {
-            logger.error(e.getLocalizedMessage(), e);
             throw new InternalErrorException(e.getLocalizedMessage(), e);
         }
     }
@@ -206,7 +205,7 @@ public class QueryController extends BasicController {
                 checkQueryAuth(sqlResponse);
 
             } catch (Throwable e) { // calcite may throw AssertError
-                logger.error("Exception when execute sql", e);
+                //logger.error("Exception when execute sql", e);
                 String errMsg = QueryUtil.makeErrorMsgUserFriendly(e);
 
                 sqlResponse = new SQLResponse(null, null, 0, true, errMsg);

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index 9068c6e..d72a40d 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -86,19 +86,23 @@
         <dependency>
             <groupId>log4j</groupId>
             <artifactId>log4j</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.slf4j</groupId>
-            <artifactId>jcl-over-slf4j</artifactId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-log4j12</artifactId>
+            <artifactId>jcl-over-slf4j</artifactId>
         </dependency>
+        
 
         <!-- Test & Env -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
index 4c599d9..200c040 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
@@ -63,7 +63,7 @@ import com.google.protobuf.HBaseZeroCopyByteString;
 
 public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
 
-    public static final Logger logger = LoggerFactory.getLogger(CubeHBaseEndpointRPC.class);
+    private static final Logger logger = LoggerFactory.getLogger(CubeHBaseEndpointRPC.class);
 
     private static ExecutorService executorService = new LoggableCachedThreadPool();
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/ExpectedSizeIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/ExpectedSizeIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/ExpectedSizeIterator.java
index 4e0d15e..7d48c1a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/ExpectedSizeIterator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/ExpectedSizeIterator.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
 import org.apache.kylin.storage.hbase.HBaseConnection;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.Iterator;
 import java.util.concurrent.ArrayBlockingQueue;
@@ -31,6 +33,7 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.TimeUnit;
 
 class ExpectedSizeIterator implements Iterator<byte[]> {
+    private static final Logger logger = LoggerFactory.getLogger(ExpectedSizeIterator.class);
 
     BlockingQueue<byte[]> queue;
 
@@ -48,7 +51,7 @@ class ExpectedSizeIterator implements Iterator<byte[]> {
         Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
         this.rpcTimeout = hconf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
         this.timeout = this.rpcTimeout * hconf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
-        CubeHBaseEndpointRPC.logger.info("rpc timeout is {} and after multiply retry times become {}", this.rpcTimeout, this.timeout);
+        logger.info("rpc timeout is {} and after multiply retry times become {}", this.rpcTimeout, this.timeout);
         this.timeout = Math.max(this.timeout, 5 * 60000);
         this.timeout *= KylinConfig.getInstanceFromEnv().getCubeVisitTimeoutTimes();
 
@@ -58,7 +61,7 @@ class ExpectedSizeIterator implements Iterator<byte[]> {
 
         this.timeout *= 1.1; // allow for some delay
 
-        CubeHBaseEndpointRPC.logger.info("Final Timeout for ExpectedSizeIterator is: " + this.timeout);
+        logger.info("Final Timeout for ExpectedSizeIterator is: " + this.timeout);
 
         this.timeoutTS = System.currentTimeMillis() + this.timeout;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/tomcat-ext/README.md
----------------------------------------------------------------------
diff --git a/tomcat-ext/README.md b/tomcat-ext/README.md
new file mode 100644
index 0000000..f293bd8
--- /dev/null
+++ b/tomcat-ext/README.md
@@ -0,0 +1,5 @@
+a customized WebappClassloader to delegate the loading of certain packages to parent classloaders 
+
+
+
+

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/tomcat-ext/pom.xml
----------------------------------------------------------------------
diff --git a/tomcat-ext/pom.xml b/tomcat-ext/pom.xml
new file mode 100644
index 0000000..1a171ee
--- /dev/null
+++ b/tomcat-ext/pom.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0"?>
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <modelVersion>4.0.0</modelVersion>
+    <packaging>jar</packaging>
+    <parent>
+        <groupId>org.apache.kylin</groupId>
+        <artifactId>kylin</artifactId>
+        <version>1.5.4-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>kylin-tomcat-ext</artifactId>
+    <name>Kylin:Tomcat-Ext</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-catalina</artifactId>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/kylin/blob/360c8ee0/tomcat-ext/src/main/java/org/apache/kylin/ext/CustomizedWebappClassloader.java
----------------------------------------------------------------------
diff --git a/tomcat-ext/src/main/java/org/apache/kylin/ext/CustomizedWebappClassloader.java b/tomcat-ext/src/main/java/org/apache/kylin/ext/CustomizedWebappClassloader.java
new file mode 100644
index 0000000..d97d583
--- /dev/null
+++ b/tomcat-ext/src/main/java/org/apache/kylin/ext/CustomizedWebappClassloader.java
@@ -0,0 +1,69 @@
+/*
+ * 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.kylin.ext;
+
+import org.apache.catalina.loader.WebappClassLoader;
+
+/**
+ * simple extension to standard WebappClassLoader
+ * the only difference is that CustomizedWebappClassloader is able to delegate more packages
+ * to parent classloaders
+ */
+public class CustomizedWebappClassloader extends WebappClassLoader {
+    /**
+     * Set of package names which are not allowed to be loaded from a webapp
+     * class loader without delegating first.
+     */
+    private static final String[] packageTriggers = { "org.slf4j" };
+
+    public CustomizedWebappClassloader() {
+    }
+
+    public CustomizedWebappClassloader(ClassLoader parent) {
+        super(parent);
+    }
+
+    /**
+     * Filter classes.
+     *
+     * @param name class name
+     * @return true if the class should be filtered
+     */
+    protected boolean filter(String name) {
+
+        if (name == null)
+            return false;
+
+        // Looking up the package
+        String packageName = null;
+        int pos = name.lastIndexOf('.');
+        if (pos != -1)
+            packageName = name.substring(0, pos);
+        else
+            return false;
+
+        for (int i = 0; i < packageTriggers.length; i++) {
+            if (packageName.startsWith(packageTriggers[i]))
+                return true;
+        }
+
+        return false;
+
+    }
+}


[2/2] kylin git commit: KYLIN-1936 fix CI

Posted by ma...@apache.org.
KYLIN-1936 fix CI


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/0badf49f
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/0badf49f
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/0badf49f

Branch: refs/heads/master
Commit: 0badf49fda295f4901d07dd0a670d3c7d6678de4
Parents: 360c8ee
Author: Hongbin Ma <ma...@apache.org>
Authored: Wed Aug 17 15:16:35 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Wed Aug 17 15:18:20 2016 +0800

----------------------------------------------------------------------
 .../src/test/java/org/apache/kylin/query/ITKylinQueryTest.java  | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0badf49f/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
index 741dd18..5db4342 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
@@ -35,6 +35,7 @@ import org.apache.kylin.query.enumerator.OLAPQuery;
 import org.apache.kylin.query.relnode.OLAPContext;
 import org.apache.kylin.query.routing.Candidate;
 import org.apache.kylin.query.schema.OLAPSchemaFactory;
+import org.apache.kylin.storage.hbase.HBaseStorage;
 import org.apache.kylin.storage.hbase.cube.v1.coprocessor.observer.ObserverEnabler;
 import org.dbunit.database.DatabaseConnection;
 import org.dbunit.database.IDatabaseConnection;
@@ -260,7 +261,9 @@ public class ITKylinQueryTest extends KylinTestBase {
 
     @Test
     public void testLimitCorrectness() throws Exception {
-        execLimitAndValidate(getQueryFolderPrefix() + "src/test/resources/query/sql");
+        if (HBaseStorage.overwriteStorageQuery == null) {//v1 query engine will not work
+            execLimitAndValidate(getQueryFolderPrefix() + "src/test/resources/query/sql");
+        }
     }
 
     @Test