You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by sh...@apache.org on 2015/11/19 08:56:02 UTC

incubator-kylin git commit: KYLIN-1155 unit test with minicluster doesn't work

Repository: incubator-kylin
Updated Branches:
  refs/heads/1.x-staging 446bc5c9a -> ae6f23c50


KYLIN-1155 unit test with minicluster doesn't work

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

Branch: refs/heads/1.x-staging
Commit: ae6f23c50631a94bb3fb69b838cdc128ddcf114f
Parents: 446bc5c
Author: shaofengshi <sh...@apache.org>
Authored: Thu Nov 19 15:55:37 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 19 15:55:37 2015 +0800

----------------------------------------------------------------------
 .../common/persistence/HBaseConnection.java     |  7 +-
 .../persistence/HBaseResourceStoreTest.java     |  6 --
 .../common/util/AbstractKylinTestCase.java      |  5 ++
 .../common/util/HBaseMiniclusterHelper.java     | 23 +++---
 .../test_case_data/localmeta/kylin.properties   |  2 +-
 .../minicluster/b-kylin/kylin.properties        | 74 --------------------
 .../minicluster/b-kylin/meta/kylin.properties   | 71 -------------------
 .../test_case_data/minicluster/kylin.properties |  4 +-
 8 files changed, 20 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ae6f23c5/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java b/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
index c4d0314..dcefc24 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
@@ -42,7 +42,6 @@ public class HBaseConnection {
 
     private static final Logger logger = LoggerFactory.getLogger(HBaseConnection.class);
 
-    private static final Map<String, Configuration> ConfigCache = new ConcurrentHashMap<String, Configuration>();
     private static final Map<String, HConnection> ConnPool = new ConcurrentHashMap<String, HConnection>();
 
     static {
@@ -62,11 +61,7 @@ public class HBaseConnection {
 
     public static HConnection get(String url) {
         // find configuration
-        Configuration conf = ConfigCache.get(url);
-        if (conf == null) {
-            conf = HadoopUtil.newHBaseConfiguration(url);
-            ConfigCache.put(url, conf);
-        }
+        Configuration conf = HadoopUtil.getCurrentHBaseConfiguration();
 
         HConnection connection = ConnPool.get(url);
         try {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ae6f23c5/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java b/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java
index b54f413..75625fb 100644
--- a/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java
+++ b/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java
@@ -45,17 +45,11 @@ public class HBaseResourceStoreTest extends HBaseMetadataTestCase {
     @Before
     public void setup() throws Exception {
         this.createTestMetadata();
-
-        HadoopUtil.setCurrentConfiguration(null);
-        HadoopUtil.setCurrentHBaseConfiguration(null);
     }
 
     @After
     public void after() throws Exception {
         this.cleanupTestMetadata();
-
-        HadoopUtil.setCurrentConfiguration(null);
-        HadoopUtil.setCurrentHBaseConfiguration(null);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ae6f23c5/common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java b/common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
index e1f05ac..58d3ac1 100644
--- a/common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
+++ b/common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
@@ -50,6 +50,8 @@ public abstract class AbstractKylinTestCase {
 
         KylinConfig.destoryInstance();
 
+        HadoopUtil.setCurrentConfiguration(null);
+        HadoopUtil.setCurrentHBaseConfiguration(null);
         if (System.getProperty(KylinConfig.KYLIN_CONF) == null && System.getenv(KylinConfig.KYLIN_CONF) == null)
             System.setProperty(KylinConfig.KYLIN_CONF, kylinConfigFolder);
 
@@ -60,6 +62,9 @@ public abstract class AbstractKylinTestCase {
         System.clearProperty(KylinConfig.KYLIN_CONF);
         KylinConfig.destoryInstance();
 
+        HadoopUtil.setCurrentConfiguration(null);
+        HadoopUtil.setCurrentHBaseConfiguration(null);
+
     }
 
     private static void cleanupCache() {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ae6f23c5/common/src/test/java/org/apache/kylin/common/util/HBaseMiniclusterHelper.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/util/HBaseMiniclusterHelper.java b/common/src/test/java/org/apache/kylin/common/util/HBaseMiniclusterHelper.java
index 26f9c03..6f31776 100644
--- a/common/src/test/java/org/apache/kylin/common/util/HBaseMiniclusterHelper.java
+++ b/common/src/test/java/org/apache/kylin/common/util/HBaseMiniclusterHelper.java
@@ -45,7 +45,7 @@ public class HBaseMiniclusterHelper {
 
     public static HBaseTestingUtility UTIL = new HBaseTestingUtility();
     private static volatile boolean clusterStarted = false;
-    private static String hbaseconnectionUrl = "";
+    private static String zkHost, zkPort, zkParent;
 
     private static final Log logger = LogFactory.getLog(HBaseMiniclusterHelper.class);
 
@@ -80,8 +80,10 @@ public class HBaseMiniclusterHelper {
 
     private static void updateKylinConfigWithMinicluster() {
 
-        KylinConfig.getInstanceFromEnv().setMetadataUrl(TEST_METADATA_TABLE + "@" + hbaseconnectionUrl);
-        KylinConfig.getInstanceFromEnv().setStorageUrl(hbaseconnectionUrl);
+        Configuration hbaseConfiguration = HadoopUtil.getCurrentHBaseConfiguration();
+        hbaseConfiguration.set(HConstants.ZOOKEEPER_QUORUM, zkHost);
+        hbaseConfiguration.set(HConstants.ZOOKEEPER_CLIENT_PORT, zkPort);
+        hbaseConfiguration.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkParent);
     }
 
     private static void startupMiniClusterAndImportData() throws Exception {
@@ -96,11 +98,11 @@ public class HBaseMiniclusterHelper {
         UTIL.getConfiguration().setInt("hbase.master.info.port", -1);//avoid port clobbering
 
         MiniHBaseCluster hbaseCluster = UTIL.startMiniCluster();
-
+        UTIL.startMiniMapReduceCluster();
         Configuration config = hbaseCluster.getConf();
-        String host = config.get(HConstants.ZOOKEEPER_QUORUM);
-        String port = config.get(HConstants.ZOOKEEPER_CLIENT_PORT);
-        String parent = config.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+        zkHost = config.get(HConstants.ZOOKEEPER_QUORUM);
+        zkPort = config.get(HConstants.ZOOKEEPER_CLIENT_PORT);
+        zkParent = config.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
 
         // see in: https://hbase.apache.org/book.html#trouble.rs.runtime.zkexpired
         config.set("zookeeper.session.timeout", "1200000");
@@ -110,14 +112,11 @@ public class HBaseMiniclusterHelper {
         config.set(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "1");
         config.set(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, "60000");
 
-        hbaseconnectionUrl = "hbase:" + host + ":" + port + ":" + parent;
         updateKylinConfigWithMinicluster();
-
-        UTIL.startMiniMapReduceCluster();
-
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         // create the metadata htables;
         @SuppressWarnings("unused")
-        HBaseResourceStore store = new HBaseResourceStore(KylinConfig.getInstanceFromEnv());
+        HBaseResourceStore store = new HBaseResourceStore(kylinConfig);
 
         // import the table content
         HbaseImporter.importHBaseData(hbaseTarLocation, UTIL.getConfiguration());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ae6f23c5/examples/test_case_data/localmeta/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kylin.properties b/examples/test_case_data/localmeta/kylin.properties
index ff9ece4..ddff900 100644
--- a/examples/test_case_data/localmeta/kylin.properties
+++ b/examples/test_case_data/localmeta/kylin.properties
@@ -24,7 +24,7 @@ kylin.rest.servers=localhost:7070
 kylin.metadata.url=
 
 # The storage for final cube file in hbase
-kylin.storage.url=hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
+kylin.storage.url=hbase
 
 # Temp folder in hdfs, make sure user has the right access to the hdfs directory
 kylin.hdfs.working.dir=/kylin

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ae6f23c5/examples/test_case_data/minicluster/b-kylin/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/minicluster/b-kylin/kylin.properties b/examples/test_case_data/minicluster/b-kylin/kylin.properties
deleted file mode 100644
index 38587f2..0000000
--- a/examples/test_case_data/minicluster/b-kylin/kylin.properties
+++ /dev/null
@@ -1,74 +0,0 @@
-#
-# 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.
-#
-
-## Config for Kylin Engine ##
-
-# List of web servers in use, this enables one web server instance to sync up with other servers.
-kylin.rest.servers=ADMIN:KYLIN@localhost
-
-# The metadata store in hbase
-kylin.metadata.url=kylin_metadata@hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
-
-# The storage for final cube file in hbase
-kylin.storage.url=hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
-
-# Temp folder in hdfs, make sure user has the right access to the hdfs directory
-kylin.hdfs.working.dir=/kylin
-
-kylin.job.mapreduce.default.reduce.input.mb=500
-
-# If true, job engine will not assume that hadoop CLI reside on the same server as it self
-# you will have to specify kylin.job.remote.cli.hostname, kylin.job.remote.cli.username and kylin.job.remote.cli.password
-kylin.job.run.as.remote.cmd=true
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.hostname=sandbox.hortonworks.com
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.username=root
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.password=hadoop
-
-# Used by test cases to prepare synthetic data for sample cube
-kylin.job.remote.cli.working.dir=/tmp/kylin
-
-# Max count of concurrent jobs running
-kylin.job.concurrent.max.limit=10
-
-# Time interval to check hadoop job status
-kylin.job.yarn.app.rest.check.interval.seconds=1
-
-
-## Config for Restful APP ##
-# database connection settings:
-ldap.server=
-ldap.username=
-ldap.password=
-ldap.user.searchBase=
-ldap.user.searchPattern=
-ldap.user.groupSearchBase=
-ldap.service.searchBase=OU=
-ldap.service.searchPattern=
-ldap.service.groupSearchBase=
-acl.adminRole=
-acl.defaultRole=
-ganglia.group=
-ganglia.port=8664
-
-###########################config info for sandbox#######################
-kylin.sandbox=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ae6f23c5/examples/test_case_data/minicluster/b-kylin/meta/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/minicluster/b-kylin/meta/kylin.properties b/examples/test_case_data/minicluster/b-kylin/meta/kylin.properties
deleted file mode 100644
index 5d890de..0000000
--- a/examples/test_case_data/minicluster/b-kylin/meta/kylin.properties
+++ /dev/null
@@ -1,71 +0,0 @@
-#
-# 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.
-#
-
-## Config for Kylin Engine ##
-
-# List of web servers in use, this enables one web server instance to sync up with other servers.
-kylin.rest.servers=ADMIN:KYLIN@localhost
-
-# The metadata store in hbase
-kylin.metadata.url=kylin_metadata@hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
-
-# The storage for final cube file in hbase
-kylin.storage.url=hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
-
-# Temp folder in hdfs, make sure user has the right access to the hdfs directory
-kylin.hdfs.working.dir=/kylin
-
-kylin.job.mapreduce.default.reduce.input.mb=500
-
-# If true, job engine will not assume that hadoop CLI reside on the same server as it self
-# you will have to specify kylin.job.remote.cli.hostname, kylin.job.remote.cli.username and kylin.job.remote.cli.password
-kylin.job.run.as.remote.cmd=true
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.hostname=sandbox.hortonworks.com
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.username=root
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.password=hadoop
-
-# Used by test cases to prepare synthetic data for sample cube
-kylin.job.remote.cli.working.dir=/tmp/kylin
-
-# Max count of concurrent jobs running
-kylin.job.concurrent.max.limit=10
-
-# Time interval to check hadoop job status
-kylin.job.yarn.app.rest.check.interval.seconds=1
-
-
-## Config for Restful APP ##
-# database connection settings:
-ldap.server=
-ldap.username=
-ldap.password=
-ldap.user.searchBase=
-ldap.user.searchPattern=
-ldap.user.groupSearchBase=
-ldap.service.searchBase=OU=
-ldap.service.searchPattern=
-ldap.service.groupSearchBase=
-acl.adminRole=
-acl.defaultRole=
-ganglia.group=
-ganglia.port=8664
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ae6f23c5/examples/test_case_data/minicluster/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/minicluster/kylin.properties b/examples/test_case_data/minicluster/kylin.properties
index 62d1dab..06f9870 100644
--- a/examples/test_case_data/minicluster/kylin.properties
+++ b/examples/test_case_data/minicluster/kylin.properties
@@ -21,10 +21,10 @@
 kylin.rest.servers=ADMIN:KYLIN@localhost
 
 # The metadata store in hbase
-kylin.metadata.url=kylin_metadata@hbase:minicluster:/hbase-unsecure
+kylin.metadata.url=kylin_metadata@hbase
 
 # The storage for final cube file in hbase
-kylin.storage.url=hbase:minicluster:/hbase-unsecure
+kylin.storage.url=hbase
 
 # Temp folder in hdfs, make sure user has the right access to the hdfs directory
 kylin.hdfs.working.dir=/tmp