You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2015/11/18 03:38:55 UTC

[01/30] incubator-kylin git commit: KYLIN-1078 Cannot have comments in the end of New Query textbox

Repository: incubator-kylin
Updated Branches:
  refs/heads/1.x-HBase1.1.3 5063c386f -> ba076d5c1


KYLIN-1078 Cannot have comments in the end of New Query textbox

Signed-off-by: Li, Yang <ya...@ebay.com>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 30eabafc0eea3b99d4025ed08931e65b74ef11e1
Parents: 41ad16e
Author: lidongsjtu <do...@ebay.com>
Authored: Wed Nov 4 16:11:23 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Nov 4 16:24:05 2015 +0800

----------------------------------------------------------------------
 server/src/main/java/org/apache/kylin/rest/util/QueryUtil.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/30eabafc/server/src/main/java/org/apache/kylin/rest/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/util/QueryUtil.java b/server/src/main/java/org/apache/kylin/rest/util/QueryUtil.java
index 16f6633..419bd69 100644
--- a/server/src/main/java/org/apache/kylin/rest/util/QueryUtil.java
+++ b/server/src/main/java/org/apache/kylin/rest/util/QueryUtil.java
@@ -107,12 +107,12 @@ public class QueryUtil {
 
         int limit = sqlRequest.getLimit();
         if (limit > 0 && !sql.toLowerCase().contains("limit")) {
-            sql += (" LIMIT " + limit);
+            sql += ("\nLIMIT " + limit);
         }
 
         int offset = sqlRequest.getOffset();
         if (offset > 0 && !sql.toLowerCase().contains("offset")) {
-            sql += (" OFFSET " + offset);
+            sql += ("\nOFFSET " + offset);
         }
         
         return healSickSql(sql);


[02/30] incubator-kylin git commit: update website download page and release note (for 1.1.1)

Posted by li...@apache.org.
update website download page and release note (for 1.1.1)

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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 4c889cd67becc52c13fa59c624f5a05ff07fe903
Parents: 30eabaf
Author: shaofengshi <sh...@apache.org>
Authored: Thu Nov 5 10:51:04 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 10:51:04 2015 +0800

----------------------------------------------------------------------
 website/_docs/release_notes.md | 7 +++++++
 website/download/index.cn.md   | 8 +++++---
 website/download/index.md      | 9 ++++++---
 3 files changed, 18 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4c889cd6/website/_docs/release_notes.md
----------------------------------------------------------------------
diff --git a/website/_docs/release_notes.md b/website/_docs/release_notes.md
index 36380c4..6e7ec38 100644
--- a/website/_docs/release_notes.md
+++ b/website/_docs/release_notes.md
@@ -16,6 +16,13 @@ or send to Apache Kylin mailing list:
 * User relative: [user@kylin.incubator.apache.org](mailto:user@kylin.incubator.apache.org)
 * Development relative: [dev@kylin.incubator.apache.org](mailto:dev@kylin.incubator.apache.org)
 
+## v1.1.1-incubating - 2015-11-04
+_Tag:_ [kylin-1.1.1-incubating](https://github.com/apache/incubator-kylin/tree/kylin-1.1.1-incubating)
+
+__Improvement__
+
+    * [KYLIN-999] - License check and cleanup for release
+
 ## v1.1-incubating - 2015-10-25
 _Tag:_ [kylin-1.1-incubating](https://github.com/apache/incubator-kylin/tree/kylin-1.1-incubating)
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4c889cd6/website/download/index.cn.md
----------------------------------------------------------------------
diff --git a/website/download/index.cn.md b/website/download/index.cn.md
index 1c3acf5..8681d69 100644
--- a/website/download/index.cn.md
+++ b/website/download/index.cn.md
@@ -17,13 +17,15 @@ __二进制包 for HBase 0.98/0.99__
   * [apache-kylin-1.1-incubating-bin.tar.gz](https://dist.apache.org/repos/dist/release/incubator/kylin/apache-kylin-1.1-incubating/apache-kylin-1.1-incubating-bin.tar.gz)
   * [安装帮助](http://kylin.incubator.apache.org/docs/install)
 
-__二进制包 for HBase 1.1__
-越来越多的用户开始部署使用HBase 1.1或更高版本,我们提供一个在HBase 1.1上编译的Kylin快照二进制包;请注意,这不是一个正式的发布版,没有经过完整的测试。
+__二进制包 for HBase 1.1.3及更高版本__
+越来越多的用户开始部署使用HBase 1.1或更高版本,我们提供一个在HBase 1.1上编译的Kylin快照二进制包;
+请注意此安装包需要HBase 1.1.3或更高版本,之前版本中有一个已知的关于fuzzy key过滤器的缺陷,会导致Kylin查询结果缺少记录: [HBASE-14269](https://issues.apache.org/jira/browse/HBASE-14269)
+此外请注意,这不是一个正式的发布版,没有经过完整的测试。
 
   * [apache-kylin-1.2-HBase1.1-incubating-SNAPSHOT-bin.tar.gz](https://dist.apache.org/repos/dist/dev/incubator/kylin/apache-kylin-1.2-incubating-snapshot/apache-kylin-1.2-HBase1.1-incubating-SNAPSHOT-bin.tar.gz)
   * Git commit [3623dd6ff93d76141bb6a5fb623a3421ae78ca93](https://github.com/apache/incubator-kylin/commit/3623dd6ff93d76141bb6a5fb623a3421ae78ca93)
 
-对于HBase 1.0用户,建议您升级到1.1或降级到0.98/0.99.
+对于HBase 1.0用户,建议您升级到1.1.3或降级到0.98/0.99.
 
 __构建二进制包__
 可以从各个版本或当前最新的开发分支中生成二进制包,请参考这篇[帮助文档](https://kylin.incubator.apache.org/development/howto_package.html)

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4c889cd6/website/download/index.md
----------------------------------------------------------------------
diff --git a/website/download/index.md b/website/download/index.md
index 03cef7a..ac893f4 100644
--- a/website/download/index.md
+++ b/website/download/index.md
@@ -18,13 +18,16 @@ For convenience, there’s binary package also available:
 * [apache-kylin-1.1-incubating-bin.tar.gz](https://dist.apache.org/repos/dist/release/incubator/kylin/apache-kylin-1.1-incubating/apache-kylin-1.1-incubating-bin.tar.gz)
 * [Installation Guide](http://kylin.incubator.apache.org/docs/install)
 
-__Binary Package (for running on HBase 1.1)__
-As there are more and more HBase 1.1 deployments, an binary snapshot build for HBase 1.1 is provided; Please note this is not a formal release, and it is not fully tested:
+__Binary Package (for running on HBase 1.1.3 or above)__
+As there are more and more HBase 1.1 deployments, an binary snapshot build for HBase 1.1.3+ is provided; 
+Note the requirement of HBase version 1.1.3 (or above). There is a known bug in HBase earlier versions about fuzzy key filter that will cause
+missing rows or lesser aggregations in Kylin query result: [HBASE-14269](https://issues.apache.org/jira/browse/HBASE-14269)
+Also, please aware this is not a formal release, and it is not fully tested:
 
 * [apache-kylin-1.2-HBase1.1-incubating-SNAPSHOT-bin.tar.gz](https://dist.apache.org/repos/dist/dev/incubator/kylin/apache-kylin-1.2-incubating-snapshot/apache-kylin-1.2-HBase1.1-incubating-SNAPSHOT-bin.tar.gz)
 * Git commit [3623dd6ff93d76141bb6a5fb623a3421ae78ca93](https://github.com/apache/incubator-kylin/commit/3623dd6ff93d76141bb6a5fb623a3421ae78ca93) 
 
-If you're using HBase 1.0, we suggest you to upgrade to 1.1 or downgrade to 0.98/0.99.
+If you're using HBase 1.0, we suggest you to upgrade to 1.1.3+ or downgrade to 0.98/0.99.
 
 __Build Binary Package__
 To build binary package from any version even latest development branch, please refer to this [guide](https://kylin.incubator.apache.org/development/howto_package.html)


[29/30] incubator-kylin git commit: KYLIN-920 & KYLIN-782 Upgrade to HBase 1.1 (with help from murkrishn )

Posted by li...@apache.org.
KYLIN-920 & KYLIN-782 Upgrade to HBase 1.1 (with help from murkrishn <mu...@ebay.com>)

Signed-off-by: Li, Yang <ya...@ebay.com>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 9d63d9a94cec6a6ca50a417c0dba215d79e58e6d
Parents: ca1e695
Author: Yang Li <li...@apache.org>
Authored: Sun Aug 16 20:22:13 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Nov 18 10:35:00 2015 +0800

----------------------------------------------------------------------
 .../common/persistence/HBaseConnection.java     | 251 ++++----
 .../common/persistence/HBaseResourceStore.java  |  31 +-
 .../common/util/HBaseRegionSizeCalculator.java  |  41 +-
 .../kylin/common/util/BasicHadoopTest.java      |  11 +-
 .../kylin/job/cube/GarbageCollectionStep.java   |  22 +-
 .../kylin/job/hadoop/cube/CubeHFileJob.java     |  18 +-
 .../job/hadoop/cube/StorageCleanupJob.java      |  26 +-
 .../kylin/job/hadoop/hbase/CreateHTableJob.java |   8 +-
 .../hadoop/invertedindex/IICreateHFileJob.java  |  22 +-
 .../hadoop/invertedindex/IICreateHTableJob.java |  11 +-
 .../apache/kylin/job/tools/CleanHtableCLI.java  |   8 +-
 .../kylin/job/tools/CubeMigrationCLI.java       |  24 +-
 .../kylin/job/tools/DeployCoprocessorCLI.java   | 625 ++++++++++---------
 .../job/tools/GridTableHBaseBenchmark.java      |  37 +-
 .../kylin/job/tools/HtableAlterMetadataCLI.java |   8 +-
 .../apache/kylin/job/tools/RowCounterCLI.java   |  11 +-
 .../org/apache/kylin/job/ExportHBaseData.java   |  18 +-
 .../kylin/job/hadoop/hbase/TestHbaseClient.java |  13 +-
 .../kylin/job/tools/HBaseRowDigestTest.java     |  11 +-
 monitor/pom.xml                                 |   6 +
 .../kylin/monitor/MonitorMetaManager.java       |  49 +-
 pom.xml                                         |  14 +-
 .../apache/kylin/rest/service/AclService.java   |  38 +-
 .../apache/kylin/rest/service/CubeService.java  |  35 +-
 .../apache/kylin/rest/service/QueryService.java |  21 +-
 .../apache/kylin/rest/service/UserService.java  |  27 +-
 .../storage/hbase/CubeSegmentTupleIterator.java |  21 +-
 .../kylin/storage/hbase/CubeStorageEngine.java  |   4 +-
 .../storage/hbase/HBaseClientKVIterator.java    | 187 +++---
 .../hbase/InvertedIndexStorageEngine.java       | 114 ++--
 .../kylin/storage/hbase/PingHBaseCLI.java       | 179 +++---
 .../storage/hbase/RegionScannerAdapter.java     |  10 +-
 .../hbase/SerializedHBaseTupleIterator.java     |   4 +-
 .../endpoint/EndpointTupleIterator.java         |  15 +-
 .../hbase/coprocessor/endpoint/IIEndpoint.java  |   2 +-
 .../observer/AggregateRegionObserver.java       |   2 +-
 .../observer/AggregationScanner.java            |  14 +-
 .../observer/ObserverAggregationCache.java      |  10 +-
 .../coprocessor/observer/ObserverEnabler.java   |   4 +-
 .../storage/hbase/InvertedIndexHBaseTest.java   | 227 ++++---
 .../observer/AggregateRegionObserverTest.java   |  72 +--
 .../minicluster/HiveMiniClusterTest.java        |   3 +-
 42 files changed, 1124 insertions(+), 1130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/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..85a08a1 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
@@ -1,123 +1,128 @@
-/*
- * 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.common.persistence;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author yangli9
- * 
- */
-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 {
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            @Override
-            public void run() {
-                for (HConnection conn : ConnPool.values()) {
-                    try {
-                        conn.close();
-                    } catch (IOException e) {
-                        e.printStackTrace();
-                    }
-                }
-            }
-        });
-    }
-
-    public static HConnection get(String url) {
-        // find configuration
-        Configuration conf = ConfigCache.get(url);
-        if (conf == null) {
-            conf = HadoopUtil.newHBaseConfiguration(url);
-            ConfigCache.put(url, conf);
-        }
-
-        HConnection connection = ConnPool.get(url);
-        try {
-            // I don't use DCL since recreate a connection is not a big issue.
-            if (connection == null) {
-                connection = HConnectionManager.createConnection(conf);
-                ConnPool.put(url, connection);
-            }
-        } catch (Throwable t) {
-            throw new StorageException("Error when open connection " + url, t);
-        }
-
-        return connection;
-    }
-
-    public static void createHTableIfNeeded(String hbaseUrl, String tableName, String... families) throws IOException {
-        createHTableIfNeeded(HBaseConnection.get(hbaseUrl), tableName, families);
-    }
-
-    public static void createHTableIfNeeded(HConnection conn, String tableName, String... families) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
-
-        try {
-            boolean tableExist = false;
-            try {
-                hbase.getTableDescriptor(TableName.valueOf(tableName));
-                tableExist = true;
-            } catch (TableNotFoundException e) {
-            }
-
-            if (tableExist) {
-                logger.debug("HTable '" + tableName + "' already exists");
-                return;
-            }
-
-            logger.debug("Creating HTable '" + tableName + "'");
-
-            HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
-
-            if (null != families && families.length > 0) {
-                for (String family : families) {
-                    HColumnDescriptor fd = new HColumnDescriptor(family);
-                    fd.setInMemory(true); // metadata tables are best in memory
-                    desc.addFamily(fd);
-                }
-            }
-            hbase.createTable(desc);
-
-            logger.debug("HTable '" + tableName + "' created");
-        } finally {
-            hbase.close();
-        }
-    }
-}
+/*
+ * 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.common.persistence;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author yangli9
+ * 
+ */
+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, Connection> ConnPool = new ConcurrentHashMap<String, Connection>();
+
+    static {
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                for (Connection conn : ConnPool.values()) {
+                    try {
+                        conn.close();
+                    } catch (IOException e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+        });
+    }
+    
+    public static Connection get() {
+        return get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+    }
+
+    public static Connection get(String url) {
+        // find configuration
+        Configuration conf = ConfigCache.get(url);
+        if (conf == null) {
+            conf = HadoopUtil.newHBaseConfiguration(url);
+            ConfigCache.put(url, conf);
+        }
+
+        Connection connection = ConnPool.get(url);
+        try {
+            // I don't use DCL since recreate a connection is not a big issue.
+            if (connection == null) {
+                connection = ConnectionFactory.createConnection(conf);
+                ConnPool.put(url, connection);
+            }
+        } catch (Throwable t) {
+            throw new StorageException("Error when open connection " + url, t);
+        }
+
+        return connection;
+    }
+
+    public static void createHTableIfNeeded(String hbaseUrl, String tableName, String... families) throws IOException {
+        createHTableIfNeeded(HBaseConnection.get(hbaseUrl), tableName, families);
+    }
+
+    public static void createHTableIfNeeded(Connection conn, String tableName, String... families) throws IOException {
+        Admin admin = conn.getAdmin();
+
+        try {
+            boolean tableExist = false;
+            try {
+                admin.getTableDescriptor(TableName.valueOf(tableName));
+                tableExist = true;
+            } catch (TableNotFoundException e) {
+            }
+
+            if (tableExist) {
+                logger.debug("HTable '" + tableName + "' already exists");
+                return;
+            }
+
+            logger.debug("Creating HTable '" + tableName + "'");
+
+            HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+
+            if (null != families && families.length > 0) {
+                for (String family : families) {
+                    HColumnDescriptor fd = new HColumnDescriptor(family);
+                    fd.setInMemory(true); // metadata tables are best in memory
+                    desc.addFamily(fd);
+                }
+            }
+            admin.createTable(desc);
+
+            logger.debug("HTable '" + tableName + "' created");
+        } finally {
+            admin.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
index 35a62b5..2338aac 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
@@ -33,13 +33,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
@@ -76,7 +77,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     //    final Map<String, String> tableNameMap; // path prefix ==> HBase table name
 
-    private HConnection getConnection() throws IOException {
+    private Connection getConnection() throws IOException {
         return HBaseConnection.get(hbaseUrl);
     }
 
@@ -119,7 +120,7 @@ public class HBaseResourceStore extends ResourceStore {
 
         ArrayList<String> result = new ArrayList<String>();
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         Scan scan = new Scan(startRow, endRow);
         scan.setFilter(new KeyOnlyFilter());
         try {
@@ -154,7 +155,7 @@ public class HBaseResourceStore extends ResourceStore {
         scan.addColumn(B_FAMILY, B_COLUMN_TS);
         scan.addColumn(B_FAMILY, B_COLUMN);
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         List<RawResource> result = Lists.newArrayList();
         try {
             ResultScanner scanner = table.getScanner(scan);
@@ -216,13 +217,12 @@ public class HBaseResourceStore extends ResourceStore {
         IOUtils.copy(content, bout);
         bout.close();
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             byte[] row = Bytes.toBytes(resPath);
             Put put = buildPut(resPath, ts, row, bout.toByteArray(), table);
 
             table.put(put);
-            table.flushCommits();
         } finally {
             IOUtils.closeQuietly(table);
         }
@@ -230,7 +230,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             byte[] row = Bytes.toBytes(resPath);
             byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
@@ -242,8 +242,6 @@ public class HBaseResourceStore extends ResourceStore {
                 throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + real + ", but it is " + oldTS);
             }
 
-            table.flushCommits();
-
             return newTS;
         } finally {
             IOUtils.closeQuietly(table);
@@ -252,11 +250,10 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected void deleteResourceImpl(String resPath) throws IOException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             Delete del = new Delete(Bytes.toBytes(resPath));
             table.delete(del);
-            table.flushCommits();
         } finally {
             IOUtils.closeQuietly(table);
         }
@@ -281,7 +278,7 @@ public class HBaseResourceStore extends ResourceStore {
                 scan.addColumn(B_FAMILY, B_COLUMN_TS);
         }
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             ResultScanner scanner = table.getScanner(scan);
             Result result = null;
@@ -300,7 +297,7 @@ public class HBaseResourceStore extends ResourceStore {
         return endRow;
     }
 
-    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
+    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, Table table) throws IOException {
         Path redirectPath = bigCellHDFSPath(resPath);
         Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
@@ -326,7 +323,7 @@ public class HBaseResourceStore extends ResourceStore {
         return redirectPath;
     }
 
-    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
+    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, Table table) throws IOException {
         int kvSizeLimit = this.kylinConfig.getHBaseKeyValueSize();
         if (content.length > kvSizeLimit) {
             writeLargeCellToHdfs(resPath, content, table);
@@ -334,8 +331,8 @@ public class HBaseResourceStore extends ResourceStore {
         }
 
         Put put = new Put(row);
-        put.add(B_FAMILY, B_COLUMN, content);
-        put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
+        put.addColumn(B_FAMILY, B_COLUMN, content);
+        put.addColumn(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
 
         return put;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java b/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
index 093ac9e..ccbb6f0 100644
--- a/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
+++ b/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
@@ -23,19 +23,24 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,30 +58,31 @@ public class HBaseRegionSizeCalculator {
     /**
      * Computes size of each region for table and given column families.
      * */
-    public HBaseRegionSizeCalculator(HTable table) throws IOException {
-        this(table, new HBaseAdmin(table.getConfiguration()));
-    }
-
-    /** Constructor for unit testing */
-    HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
-
+    public HBaseRegionSizeCalculator(String tableName , Connection hbaseConnection) throws IOException {
+        Table table = null;
+        Admin admin = null;
+        
         try {
+            table = hbaseConnection.getTable(TableName.valueOf(tableName));
+            admin = hbaseConnection.getAdmin();
+            
             if (!enabled(table.getConfiguration())) {
                 logger.info("Region size calculation disabled.");
                 return;
             }
 
-            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
+            logger.info("Calculating region sizes for table \"" + table.getName() + "\".");
 
             // Get regions for table.
-            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
+            RegionLocator regionLocator = hbaseConnection.getRegionLocator(table.getName());
+            List<HRegionLocation> regionLocationList = regionLocator.getAllRegionLocations();
             Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
 
-            for (HRegionInfo regionInfo : tableRegionInfos) {
-                tableRegions.add(regionInfo.getRegionName());
+            for (HRegionLocation hRegionLocation : regionLocationList) {
+                tableRegions.add(hRegionLocation.getRegionInfo().getRegionName());
             }
 
-            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
+            ClusterStatus clusterStatus = admin.getClusterStatus();
             Collection<ServerName> servers = clusterStatus.getServers();
             final long megaByte = 1024L * 1024L;
 
@@ -99,7 +105,8 @@ public class HBaseRegionSizeCalculator {
                 }
             }
         } finally {
-            hBaseAdmin.close();
+            IOUtils.closeQuietly(table);
+            IOUtils.closeQuietly(admin);
         }
 
     }
@@ -124,4 +131,4 @@ public class HBaseRegionSizeCalculator {
     public Map<byte[], Long> getRegionSizeMap() {
         return Collections.unmodifiableMap(sizeMap);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java b/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java
index 6d2762c..481fc6c 100644
--- a/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java
+++ b/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java
@@ -21,12 +21,11 @@ package org.apache.kylin.common.util;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -56,16 +55,14 @@ public class BasicHadoopTest {
         cf.setBlocksize(4 * 1024 * 1024); // set to 4MB
         tableDesc.addFamily(cf);
 
-        Configuration conf = HBaseConfiguration.create();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get().getAdmin();
         admin.createTable(tableDesc);
         admin.close();
     }
 
     @Test
     public void testRetriveHtableHost() throws IOException {
-        Configuration conf = HBaseConfiguration.create();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables();
         for (HTableDescriptor table : tableDescriptors) {
             String value = table.getValue("KYLIN_HOST");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
index f2f1fc0..8c61a3a 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
@@ -24,14 +24,13 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.job.cmd.ShellCmdOutput;
 import org.apache.kylin.job.exception.ExecuteException;
@@ -99,19 +98,18 @@ public class GarbageCollectionStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Configuration conf = HBaseConfiguration.create();
-            HBaseAdmin admin = null;
+            Admin admin = null;
             try {
-                admin = new HBaseAdmin(conf);
+                admin = HBaseConnection.get().getAdmin();
                 for (String table : oldTables) {
-                    if (admin.tableExists(table)) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                    if (admin.tableExists(TableName.valueOf(table))) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf(table));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(table)) {
-                                admin.disableTable(table);
+                            if (admin.isTableEnabled(TableName.valueOf(table))) {
+                                admin.disableTable(TableName.valueOf(table));
                             }
-                            admin.deleteTable(table);
+                            admin.deleteTable(TableName.valueOf(table));
                             logger.debug("Dropped HBase table " + table);
                             output.append("Dropped HBase table " + table + " \n");
                         } else {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
index 3c1e4a5..6f36eff 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
@@ -19,11 +19,15 @@
 package org.apache.kylin.job.hadoop.cube;
 
 import org.apache.commons.cli.Options;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
 import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.mapreduce.Job;
@@ -31,6 +35,7 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.job.constant.BatchConstants;
@@ -47,6 +52,8 @@ public class CubeHFileJob extends AbstractHadoopJob {
 
     public int run(String[] args) throws Exception {
         Options options = new Options();
+        Connection connection = null;
+        Table table = null;
 
         try {
             options.addOption(OPTION_JOB_NAME);
@@ -80,10 +87,12 @@ public class CubeHFileJob extends AbstractHadoopJob {
             attachKylinPropsAndMetadata(cube, job.getConfiguration());
 
             String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
-            HTable htable = new HTable(conf, tableName);
+            connection = HBaseConnection.get();
+            table = connection.getTable(TableName.valueOf(tableName));
+            RegionLocator regionLocator = connection.getRegionLocator(TableName.valueOf(tableName));
 
             //Automatic config !
-            HFileOutputFormat.configureIncrementalLoad(job, htable);
+            HFileOutputFormat2.configureIncrementalLoad(job, table, regionLocator);
 
             // set block replication to 3 for hfiles
             conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, "3");
@@ -96,6 +105,7 @@ public class CubeHFileJob extends AbstractHadoopJob {
             printUsage(options);
             throw e;
         } finally {
+            IOUtils.closeQuietly(table);
             if (job != null)
                 cleanupTempConfFile(job.getConfiguration());
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
index 3b25ee1..184b6cd 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
@@ -18,6 +18,13 @@
 
 package org.apache.kylin.job.hadoop.cube;
 
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
@@ -28,10 +35,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -50,13 +59,6 @@ import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.StringReader;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
 /**
  * @author ysong1
  */
@@ -107,7 +109,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
         IIManager iiManager = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
 
         // get all kylin hbase tables
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -141,9 +143,9 @@ public class StorageCleanupJob extends AbstractHadoopJob {
             // drop tables
             for (String htableName : allTablesNeedToBeDropped) {
                 log.info("Deleting HBase table " + htableName);
-                if (hbaseAdmin.tableExists(htableName)) {
-                    hbaseAdmin.disableTable(htableName);
-                    hbaseAdmin.deleteTable(htableName);
+                if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+                    hbaseAdmin.disableTable(TableName.valueOf(htableName));
+                    hbaseAdmin.deleteTable(TableName.valueOf(htableName));
                     log.info("Deleted HBase table " + htableName);
                 } else {
                     log.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
index 027c0ca..9f5e062 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
@@ -25,11 +25,10 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
@@ -42,6 +41,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -81,7 +81,7 @@ public class CreateHTableJob extends AbstractHadoopJob {
         tableDesc.setValue(IRealizationConstants.HTableTag, config.getMetadataUrlPrefix());
 
         Configuration conf = HadoopUtil.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get().getAdmin();
 
         try {
             if (User.isHBaseSecurityEnabled(conf)) {
@@ -139,7 +139,7 @@ public class CreateHTableJob extends AbstractHadoopJob {
 
             byte[][] splitKeys = getSplits(conf, partitionFilePath);
 
-            if (admin.tableExists(tableName)) {
+            if (admin.tableExists(TableName.valueOf(tableName))) {
                 // admin.disableTable(tableName);
                 // admin.deleteTable(tableName);
                 throw new RuntimeException("HBase table " + tableName + " exists!");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
index c032bbc..fa42148 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
@@ -19,17 +19,20 @@
 package org.apache.kylin.job.hadoop.invertedindex;
 
 import org.apache.commons.cli.Options;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.slf4j.Logger;
@@ -45,6 +48,8 @@ public class IICreateHFileJob extends AbstractHadoopJob {
 
     public int run(String[] args) throws Exception {
         Options options = new Options();
+        Connection connection = null;
+        Table table = null;
 
         try {
             options.addOption(OPTION_JOB_NAME);
@@ -69,8 +74,11 @@ public class IICreateHFileJob extends AbstractHadoopJob {
             job.setMapOutputValueClass(KeyValue.class);
 
             String tableName = getOptionValue(OPTION_HTABLE_NAME);
-            HTable htable = new HTable(HBaseConfiguration.create(getConf()), tableName);
-            HFileOutputFormat.configureIncrementalLoad(job, htable);
+
+            connection = HBaseConnection.get();
+            table = connection.getTable(TableName.valueOf(tableName));
+            RegionLocator regionLocator = connection.getRegionLocator(TableName.valueOf(tableName));
+            HFileOutputFormat2.configureIncrementalLoad(job, table, regionLocator);
 
             this.deletePath(job.getConfiguration(), output);
 
@@ -78,6 +86,8 @@ public class IICreateHFileJob extends AbstractHadoopJob {
         } catch (Exception e) {
             printUsage(options);
             throw e;
+        } finally {
+            IOUtils.closeQuietly(table);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
index 32d065a..63777ef 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
@@ -24,11 +24,12 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.invertedindex.IIInstance;
@@ -78,10 +79,10 @@ public class IICreateHTableJob extends AbstractHadoopJob {
             DeployCoprocessorCLI.deployCoprocessor(tableDesc);
 
             // drop the table first
-            HBaseAdmin admin = new HBaseAdmin(conf);
-            if (admin.tableExists(tableName)) {
-                admin.disableTable(tableName);
-                admin.deleteTable(tableName);
+            Admin admin = HBaseConnection.get().getAdmin();
+            if (admin.tableExists(TableName.valueOf(tableName))) {
+                admin.disableTable(TableName.valueOf(tableName));
+                admin.deleteTable(TableName.valueOf(tableName));
             }
 
             // create table

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java b/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
index b6e5af5..7fc1d72 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
@@ -21,11 +21,10 @@ package org.apache.kylin.job.tools;
 import java.io.IOException;
 
 import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -54,8 +53,7 @@ public class CleanHtableCLI extends AbstractHadoopJob {
     }
 
     private void clean() throws IOException {
-        Configuration conf = HBaseConfiguration.create();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
 
         for (HTableDescriptor descriptor : hbaseAdmin.listTables()) {
             String name = descriptor.getNameAsString().toLowerCase();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java b/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
index 2d4b0bf..23e2722 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
@@ -23,14 +23,15 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
@@ -73,7 +74,7 @@ public class CubeMigrationCLI {
     private static ResourceStore srcStore;
     private static ResourceStore dstStore;
     private static FileSystem hdfsFS;
-    private static HBaseAdmin hbaseAdmin;
+    private static Admin hbaseAdmin;
 
     public static void main(String[] args) throws IOException, InterruptedException {
 
@@ -113,8 +114,7 @@ public class CubeMigrationCLI {
 
         checkAndGetHbaseUrl();
 
-        Configuration conf = HBaseConfiguration.create();
-        hbaseAdmin = new HBaseAdmin(conf);
+        hbaseAdmin = HBaseConnection.get().getAdmin();
 
         hdfsFS = FileSystem.get(new Configuration());
 
@@ -130,6 +130,8 @@ public class CubeMigrationCLI {
         } else {
             showOpts();
         }
+
+        IOUtils.closeQuietly(hbaseAdmin);
     }
 
     public static void moveCube(String srcCfgUri, String dstCfgUri, String cubeName, String projectName, String overwriteIfExists, String realExecute) throws IOException, InterruptedException {
@@ -284,10 +286,10 @@ public class CubeMigrationCLI {
         case CHANGE_HTABLE_HOST: {
             String tableName = (String) opt.params[0];
             HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-            hbaseAdmin.disableTable(tableName);
+            hbaseAdmin.disableTable(TableName.valueOf(tableName));
             desc.setValue(IRealizationConstants.HTableTag, dstConfig.getMetadataUrlPrefix());
-            hbaseAdmin.modifyTable(tableName, desc);
-            hbaseAdmin.enableTable(tableName);
+            hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+            hbaseAdmin.enableTable(TableName.valueOf(tableName));
             logger.info("CHANGE_HTABLE_HOST is completed");
             break;
         }
@@ -400,10 +402,10 @@ public class CubeMigrationCLI {
         case CHANGE_HTABLE_HOST: {
             String tableName = (String) opt.params[0];
             HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-            hbaseAdmin.disableTable(tableName);
+            hbaseAdmin.disableTable(TableName.valueOf(tableName));
             desc.setValue(IRealizationConstants.HTableTag, srcConfig.getMetadataUrlPrefix());
-            hbaseAdmin.modifyTable(tableName, desc);
-            hbaseAdmin.enableTable(tableName);
+            hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+            hbaseAdmin.enableTable(TableName.valueOf(tableName));
             break;
         }
         case COPY_FILE_IN_META: {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java b/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
index 5482684..239c7ec 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
@@ -1,313 +1,314 @@
-/*
- * 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.job.tools;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Matcher;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author yangli9
- */
-public class DeployCoprocessorCLI {
-
-    private static final Logger logger = LoggerFactory.getLogger(DeployCoprocessorCLI.class);
-
-    public static final String OBSERVER_CLS_NAME = "org.apache.kylin.storage.hbase.coprocessor.observer.AggregateRegionObserver";
-    public static final String ENDPOINT_CLS_NAMAE = "org.apache.kylin.storage.hbase.coprocessor.endpoint.IIEndpoint";
-
-    public static void main(String[] args) throws IOException {
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        FileSystem fileSystem = FileSystem.get(hconf);
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
-
-        String localCoprocessorJar = new File(args[0]).getAbsolutePath();
-        logger.info("Identify coprocessor jar " + localCoprocessorJar);
-
-        List<String> tableNames = getHTableNames(kylinConfig);
-        logger.info("Identify tables " + tableNames);
-
-        Set<String> oldJarPaths = getCoprocessorJarPaths(hbaseAdmin, tableNames);
-        logger.info("Old coprocessor jar: " + oldJarPaths);
-
-        Path hdfsCoprocessorJar = uploadCoprocessorJar(localCoprocessorJar, fileSystem, oldJarPaths);
-        logger.info("New coprocessor jar: " + hdfsCoprocessorJar);
-
-        List<String> processedTables = resetCoprocessorOnHTables(hbaseAdmin, hdfsCoprocessorJar, tableNames);
-
-        // Don't remove old jars, missing coprocessor jar will fail hbase
-        // removeOldJars(oldJarPaths, fileSystem);
-
-        hbaseAdmin.close();
-
-        logger.info("Processed " + processedTables);
-        logger.info("Active coprocessor jar: " + hdfsCoprocessorJar);
-    }
-
-    public static void deployCoprocessor(HTableDescriptor tableDesc) {
-        try {
-            initHTableCoprocessor(tableDesc);
-            logger.info("hbase table " + tableDesc.getName() + " deployed with coprocessor.");
-
-        } catch (Exception ex) {
-            logger.error("Error deploying coprocessor on " + tableDesc.getName(), ex);
-            logger.error("Will try creating the table without coprocessor.");
-        }
-    }
-
-    private static void initHTableCoprocessor(HTableDescriptor desc) throws IOException {
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+/*
+ * 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.job.tools;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IIManager;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author yangli9
+ */
+public class DeployCoprocessorCLI {
+
+    private static final Logger logger = LoggerFactory.getLogger(DeployCoprocessorCLI.class);
+
+    public static final String OBSERVER_CLS_NAME = "org.apache.kylin.storage.hbase.coprocessor.observer.AggregateRegionObserver";
+    public static final String ENDPOINT_CLS_NAMAE = "org.apache.kylin.storage.hbase.coprocessor.endpoint.IIEndpoint";
+
+    public static void main(String[] args) throws IOException {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        FileSystem fileSystem = FileSystem.get(hconf);
-
-        String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();
-        Path hdfsCoprocessorJar = DeployCoprocessorCLI.uploadCoprocessorJar(localCoprocessorJar, fileSystem, null);
-
-        DeployCoprocessorCLI.addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
-    }
-
-    public static void addCoprocessorOnHTable(HTableDescriptor desc, Path hdfsCoprocessorJar) throws IOException {
-        logger.info("Add coprocessor on " + desc.getNameAsString());
-        desc.addCoprocessor(ENDPOINT_CLS_NAMAE, hdfsCoprocessorJar, 1000, null);
-        desc.addCoprocessor(OBSERVER_CLS_NAME, hdfsCoprocessorJar, 1001, null);
-    }
-
-    public static void resetCoprocessor(String tableName, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
-        logger.info("Disable " + tableName);
-        hbaseAdmin.disableTable(tableName);
-
-        logger.info("Unset coprocessor on " + tableName);
-        HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-        while (desc.hasCoprocessor(OBSERVER_CLS_NAME)) {
-            desc.removeCoprocessor(OBSERVER_CLS_NAME);
-        }
-        while (desc.hasCoprocessor(ENDPOINT_CLS_NAMAE)) {
-            desc.removeCoprocessor(ENDPOINT_CLS_NAMAE);
-        }
-
-        addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
-        hbaseAdmin.modifyTable(tableName, desc);
-
-        logger.info("Enable " + tableName);
-        hbaseAdmin.enableTable(tableName);
-    }
-
-    private static List<String> resetCoprocessorOnHTables(HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
-        List<String> processed = new ArrayList<String>();
-
-        for (String tableName : tableNames) {
-            try {
-                resetCoprocessor(tableName, hbaseAdmin, hdfsCoprocessorJar);
-                processed.add(tableName);
-            } catch (IOException ex) {
-                logger.error("Error processing " + tableName, ex);
-            }
-        }
-        return processed;
-    }
-
-    public static Path getNewestCoprocessorJar(KylinConfig config, FileSystem fileSystem) throws IOException {
-        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, config);
-        FileStatus newestJar = null;
-        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
-            if (fileStatus.getPath().toString().endsWith(".jar")) {
-                if (newestJar == null) {
-                    newestJar = fileStatus;
-                } else {
-                    if (newestJar.getModificationTime() < fileStatus.getModificationTime())
-                        newestJar = fileStatus;
-                }
-            }
-        }
-        if (newestJar == null)
-            return null;
-
-        Path path = newestJar.getPath().makeQualified(fileSystem.getUri(), null);
-        logger.info("The newest coprocessor is " + path.toString());
-        return path;
-    }
-
-    public static Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem, Set<String> oldJarPaths) throws IOException {
-        Path uploadPath = null;
-        File localCoprocessorFile = new File(localCoprocessorJar);
-
-        // check existing jars
-        if (oldJarPaths == null) {
-            oldJarPaths = new HashSet<String>();
-        }
-        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, KylinConfig.getInstanceFromEnv());
-        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
-            if (fileStatus.getLen() == localCoprocessorJar.length() && fileStatus.getModificationTime() == localCoprocessorFile.lastModified()) {
-                uploadPath = fileStatus.getPath();
-                break;
-            }
-            String filename = fileStatus.getPath().toString();
-            if (filename.endsWith(".jar")) {
-                oldJarPaths.add(filename);
-            }
-        }
-
-        // upload if not existing
-        if (uploadPath == null) {
-            // figure out a unique new jar file name
-            Set<String> oldJarNames = new HashSet<String>();
-            for (String path : oldJarPaths) {
-                oldJarNames.add(new Path(path).getName());
-            }
-            String baseName = getBaseFileName(localCoprocessorJar);
-            String newName = null;
-            int i = 0;
-            while (newName == null) {
-                newName = baseName + "-" + (i++) + ".jar";
-                if (oldJarNames.contains(newName))
-                    newName = null;
-            }
-
-            // upload
-            uploadPath = new Path(coprocessorDir, newName);
-            FileInputStream in = null;
-            FSDataOutputStream out = null;
-            try {
-                in = new FileInputStream(localCoprocessorFile);
-                out = fileSystem.create(uploadPath);
-                IOUtils.copy(in, out);
-            } finally {
-                IOUtils.closeQuietly(in);
-                IOUtils.closeQuietly(out);
-            }
-
-            fileSystem.setTimes(uploadPath, localCoprocessorFile.lastModified(), -1);
-
-        }
-
-        uploadPath = uploadPath.makeQualified(fileSystem.getUri(), null);
-        return uploadPath;
-    }
-
-    private static String getBaseFileName(String localCoprocessorJar) {
-        File localJar = new File(localCoprocessorJar);
-        String baseName = localJar.getName();
-        if (baseName.endsWith(".jar"))
-            baseName = baseName.substring(0, baseName.length() - ".jar".length());
-        return baseName;
-    }
-
-    private static Path getCoprocessorHDFSDir(FileSystem fileSystem, KylinConfig config) throws IOException {
-        String hdfsWorkingDirectory = config.getHdfsWorkingDirectory();
-        Path coprocessorDir = new Path(hdfsWorkingDirectory, "coprocessor");
-        fileSystem.mkdirs(coprocessorDir);
-        return coprocessorDir;
-    }
-
-    private static Set<String> getCoprocessorJarPaths(HBaseAdmin hbaseAdmin, List<String> tableNames) throws IOException {
-        HashSet<String> result = new HashSet<String>();
-
-        for (String tableName : tableNames) {
-            HTableDescriptor tableDescriptor = null;
-            try {
-                tableDescriptor = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-            } catch (TableNotFoundException e) {
-                logger.warn("Table not found " + tableName, e);
-                continue;
-            }
-
-            Matcher keyMatcher;
-            Matcher valueMatcher;
-            for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : tableDescriptor.getValues().entrySet()) {
-                keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
-                if (!keyMatcher.matches()) {
-                    continue;
-                }
-                valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes.toString(e.getValue().get()));
-                if (!valueMatcher.matches()) {
-                    continue;
-                }
-
-                String jarPath = valueMatcher.group(1).trim();
-                String clsName = valueMatcher.group(2).trim();
-
-                if (OBSERVER_CLS_NAME.equals(clsName)) {
-                    result.add(jarPath);
-                }
-            }
-        }
-
-        return result;
-    }
-
-    private static List<String> getHTableNames(KylinConfig config) {
-        CubeManager cubeMgr = CubeManager.getInstance(config);
-
-        ArrayList<String> result = new ArrayList<String>();
-        for (CubeInstance cube : cubeMgr.listAllCubes()) {
-            for (CubeSegment seg : cube.getSegments(SegmentStatusEnum.READY)) {
-                String tableName = seg.getStorageLocationIdentifier();
-                if (StringUtils.isBlank(tableName) == false) {
-                    result.add(tableName);
-                    System.out.println("added new table: " + tableName);
-                }
-            }
-        }
-
-        for (IIInstance ii : IIManager.getInstance(config).listAllIIs()) {
-            for (IISegment seg : ii.getSegments(SegmentStatusEnum.READY)) {
-                String tableName = seg.getStorageLocationIdentifier();
-                if (StringUtils.isBlank(tableName) == false) {
-                    result.add(tableName);
-                    System.out.println("added new table: " + tableName);
-                }
-            }
-        }
-
-        return result;
-    }
-}
+        FileSystem fileSystem = FileSystem.get(hconf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
+
+        String localCoprocessorJar = new File(args[0]).getAbsolutePath();
+        logger.info("Identify coprocessor jar " + localCoprocessorJar);
+
+        List<String> tableNames = getHTableNames(kylinConfig);
+        logger.info("Identify tables " + tableNames);
+
+        Set<String> oldJarPaths = getCoprocessorJarPaths(hbaseAdmin, tableNames);
+        logger.info("Old coprocessor jar: " + oldJarPaths);
+
+        Path hdfsCoprocessorJar = uploadCoprocessorJar(localCoprocessorJar, fileSystem, oldJarPaths);
+        logger.info("New coprocessor jar: " + hdfsCoprocessorJar);
+
+        List<String> processedTables = resetCoprocessorOnHTables(hbaseAdmin, hdfsCoprocessorJar, tableNames);
+
+        // Don't remove old jars, missing coprocessor jar will fail hbase
+        // removeOldJars(oldJarPaths, fileSystem);
+
+        hbaseAdmin.close();
+
+        logger.info("Processed " + processedTables);
+        logger.info("Active coprocessor jar: " + hdfsCoprocessorJar);
+    }
+
+    public static void deployCoprocessor(HTableDescriptor tableDesc) {
+        try {
+            initHTableCoprocessor(tableDesc);
+            logger.info("hbase table " + tableDesc.getTableName() + " deployed with coprocessor.");
+
+        } catch (Exception ex) {
+            logger.error("Error deploying coprocessor on " + tableDesc.getTableName(), ex);
+            logger.error("Will try creating the table without coprocessor.");
+        }
+    }
+
+    private static void initHTableCoprocessor(HTableDescriptor desc) throws IOException {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
+        FileSystem fileSystem = FileSystem.get(hconf);
+
+        String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();
+        Path hdfsCoprocessorJar = DeployCoprocessorCLI.uploadCoprocessorJar(localCoprocessorJar, fileSystem, null);
+
+        DeployCoprocessorCLI.addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
+    }
+
+    public static void addCoprocessorOnHTable(HTableDescriptor desc, Path hdfsCoprocessorJar) throws IOException {
+        logger.info("Add coprocessor on " + desc.getNameAsString());
+        desc.addCoprocessor(ENDPOINT_CLS_NAMAE, hdfsCoprocessorJar, 1000, null);
+        desc.addCoprocessor(OBSERVER_CLS_NAME, hdfsCoprocessorJar, 1001, null);
+    }
+
+    public static void resetCoprocessor(String tableName, Admin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
+        logger.info("Disable " + tableName);
+        hbaseAdmin.disableTable(TableName.valueOf(tableName));
+
+        logger.info("Unset coprocessor on " + tableName);
+        HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+        while (desc.hasCoprocessor(OBSERVER_CLS_NAME)) {
+            desc.removeCoprocessor(OBSERVER_CLS_NAME);
+        }
+        while (desc.hasCoprocessor(ENDPOINT_CLS_NAMAE)) {
+            desc.removeCoprocessor(ENDPOINT_CLS_NAMAE);
+        }
+
+        addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
+        hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+
+        logger.info("Enable " + tableName);
+        hbaseAdmin.enableTable(TableName.valueOf(tableName));
+    }
+
+    private static List<String> resetCoprocessorOnHTables(Admin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
+        List<String> processed = new ArrayList<String>();
+
+        for (String tableName : tableNames) {
+            try {
+                resetCoprocessor(tableName, hbaseAdmin, hdfsCoprocessorJar);
+                processed.add(tableName);
+            } catch (IOException ex) {
+                logger.error("Error processing " + tableName, ex);
+            }
+        }
+        return processed;
+    }
+
+    public static Path getNewestCoprocessorJar(KylinConfig config, FileSystem fileSystem) throws IOException {
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, config);
+        FileStatus newestJar = null;
+        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
+            if (fileStatus.getPath().toString().endsWith(".jar")) {
+                if (newestJar == null) {
+                    newestJar = fileStatus;
+                } else {
+                    if (newestJar.getModificationTime() < fileStatus.getModificationTime())
+                        newestJar = fileStatus;
+                }
+            }
+        }
+        if (newestJar == null)
+            return null;
+
+        Path path = newestJar.getPath().makeQualified(fileSystem.getUri(), null);
+        logger.info("The newest coprocessor is " + path.toString());
+        return path;
+    }
+
+    public static Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem, Set<String> oldJarPaths) throws IOException {
+        Path uploadPath = null;
+        File localCoprocessorFile = new File(localCoprocessorJar);
+
+        // check existing jars
+        if (oldJarPaths == null) {
+            oldJarPaths = new HashSet<String>();
+        }
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, KylinConfig.getInstanceFromEnv());
+        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
+            if (fileStatus.getLen() == localCoprocessorJar.length() && fileStatus.getModificationTime() == localCoprocessorFile.lastModified()) {
+                uploadPath = fileStatus.getPath();
+                break;
+            }
+            String filename = fileStatus.getPath().toString();
+            if (filename.endsWith(".jar")) {
+                oldJarPaths.add(filename);
+            }
+        }
+
+        // upload if not existing
+        if (uploadPath == null) {
+            // figure out a unique new jar file name
+            Set<String> oldJarNames = new HashSet<String>();
+            for (String path : oldJarPaths) {
+                oldJarNames.add(new Path(path).getName());
+            }
+            String baseName = getBaseFileName(localCoprocessorJar);
+            String newName = null;
+            int i = 0;
+            while (newName == null) {
+                newName = baseName + "-" + (i++) + ".jar";
+                if (oldJarNames.contains(newName))
+                    newName = null;
+            }
+
+            // upload
+            uploadPath = new Path(coprocessorDir, newName);
+            FileInputStream in = null;
+            FSDataOutputStream out = null;
+            try {
+                in = new FileInputStream(localCoprocessorFile);
+                out = fileSystem.create(uploadPath);
+                IOUtils.copy(in, out);
+            } finally {
+                IOUtils.closeQuietly(in);
+                IOUtils.closeQuietly(out);
+            }
+
+            fileSystem.setTimes(uploadPath, localCoprocessorFile.lastModified(), -1);
+
+        }
+
+        uploadPath = uploadPath.makeQualified(fileSystem.getUri(), null);
+        return uploadPath;
+    }
+
+    private static String getBaseFileName(String localCoprocessorJar) {
+        File localJar = new File(localCoprocessorJar);
+        String baseName = localJar.getName();
+        if (baseName.endsWith(".jar"))
+            baseName = baseName.substring(0, baseName.length() - ".jar".length());
+        return baseName;
+    }
+
+    private static Path getCoprocessorHDFSDir(FileSystem fileSystem, KylinConfig config) throws IOException {
+        String hdfsWorkingDirectory = config.getHdfsWorkingDirectory();
+        Path coprocessorDir = new Path(hdfsWorkingDirectory, "coprocessor");
+        fileSystem.mkdirs(coprocessorDir);
+        return coprocessorDir;
+    }
+
+    private static Set<String> getCoprocessorJarPaths(Admin hbaseAdmin, List<String> tableNames) throws IOException {
+        HashSet<String> result = new HashSet<String>();
+
+        for (String tableName : tableNames) {
+            HTableDescriptor tableDescriptor = null;
+            try {
+                tableDescriptor = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            } catch (TableNotFoundException e) {
+                logger.warn("Table not found " + tableName, e);
+                continue;
+            }
+
+            Matcher keyMatcher;
+            Matcher valueMatcher;
+            for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : tableDescriptor.getValues().entrySet()) {
+                keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
+                if (!keyMatcher.matches()) {
+                    continue;
+                }
+                valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes.toString(e.getValue().get()));
+                if (!valueMatcher.matches()) {
+                    continue;
+                }
+
+                String jarPath = valueMatcher.group(1).trim();
+                String clsName = valueMatcher.group(2).trim();
+
+                if (OBSERVER_CLS_NAME.equals(clsName)) {
+                    result.add(jarPath);
+                }
+            }
+        }
+
+        return result;
+    }
+
+    private static List<String> getHTableNames(KylinConfig config) {
+        CubeManager cubeMgr = CubeManager.getInstance(config);
+
+        ArrayList<String> result = new ArrayList<String>();
+        for (CubeInstance cube : cubeMgr.listAllCubes()) {
+            for (CubeSegment seg : cube.getSegments(SegmentStatusEnum.READY)) {
+                String tableName = seg.getStorageLocationIdentifier();
+                if (StringUtils.isBlank(tableName) == false) {
+                    result.add(tableName);
+                    System.out.println("added new table: " + tableName);
+                }
+            }
+        }
+
+        for (IIInstance ii : IIManager.getInstance(config).listAllIIs()) {
+            for (IISegment seg : ii.getSegments(SegmentStatusEnum.READY)) {
+                String tableName = seg.getStorageLocationIdentifier();
+                if (StringUtils.isBlank(tableName) == false) {
+                    result.add(tableName);
+                    System.out.println("added new table: " + tableName);
+                }
+            }
+        }
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java b/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java
index 70e1df6..5fe5e58 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java
@@ -28,13 +28,13 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.Bytes;
@@ -74,8 +74,7 @@ public class GridTableHBaseBenchmark {
     public static void testGridTable(double hitRatio, double indexRatio) throws IOException {
         System.out.println("Testing grid table scanning, hit ratio " + hitRatio + ", index ratio " + indexRatio);
         String hbaseUrl = "hbase"; // use hbase-site.xml on classpath
-
-        HConnection conn = HBaseConnection.get(hbaseUrl);
+        Connection conn = HBaseConnection.get(hbaseUrl);
         createHTableIfNeeded(conn, TEST_TABLE);
         prepareData(conn);
 
@@ -91,10 +90,10 @@ public class GridTableHBaseBenchmark {
 
     }
 
-    private static void testColumnScan(HConnection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
+    private static void testColumnScan(Connection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
         Stats stats = new Stats("COLUMN_SCAN");
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -122,20 +121,20 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void testRowScanNoIndexFullScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexFullScan(Connection conn, boolean[] hits) throws IOException {
         fullScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_FULL"));
     }
 
-    private static void testRowScanNoIndexSkipScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexSkipScan(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_SKIP"));
     }
 
-    private static void testRowScanWithIndex(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanWithIndex(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_IDX"));
     }
 
-    private static void fullScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void fullScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -156,11 +155,11 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void jumpScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
+    private static void jumpScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
 
         final int jumpThreshold = 6; // compensate for Scan() overhead, totally by experience
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
 
             stats.markStart();
@@ -204,8 +203,8 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void prepareData(HConnection conn) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void prepareData(Connection conn) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
 
         try {
             // check how many rows existing
@@ -232,7 +231,7 @@ public class GridTableHBaseBenchmark {
                 byte[] rowkey = Bytes.toBytes(i);
                 Put put = new Put(rowkey);
                 byte[] cell = randomBytes();
-                put.add(CF, QN, cell);
+                put.addColumn(CF, QN, cell);
                 table.put(put);
                 nBytes += cell.length;
                 dot(i, N_ROWS);
@@ -258,8 +257,8 @@ public class GridTableHBaseBenchmark {
         return bytes;
     }
 
-    private static void createHTableIfNeeded(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    private static void createHTableIfNeeded(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
 
         try {
             boolean tableExist = false;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java b/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
index 53930e3..e283748 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
@@ -23,12 +23,11 @@ import java.io.IOException;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -71,8 +70,7 @@ public class HtableAlterMetadataCLI extends AbstractHadoopJob {
     }
 
     private void alter() throws IOException {
-        Configuration conf = HBaseConfiguration.create();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
         HTableDescriptor table = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
 
         hbaseAdmin.disableTable(table.getTableName());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java b/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java
index 3329d27..4d44088 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java
@@ -22,11 +22,12 @@ import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.slf4j.Logger;
@@ -69,8 +70,8 @@ public class RowCounterCLI {
 
         logger.info("My Scan " + scan.toString());
 
-        HConnection conn = HConnectionManager.createConnection(conf);
-        HTableInterface tableInterface = conn.getTable(htableName);
+        Connection conn = ConnectionFactory.createConnection(conf);
+        Table tableInterface = conn.getTable(TableName.valueOf(htableName));
 
         Iterator<Result> iterator = tableInterface.getScanner(scan).iterator();
         int counter = 0;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java b/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
index e784a41..95a483d 100644
--- a/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
+++ b/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
@@ -22,10 +22,11 @@ import java.io.File;
 import java.io.IOException;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
@@ -39,7 +40,7 @@ public class ExportHBaseData {
     KylinConfig kylinConfig;
     HTableDescriptor[] allTables;
     Configuration config;
-    HBaseAdmin hbase;
+    Admin admin;
     CliCommandExecutor cli;
     String exportHdfsFolder;
     String exportLocalFolderParent;
@@ -75,12 +76,11 @@ public class ExportHBaseData {
         int cut = metadataUrl.indexOf('@');
         tableNameBase = metadataUrl.substring(0, cut);
         String hbaseUrl = cut < 0 ? metadataUrl : metadataUrl.substring(cut + 1);
-
-        HConnection conn = HBaseConnection.get(hbaseUrl);
+        Connection conn = HBaseConnection.get(hbaseUrl);
         try {
-            hbase = new HBaseAdmin(conn);
-            config = hbase.getConfiguration();
-            allTables = hbase.listTables();
+            admin = conn.getAdmin();
+            config = admin.getConfiguration();
+            allTables = admin.listTables();
         } catch (IOException e) {
             e.printStackTrace();
             throw e;
@@ -89,6 +89,8 @@ public class ExportHBaseData {
 
     public void tearDown() {
 
+        // close hbase admin
+        IOUtils.closeQuietly(admin);
         // cleanup hdfs
         try {
             if (cli != null && exportHdfsFolder != null) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
index f2b9ed6..5a04d20 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
@@ -22,8 +22,11 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.Bytes;
 
 /**
@@ -90,13 +93,15 @@ public class TestHbaseClient {
         conf.set("hbase.zookeeper.quorum", "hbase_host");
         conf.set("zookeeper.znode.parent", "/hbase-unsecure");
 
-        HTable table = new HTable(conf, "test1");
+        Connection connection = ConnectionFactory.createConnection(conf);
+        Table table = connection.getTable(TableName.valueOf("test1"));
         Put put = new Put(Bytes.toBytes("row1"));
 
-        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual1"), Bytes.toBytes("val1"));
-        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual2"), Bytes.toBytes("val2"));
+        put.addColumn(Bytes.toBytes("colfam1"), Bytes.toBytes("qual1"), Bytes.toBytes("val1"));
+        put.addColumn(Bytes.toBytes("colfam1"), Bytes.toBytes("qual2"), Bytes.toBytes("val2"));
 
         table.put(put);
         table.close();
+        connection.close();
     }
 }



[22/30] incubator-kylin git commit: update bin/metastore.sh to use bin/kylin.sh

Posted by li...@apache.org.
update bin/metastore.sh to use bin/kylin.sh

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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 1afb32dc55558562d3b096a77bcb0faf7e5495a0
Parents: db7f8be
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 16 16:30:04 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Nov 16 16:30:04 2015 +0800

----------------------------------------------------------------------
 bin/metastore.sh | 11 ++++-------
 1 file changed, 4 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1afb32dc/bin/metastore.sh
----------------------------------------------------------------------
diff --git a/bin/metastore.sh b/bin/metastore.sh
index 53aaa80..39593d4 100755
--- a/bin/metastore.sh
+++ b/bin/metastore.sh
@@ -28,9 +28,6 @@
 dir=$(dirname ${0})
 source ${dir}/check-env.sh
 
-_jobjar=`ls ${KYLIN_HOME}/lib |grep kylin-job`
-_fulljobjar="${KYLIN_HOME}/lib/${_jobjar}"
-
 if [ $1 == "backup" ]
 then
 
@@ -41,7 +38,7 @@ then
     echo "Starting backup to ${_file}"
     mkdir -p ${_file}
 
-    hbase  org.apache.hadoop.util.RunJar ${_fulljobjar}   org.apache.kylin.common.persistence.ResourceTool download ${_file}
+    ${KYLIN_HOME}/bin/kylin.sh org.apache.kylin.common.persistence.ResourceTool download ${_file}
     echo "metadata store backed up to ${_file}"
 
 elif [ $1 == "restore" ]
@@ -49,17 +46,17 @@ then
 
     _file=$2
     echo "Starting restoring $_file"
-    hbase  org.apache.hadoop.util.RunJar  ${_fulljobjar}   org.apache.kylin.common.persistence.ResourceTool upload $_file
+    ${KYLIN_HOME}/bin/kylin.sh org.apache.kylin.common.persistence.ResourceTool upload $_file
 
 elif [ $1 == "reset" ]
 then
 
-    hbase  org.apache.hadoop.util.RunJar ${_fulljobjar}   org.apache.kylin.common.persistence.ResourceTool  reset
+    ${KYLIN_HOME}/bin/kylin.sh org.apache.kylin.common.persistence.ResourceTool  reset
     
 elif [ $1 == "clean" ]
 then
 
-    hbase  org.apache.hadoop.util.RunJar ${_fulljobjar}  org.apache.kylin.job.hadoop.cube.MetadataCleanupJob "${@:2}"
+    ${KYLIN_HOME}/bin/kylin.sh org.apache.kylin.job.hadoop.cube.MetadataCleanupJob "${@:2}"
 
 else
     echo "usage: metastore.sh backup"


[10/30] incubator-kylin git commit: add v1.1 release blog

Posted by li...@apache.org.
add v1.1 release blog


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 70e3b39c5566018a7a9781a19a08e354a017749d
Parents: 824f96a
Author: Luke Han <lu...@apache.org>
Authored: Mon Nov 2 20:39:22 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Thu Nov 5 13:49:07 2015 +0800

----------------------------------------------------------------------
 .../blog/2015-09-06-release-v1.0-incubating.md  |  2 +-
 .../2015-10-25-release-v1.1-incubating.cn.md    | 53 ++++++++++++++++++++
 .../blog/2015-10-25-release-v1.1-incubating.md  | 24 +++++----
 3 files changed, 68 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/70e3b39c/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-06-release-v1.0-incubating.md b/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
index 085c7a1..cd13cc6 100644
--- a/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
+++ b/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
@@ -1,6 +1,6 @@
 ---
 layout: post-blog
-title:  Apache Kylin 1.0 (incubating) Release Announcement
+title:  Apache Kylin v1.0 (incubating) Release Announcement
 date:   2015-09-06 17:28:00
 author: Luke Han
 categories: blog

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/70e3b39c/website/_posts/blog/2015-10-25-release-v1.1-incubating.cn.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-10-25-release-v1.1-incubating.cn.md b/website/_posts/blog/2015-10-25-release-v1.1-incubating.cn.md
new file mode 100644
index 0000000..7d994cc
--- /dev/null
+++ b/website/_posts/blog/2015-10-25-release-v1.1-incubating.cn.md
@@ -0,0 +1,53 @@
+---
+layout: post-blog
+title:  Apache Kylin v1.1 (incubating) 正式发布
+date:   2015-10-25 17:28:00
+author: Luke Han
+categories: blog
+---
+
+Apache Kylin社区非常高兴宣布Apache Kylin v1.1 (incubating)正式发布.
+ 
+Apache Kylin一个开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据,最初由eBay Inc. 开发并贡献至开源社区。
+
+下载Apache Kylin v1.1 (incubating) 源代码及二进制安装包, 
+请访问[下载](http://kylin.incubator.apache.org/cn/download/)页面.
+
+这是一个主要的版本发布带来了更稳定,健壮及更好管理的版本,Apache Kylin社区解决了56个issue,包括Bug修复,功能增强及一些新特性等。
+
+## 主要变化
+
+__Kylin 核心功能增强__
+
+* 支持Cube数据保留时间设置 [KYLIN-906](https://issues.apache.org/jira/browse/KYLIN-906)
+* 升级Apache Calcite至1.4 [KYLIN-1047](https://issues.apache.org/jira/browse/KYLIN-1047) 
+* 在Cube构建成功后清理Hive中间文件 [KYLIN-589](https://issues.apache.org/jira/browse/KYLIN-589)
+* 当Hive返回空值时继续Cube构建任务 [KYLIN-772](https://issues.apache.org/jira/browse/KYLIN-772)
+* 支持可配置HBase压缩算法,包括Snappy及GZip [KYLIN-956](https://issues.apache.org/jira/browse/KYLIN-956) 
+* 支持将Cube数据导入到独立的HBase集群 [KYLIN-957](https://issues.apache.org/jira/browse/KYLIN-957)
+* 将Roaring bitmaps引入InvertedIndex模块 [KYLIN-1034](https://issues.apache.org/jira/browse/KYLIN-1034)
+
+__主要Bug修复__
+
+* 当有多个IN条件时SQL执行缓慢 [KYLIN-740](https://issues.apache.org/jira/browse/KYLIN-740)
+* Jobs页面加载Bug [KYLIN-950](https://issues.apache.org/jira/browse/KYLIN-950)
+* 查询缓存没有随元数据更新而刷新 [KYLIN-771](https://issues.apache.org/jira/browse/KYLIN-771)
+* 在事实表上执行“select * from fact”不工作 [KYLIN-847](https://issues.apache.org/jira/browse/KYLIN-847)
+* SQL执行时报Float 不能转换成为Double异常 [KYLIN-918](https://issues.apache.org/jira/browse/KYLIN-918)
+* 更新Cube数据模型失败后元数据状态不一致 [KYLIN-958](https://issues.apache.org/jira/browse/KYLIN-958)
+* SQL中关键字"offset"的Bug [KYLIN-983](https://issues.apache.org/jira/browse/KYLIN-983)
+* 平均值函数AVG不工作 [KYLIN-985](https://issues.apache.org/jira/browse/KYLIN-985)
+* 字典中空值''导致Cube合并失败 [KYLIN-1004](https://issues.apache.org/jira/browse/KYLIN-1004)
+
+
+__升级__  
+我们建议从v0.7.x及v1.0升级到此版本已获得更好的性能,稳定性及Bug修复等。
+并且与社区最新特性及支持保持同步。
+
+__支持__  
+升级和使用过程中有任何问题,请: 
+提交至Kylin的JIRA: [https://issues.apache.org/jira/browse/KYLIN/](https://issues.apache.org/jira/browse/KYLIN/)  
+或者  
+发送邮件到Apache Kylin邮件列表: [dev@kylin.incubator.apache.org](mailto:dev@kylin.incubator.apache.org)  
+
+_感谢各位的贡献!_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/70e3b39c/website/_posts/blog/2015-10-25-release-v1.1-incubating.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-10-25-release-v1.1-incubating.md b/website/_posts/blog/2015-10-25-release-v1.1-incubating.md
index 645f5f3..409289a 100644
--- a/website/_posts/blog/2015-10-25-release-v1.1-incubating.md
+++ b/website/_posts/blog/2015-10-25-release-v1.1-incubating.md
@@ -1,19 +1,19 @@
 ---
 layout: post-blog
-title:  Apache Kylin 1.1 (incubating) Release Announcement
+title:  Apache Kylin v1.1 (incubating) Release Announcement
 date:   2015-10-25 17:28:00
 author: Luke Han
 categories: blog
 ---
 
-The Apache Kylin team is pleased to announce the release of Apache Kylin v1.1 (incubating). 
-Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface 
-and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets, original contributed from eBay Inc.
+The Apache Kylin community is pleased to announce the release of Apache Kylin v1.1 (incubating).
+ 
+Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets, original contributed from eBay Inc.
 
-To download Apache Kylin v1.1 (incubating) visit the [download](http://kylin.incubator.apache.org/download) page.
+To download Apache Kylin v1.1 (incubating) source code or binary package: 
+please visit the [download](http://kylin.incubator.apache.org/download) page.
 
-This is a major release which brings more stable, robust and well management version, Apache Kylin team resolved about 
-56 issues including bug fixes, improvements, and few new features.
+This is a major release which brings more stable, robust and well management version, Apache Kylin community resolved about 56 issues including bug fixes, improvements, and few new features.
 
 ## Change Highlights
 
@@ -40,10 +40,14 @@ __Main Bug Fixes__
 * Dictionary with '' value cause cube merge fail [KYLIN-1004](https://issues.apache.org/jira/browse/KYLIN-1004)
 
 
-__Upgrade__
-
+__Upgrade__  
 We recommend to upgrade to this version from v0.7.x and v1.0 for better performance, stability and bug fixes.
 Also to keep up to date with community with latest features and supports.
-Any issue or question during upgrade, please send to Apache Kylin dev mailing list: [dev@kylin.incubator.apache.org](mailto:dev@kylin.incubator.apache.org)
+
+__Support__  
+Any issue or question during upgrade, please 
+open JIRA to Kylin project: [https://issues.apache.org/jira/browse/KYLIN/](https://issues.apache.org/jira/browse/KYLIN/)  
+or  
+send mail to Apache Kylin dev mailing list: [dev@kylin.incubator.apache.org](mailto:dev@kylin.incubator.apache.org)  
 
 _Great thanks to everyone who contributed!_
\ No newline at end of file


[03/30] incubator-kylin git commit: KYLIN-999 fix for google fonts license and cleanup files

Posted by li...@apache.org.
KYLIN-999 fix for google fonts license and cleanup files

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 46c216371788b83bd178934be188558d6a7e6793
Parents: efd234e
Author: Luke Han <lu...@apache.org>
Authored: Wed Nov 4 10:16:23 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 11:16:21 2015 +0800

----------------------------------------------------------------------
 webapp/app/.gitignore                 |   0
 webapp/app/css/AdminLTE.css           |   1 -
 webapp/app/image/Himg.png             | Bin 913426 -> 0 bytes
 webapp/app/image/forkme_right_red.png | Bin 7927 -> 0 bytes
 4 files changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/46c21637/webapp/app/.gitignore
----------------------------------------------------------------------
diff --git a/webapp/app/.gitignore b/webapp/app/.gitignore
deleted file mode 100644
index e69de29..0000000

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/46c21637/webapp/app/css/AdminLTE.css
----------------------------------------------------------------------
diff --git a/webapp/app/css/AdminLTE.css b/webapp/app/css/AdminLTE.css
index d86593d..f42878d 100644
--- a/webapp/app/css/AdminLTE.css
+++ b/webapp/app/css/AdminLTE.css
@@ -1,4 +1,3 @@
-@import url(https://fonts.googleapis.com/css?family=Source+Sans+Pro:300,400,600,700,300italic,400italic,600italic);
 /*!
  *   AdminLTE v2.1.0
  *   Author: Almsaeed Studio

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/46c21637/webapp/app/image/Himg.png
----------------------------------------------------------------------
diff --git a/webapp/app/image/Himg.png b/webapp/app/image/Himg.png
deleted file mode 100644
index 9b34a77..0000000
Binary files a/webapp/app/image/Himg.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/46c21637/webapp/app/image/forkme_right_red.png
----------------------------------------------------------------------
diff --git a/webapp/app/image/forkme_right_red.png b/webapp/app/image/forkme_right_red.png
deleted file mode 100644
index 1e19c21..0000000
Binary files a/webapp/app/image/forkme_right_red.png and /dev/null differ


[18/30] incubator-kylin git commit: update download page with v1.1.1 release

Posted by li...@apache.org.
update download page with v1.1.1 release


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 819ebd61828153bc836474ae4a30a9858a075150
Parents: 6ad0786
Author: shaofengshi <sh...@apache.org>
Authored: Thu Nov 12 17:30:03 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 12 17:30:03 2015 +0800

----------------------------------------------------------------------
 website/_dev/howto_release.md | 2 +-
 website/download/index.cn.md  | 8 ++++----
 website/download/index.md     | 8 ++++----
 3 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/819ebd61/website/_dev/howto_release.md
----------------------------------------------------------------------
diff --git a/website/_dev/howto_release.md b/website/_dev/howto_release.md
index f6db22a..7ea8b54 100644
--- a/website/_dev/howto_release.md
+++ b/website/_dev/howto_release.md
@@ -283,7 +283,7 @@ $ checkHash apache-kylin-X.Y.Z-incubating-rcN
 ## Apache voting process  
 
 __Vote on Apache Kylin dev mailing list__  
-Release vote on dev list:  
+Release vote on dev list, use the commit id that generated by Maven release plugin, whose message looks like "[maven-release-plugin] prepare release kylin-x.x.x-incubating":  
 
 {% highlight text %}
 To: dev@kylin.incubator.apache.org

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/819ebd61/website/download/index.cn.md
----------------------------------------------------------------------
diff --git a/website/download/index.cn.md b/website/download/index.cn.md
index 8681d69..8eb8dbb 100644
--- a/website/download/index.cn.md
+++ b/website/download/index.cn.md
@@ -6,15 +6,15 @@ title: 下载
 __最新发布(源代码)__  
 最新发布的Apache Kylin可以从ASF网站下载::
 
-* [Apache Kylin v1.1-incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-1.1-incubating/)
+* [Apache Kylin v1.1.1-incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-1.1.1-incubating/)
 * [发布日志](http://kylin.incubator.apache.org/docs/release_notes.html)
-* Git 标签: [kylin-1.1-incubating](https://github.com/apache/incubator-kylin/tree/kylin-1.1-incubating)
-* Git Commit: [1955a2f9aea7b7f608f0496c00807715ea4246a5](https://github.com/apache/incubator-kylin/commit/1955a2f9aea7b7f608f0496c00807715ea4246a5)
+* Git 标签: [kylin-1.1.1-incubating](https://github.com/apache/incubator-kylin/tree/kylin-1.1.1-incubating)
+* Git Commit: [6a9499b4d0fabb54211a8a536c2e18d3fe8b4a5d](https://github.com/apache/incubator-kylin/commit/6a9499b4d0fabb54211a8a536c2e18d3fe8b4a5d)
 
 __二进制包 for HBase 0.98/0.99__
 为方便使用,我们提供预打包的二进制安装包:
 
-  * [apache-kylin-1.1-incubating-bin.tar.gz](https://dist.apache.org/repos/dist/release/incubator/kylin/apache-kylin-1.1-incubating/apache-kylin-1.1-incubating-bin.tar.gz)
+  * [apache-kylin-1.1.1-incubating-bin.tar.gz](https://dist.apache.org/repos/dist/release/incubator/kylin/apache-kylin-1.1.1-incubating/apache-kylin-1.1.1-incubating-bin.tar.gz)
   * [安装帮助](http://kylin.incubator.apache.org/docs/install)
 
 __二进制包 for HBase 1.1.3及更高版本__

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/819ebd61/website/download/index.md
----------------------------------------------------------------------
diff --git a/website/download/index.md b/website/download/index.md
index ac893f4..01b4444 100644
--- a/website/download/index.md
+++ b/website/download/index.md
@@ -7,15 +7,15 @@ permalink: /download/index.html
 __Latest Release(Source Code)__  
 The latest release of Apache Kylin can be downloaded from the ASF:
 
-* [Apache Kylin v1.1-incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-1.1-incubating/)
+* [Apache Kylin v1.1.1-incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-1.1.1-incubating/)
 * [Release Notes](http://kylin.incubator.apache.org/docs/release_notes.html)
-* Git Tag: [kylin-1.1-incubating](https://github.com/apache/incubator-kylin/tree/kylin-1.1-incubating)
-* Git Commit: [1955a2f9aea7b7f608f0496c00807715ea4246a5](https://github.com/apache/incubator-kylin/commit/1955a2f9aea7b7f608f0496c00807715ea4246a5)
+* Git Tag: [kylin-1.1.1-incubating](https://github.com/apache/incubator-kylin/tree/kylin-1.1.1-incubating)
+* Git Commit: [6a9499b4d0fabb54211a8a536c2e18d3fe8b4a5d](https://github.com/apache/incubator-kylin/commit/6a9499b4d0fabb54211a8a536c2e18d3fe8b4a5d)
 
 __Binary Package (for running on HBase 0.98/0.99)__
 For convenience, there’s binary package also available: 
 
-* [apache-kylin-1.1-incubating-bin.tar.gz](https://dist.apache.org/repos/dist/release/incubator/kylin/apache-kylin-1.1-incubating/apache-kylin-1.1-incubating-bin.tar.gz)
+* [apache-kylin-1.1.1-incubating-bin.tar.gz](https://dist.apache.org/repos/dist/release/incubator/kylin/apache-kylin-1.1.1-incubating/apache-kylin-1.1.1-incubating-bin.tar.gz)
 * [Installation Guide](http://kylin.incubator.apache.org/docs/install)
 
 __Binary Package (for running on HBase 1.1.3 or above)__


[08/30] incubator-kylin git commit: KYLIN-999 update rat rule in pom

Posted by li...@apache.org.
KYLIN-999 update rat rule in pom

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 782dd52ea1df7038500d7567fc5fa252bcfdb9be
Parents: b37e026
Author: Luke Han <lu...@apache.org>
Authored: Wed Nov 4 15:09:42 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 11:16:22 2015 +0800

----------------------------------------------------------------------
 pom.xml                                        | 194 ++++++++++----------
 src/main/config/assemblies/source-assembly.xml |   2 +
 2 files changed, 104 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/782dd52e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 16da25e..90ba294 100644
--- a/pom.xml
+++ b/pom.xml
@@ -598,97 +598,6 @@
                 </plugin>
             </plugins>
         </pluginManagement>
-        <plugins>
-            <!-- Apache-RAT checks for files without headers.
-                         If run on a messy developer's sandbox, it will fail.
-                         This serves as a reminder to only build a release in a clean
-                         sandbox! -->
-                    <plugin>
-                        <groupId>org.apache.rat</groupId>
-                        <artifactId>apache-rat-plugin</artifactId>
-                        <configuration>
-                            <numUnapprovedLicenses>0</numUnapprovedLicenses>
-                            <excludes>
-                                <exclude>DEPENDENCIES</exclude>
-                                <exclude>.idea/**</exclude>
-                                <exclude>.git/**</exclude>
-                                <exclude>.settings/**</exclude>
-                                <!-- text files without comments -->
-                                <exclude>**/*.csv</exclude>
-                                <exclude>**/*.json</exclude>
-                                <exclude>**/*.md</exclude>
-                                <!-- binary files -->
-                                <exclude>**/*.dict</exclude>
-                                <exclude>**/*.dic</exclude>
-                                <exclude>**/*.snapshot</exclude>
-                                <exclude>**/*.pdf</exclude>
-                                <exclude>**/*.log</exclude>
-                                <exclude>dist/**</exclude>
-                                <exclude>server/logs**</exclude>
-
-                                <exclude>**/.checkstyle</exclude>
-                                <!--Job's Test Data-->
-                                <exclude>**/src/test/resources/**</exclude>
-
-                                <!-- generated files -->
-                                <exclude>**/target/**</exclude>
-                                <exclude>lib/**</exclude>
-                                <!-- Kylin's website content -->
-                                <exclude>**/.sass-cache/**</exclude>
-
-                                <exclude>website/**</exclude>
-
-                                <!-- tomcat package -->
-                                <exclude>tomcat/**</exclude>
-                                <!-- front end libary and generated files -->
-                                <exclude>webapp/node_modules/**</exclude>
-                                <exclude>webapp/dist/**</exclude>
-                                <exclude>webapp/app/components/**</exclude>
-                                <!-- json configuration file-->
-                                <exclude>webapp/.bowerrc</exclude>
-                                <exclude>webapp/.jshintrc</exclude>
-                                <!-- generated dict files -->
-                                <exclude>dictionary/metastore_db/**</exclude>
-
-                                <!-- MIT license -->
-                                <exclude>webapp/app/css/AdminLTE.css</exclude>
-
-                                <!-- jdbc log -->
-                                <exclude>jdbc/kylin_jdbc.log*</exclude>
-                                <!-- server log -->
-                                <exclude>server/logs/**</exclude>
-
-                                <!-- HBase MiniCluster Testing Data, for testing only -->
-                                <exclude>examples/test_case_data/minicluster/hbase-export.tar.gz</exclude>
-                                <exclude>examples/test_case_data/**/*.xml</exclude>
-
-                            </excludes>
-                        </configuration>
-                        <executions>
-                            <execution>
-                                <phase>verify</phase>
-                                <goals>
-                                    <goal>check</goal>
-                                </goals>
-                            </execution>
-                        </executions>
-                        <dependencies>
-                            <dependency>
-                                <groupId>org.apache.maven.doxia</groupId>
-                                <artifactId>doxia-core</artifactId>
-                                <version>1.6</version>
-                                <exclusions>
-                                    <exclusion>
-                                        <groupId>xerces</groupId>
-                                        <artifactId>xercesImpl</artifactId>
-                                    </exclusion>
-                                </exclusions>
-                            </dependency>
-                        </dependencies>
-                    </plugin>
-
-            
-        </plugins>
     </build>
 
     <modules>
@@ -816,8 +725,108 @@
                         </configuration>
                     </plugin>
 
+                    <!-- Apache-RAT checks for files without headers.
+                         If run on a messy developer's sandbox, it will fail.
+                         This serves as a reminder to only build a release in a clean
+                         sandbox! -->
+                    <plugin>
+                        <groupId>org.apache.rat</groupId>
+                        <artifactId>apache-rat-plugin</artifactId>
+                        <configuration>
+                            <numUnapprovedLicenses>0</numUnapprovedLicenses>
+                            <excludes>
+                                <exclude>DEPENDENCIES</exclude>
+                                <exclude>README.*</exclude>
+                                <exclude>**/LICENSE*</exclude>
+                                <!-- Local git repo -->
+                                <exclude>.git/**</exclude>
+                                <!-- IDE files -->
+                                <exclude>.idea/**</exclude>
+                                <exclude>**/*.iml</exclude>                                
+                                <exclude>.settings/**</exclude>
+                                <exclude>**/.classpath</exclude>
+                                <exclude>**/.project</exclude>
+
+                                <!-- text files without comments -->
+                                <exclude>**/*.csv</exclude>
+                                <exclude>**/*.json</exclude>
+                                <exclude>**/*.md</exclude>
+                                <!-- binary files -->
+                                <exclude>**/*.dict</exclude>
+                                <exclude>**/*.dic</exclude>
+                                <exclude>**/*.snapshot</exclude>
+                                <exclude>**/*.log</exclude>
+                                
+                                <!-- image files constitute images required for documentation. .pptx contain the sources for images -->
+                                <exclude>**/*.png</exclude>
+                                <exclude>**/*.jpg</exclude>
+                                <exclude>**/*.gif</exclude>
+                                <exclude>**/*.ico</exclude>
+
+                                <exclude>server/logs**</exclude>
+
+                                <exclude>**/.checkstyle</exclude>
+                                <!--Job's Test Data-->
+                                <exclude>**/src/test/resources/**</exclude>
+
+                                <!-- generated files -->
+                                <exclude>**/target/**</exclude>
+                                <exclude>lib/**</exclude>
+                                <exclude>dist/**</exclude>
+                                <!-- Kylin's website content -->
+                                <exclude>**/.sass-cache/**</exclude>
+
+                                <exclude>website/**</exclude>
+
+                                <!-- front end libary and generated files -->
+                                <exclude>webapp/node_modules/**</exclude>
+                                <exclude>webapp/dist/**</exclude>
+                                <exclude>webapp/app/components/**</exclude>
+                                <!-- json configuration file-->
+                                <exclude>webapp/.bowerrc</exclude>
+                                <exclude>webapp/.jshintrc</exclude>
+                                <!-- generated dict files -->
+                                <exclude>dictionary/metastore_db/**</exclude>
+
+                                <!-- MIT license -->
+                                <exclude>webapp/app/css/AdminLTE.css</exclude>
+
+                                <!-- jdbc log -->
+                                <exclude>jdbc/kylin_jdbc.log*</exclude>
+                                <!-- server log -->
+                                <exclude>server/logs/**</exclude>
+
+                                <!-- HBase MiniCluster Testing Data, for testing only -->
+                                <exclude>examples/test_case_data/minicluster/hbase-export.tar.gz</exclude>
+                                <exclude>examples/test_case_data/**/*.xml</exclude>
+
+                            </excludes>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <phase>verify</phase>
+                                <goals>
+                                    <goal>check</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                        <dependencies>
+                            <dependency>
+                                <groupId>org.apache.maven.doxia</groupId>
+                                <artifactId>doxia-core</artifactId>
+                                <version>1.6</version>
+                                <exclusions>
+                                    <exclusion>
+                                        <groupId>xerces</groupId>
+                                        <artifactId>xercesImpl</artifactId>
+                                    </exclusion>
+                                </exclusions>
+                            </dependency>
+                        </dependencies>
+                    </plugin>
+
                     <!-- Override the parent assembly execution to customize the assembly
-              descriptor and final name. -->
+                        descriptor and final name. -->
                     <plugin>
                         <artifactId>maven-assembly-plugin</artifactId>
                         <executions>
@@ -833,6 +842,7 @@
                                     <appendAssemblyId>true</appendAssemblyId>
                                     <descriptor>src/main/config/assemblies/source-assembly.xml</descriptor>
                                     <finalName>apache-kylin-${project.version}</finalName>
+                                    <tarLongFileMode>gnu</tarLongFileMode>
                                 </configuration>
                             </execution>
                         </executions>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/782dd52e/src/main/config/assemblies/source-assembly.xml
----------------------------------------------------------------------
diff --git a/src/main/config/assemblies/source-assembly.xml b/src/main/config/assemblies/source-assembly.xml
index 9c28851..7844515 100644
--- a/src/main/config/assemblies/source-assembly.xml
+++ b/src/main/config/assemblies/source-assembly.xml
@@ -98,6 +98,8 @@ limitations under the License.
                 </exclude>
                 <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?lib(/.*)?]
                 </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?docs(/.*)?]
+                </exclude>
             </excludes>
         </fileSet>
         <!-- LICENSE, NOTICE, DEPENDENCIES, git.properties, etc. calculated at build time -->


[30/30] incubator-kylin git commit: Merge branch '1.x-HBase1.1.3' into tmp

Posted by li...@apache.org.
Merge branch '1.x-HBase1.1.3' into tmp


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: ba076d5c13ccfacdb1f5505ef782d8e9f89e55f1
Parents: 9d63d9a 5063c38
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Nov 18 10:37:30 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Nov 18 10:37:30 2015 +0800

----------------------------------------------------------------------

----------------------------------------------------------------------



[14/30] incubator-kylin git commit: KYLIN-1099 Remove dedup in dictionary value enumerator

Posted by li...@apache.org.
KYLIN-1099 Remove dedup in dictionary value enumerator

Signed-off-by: Li, Yang <ya...@ebay.com>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: a441237afbf45736eea74cf6638e34de60a82cea
Parents: 882ca61
Author: lidongsjtu <do...@ebay.com>
Authored: Mon Nov 2 09:55:05 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Nov 11 10:18:13 2015 +0800

----------------------------------------------------------------------
 .../dict/ListDictionaryValueEnumerator.java     | 50 ++++++++++++++++++++
 .../dict/MultipleDictionaryValueEnumerator.java | 10 +---
 .../kylin/dict/TableColumnValueEnumerator.java  | 13 ++---
 .../lookup/ListDictionaryValueEnumerator.java   | 50 --------------------
 .../job/hadoop/cube/MergeCuboidMapperTest.java  |  2 +-
 5 files changed, 55 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a441237a/dictionary/src/main/java/org/apache/kylin/dict/ListDictionaryValueEnumerator.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/org/apache/kylin/dict/ListDictionaryValueEnumerator.java b/dictionary/src/main/java/org/apache/kylin/dict/ListDictionaryValueEnumerator.java
new file mode 100644
index 0000000..9ae08a8
--- /dev/null
+++ b/dictionary/src/main/java/org/apache/kylin/dict/ListDictionaryValueEnumerator.java
@@ -0,0 +1,50 @@
+/*
+ * 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.dict;
+
+import org.apache.kylin.dict.IDictionaryValueEnumerator;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.ListIterator;
+
+/**
+ * Created by dongli on 10/28/15.
+ */
+public class ListDictionaryValueEnumerator implements IDictionaryValueEnumerator {
+    ListIterator<byte[]> listIterator;
+
+    public ListDictionaryValueEnumerator(List<byte[]> list) {
+        listIterator = list.listIterator();
+    }
+
+    @Override
+    public byte[] current() throws IOException {
+        return listIterator.next();
+    }
+
+    @Override
+    public boolean moveNext() throws IOException {
+        return listIterator.hasNext();
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a441237a/dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java b/dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java
index 4cf72d3..13f7394 100644
--- a/dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java
+++ b/dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java
@@ -19,12 +19,9 @@
 package org.apache.kylin.dict;
 
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
 import org.apache.kylin.common.util.Bytes;
 
 import java.io.IOException;
-import java.util.HashSet;
 import java.util.List;
 
 /**
@@ -32,7 +29,6 @@ import java.util.List;
  */
 @SuppressWarnings("rawtypes")
 public class MultipleDictionaryValueEnumerator implements IDictionaryValueEnumerator {
-    private HashSet<byte[]> dedup = Sets.newHashSet();
     private int curDictIndex = 0;
     private Dictionary curDict;
     private int curKey;
@@ -69,11 +65,7 @@ public class MultipleDictionaryValueEnumerator implements IDictionaryValueEnumer
                 }
             }
 
-            if (dedup.contains(curValue)) {
-                return moveNext();
-            } else {
-                return true;
-            }
+            return true;
         }
         curValue = null;
         return false;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a441237a/dictionary/src/main/java/org/apache/kylin/dict/TableColumnValueEnumerator.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/org/apache/kylin/dict/TableColumnValueEnumerator.java b/dictionary/src/main/java/org/apache/kylin/dict/TableColumnValueEnumerator.java
index 64a0e39..42f5791 100644
--- a/dictionary/src/main/java/org/apache/kylin/dict/TableColumnValueEnumerator.java
+++ b/dictionary/src/main/java/org/apache/kylin/dict/TableColumnValueEnumerator.java
@@ -18,13 +18,11 @@
 
 package org.apache.kylin.dict;
 
-import com.google.common.collect.Sets;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.dict.lookup.ReadableTable;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.Set;
 
 /**
  * Created by dongli on 10/28/15.
@@ -34,7 +32,6 @@ public class TableColumnValueEnumerator implements IDictionaryValueEnumerator {
     private ReadableTable.TableReader reader;
     private int colIndex;
     private byte[] colValue;
-    private Set<String> dedup = Sets.newHashSet();
 
     public TableColumnValueEnumerator(ReadableTable.TableReader reader, int colIndex) {
         this.reader = reader;
@@ -55,13 +52,9 @@ public class TableColumnValueEnumerator implements IDictionaryValueEnumerator {
                 }
                 colStrValue = split[colIndex];
             }
-            if (!dedup.contains(colStrValue)) {
-                dedup.add(colStrValue);
-                colValue = Bytes.toBytes(colStrValue);
-                return true;
-            } else {
-                return moveNext();
-            }
+
+            colValue = Bytes.toBytes(colStrValue);
+            return true;
 
         } else {
             colValue = null;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a441237a/dictionary/src/main/java/org/apache/kylin/dict/lookup/ListDictionaryValueEnumerator.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/org/apache/kylin/dict/lookup/ListDictionaryValueEnumerator.java b/dictionary/src/main/java/org/apache/kylin/dict/lookup/ListDictionaryValueEnumerator.java
deleted file mode 100644
index 581ee43..0000000
--- a/dictionary/src/main/java/org/apache/kylin/dict/lookup/ListDictionaryValueEnumerator.java
+++ /dev/null
@@ -1,50 +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.
- */
-
-package org.apache.kylin.dict.lookup;
-
-import org.apache.kylin.dict.IDictionaryValueEnumerator;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.ListIterator;
-
-/**
- * Created by dongli on 10/28/15.
- */
-public class ListDictionaryValueEnumerator implements IDictionaryValueEnumerator {
-    ListIterator<byte[]> listIterator;
-
-    public ListDictionaryValueEnumerator(List<byte[]> list) {
-        listIterator = list.listIterator();
-    }
-
-    @Override
-    public byte[] current() throws IOException {
-        return listIterator.next();
-    }
-
-    @Override
-    public boolean moveNext() throws IOException {
-        return listIterator.hasNext();
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a441237a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapperTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapperTest.java
index 7a2ebad..9a1fdfb 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapperTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapperTest.java
@@ -38,7 +38,7 @@ import org.apache.kylin.dict.DictionaryGenerator;
 import org.apache.kylin.dict.DictionaryInfo;
 import org.apache.kylin.dict.DictionaryManager;
 import org.apache.kylin.dict.TrieDictionary;
-import org.apache.kylin.dict.lookup.ListDictionaryValueEnumerator;
+import org.apache.kylin.dict.ListDictionaryValueEnumerator;
 import org.apache.kylin.dict.lookup.ReadableTable.TableSignature;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.TblColRef;


[16/30] incubator-kylin git commit: KYLIN-1139 disable hive concurrency lock

Posted by li...@apache.org.
KYLIN-1139 disable hive concurrency lock


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 9e8a7178babbaf5d548cdd51088c04a6e887dbba
Parents: 6b6be73
Author: shaofengshi <sh...@apache.org>
Authored: Thu Nov 12 16:40:25 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 12 16:40:25 2015 +0800

----------------------------------------------------------------------
 conf/kylin_job_conf.xml | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9e8a7178/conf/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/conf/kylin_job_conf.xml b/conf/kylin_job_conf.xml
index 1b718a0..ad3427c 100644
--- a/conf/kylin_job_conf.xml
+++ b/conf/kylin_job_conf.xml
@@ -67,4 +67,10 @@ limitations under the License. See accompanying LICENSE file.
         <value>2</value>
         <description>Block replication</description>
     </property>
+
+    <property>
+        <name>hive.support.concurrency</name>
+        <value>false</value>
+        <description>Hive concurrency lock</description>
+    </property>
 </configuration>
\ No newline at end of file


[23/30] incubator-kylin git commit: KYLIN-1152 Modify resource store to always return content and timestamp together

Posted by li...@apache.org.
KYLIN-1152 Modify resource store to always return content and timestamp together


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 2f788e71e7a12ec54cc99b5ec0305924847a178e
Parents: 1afb32d
Author: Li, Yang <ya...@ebay.com>
Authored: Tue Nov 17 13:15:54 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Nov 17 13:17:04 2015 +0800

----------------------------------------------------------------------
 .../common/persistence/FileResourceStore.java   | 16 +++------
 .../common/persistence/HBaseResourceStore.java  | 36 ++++++++++----------
 .../kylin/common/persistence/RawResource.java   |  6 ++--
 .../kylin/common/persistence/ResourceStore.java | 27 ++++++---------
 .../kylin/common/persistence/ResourceTool.java  | 12 +++----
 .../apache/kylin/job/CubeMetadataUpgrade.java   |  9 ++---
 .../kylin/job/hadoop/AbstractHadoopJob.java     | 12 +++----
 .../job/hadoop/cube/MetadataCleanupJob.java     |  9 +++--
 .../kylin/job/tools/CubeMigrationCLI.java       |  9 +++--
 .../java/org/apache/kylin/job/DeployUtil.java   |  4 +--
 .../kylin/job/dataGen/FactTableGenerator.java   |  7 ++--
 .../apache/kylin/metadata/MetadataManager.java  |  7 ++--
 .../org/apache/kylin/query/test/H2Database.java | 15 ++++----
 .../apache/kylin/query/test/KylinQueryTest.java |  4 +--
 14 files changed, 82 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
index 9648f6b..98c6b18 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
@@ -75,14 +75,14 @@ public class FileResourceStore extends ResourceStore {
             for (String resource : resources) {
                 if (resource.compareTo(rangeStart) >= 0 && resource.compareTo(rangeEnd) <= 0) {
                     if (existsImpl(resource)) {
-                        result.add(new RawResource(getResourceImpl(resource), getResourceTimestampImpl(resource)));
+                        result.add(getResourceImpl(resource));
                     }
                 }
             }
             return result;
         } catch (IOException ex) {
             for (RawResource rawResource : result) {
-                IOUtils.closeQuietly(rawResource.resource);
+                IOUtils.closeQuietly(rawResource.inputStream);
             }
             throw ex;
         } catch (Exception ex) {
@@ -91,21 +91,15 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
-    protected InputStream getResourceImpl(String resPath) throws IOException {
+    protected RawResource getResourceImpl(String resPath) throws IOException {
         File f = file(resPath);
         if (f.exists() && f.isFile())
-            return new FileInputStream(file(resPath));
+            return new RawResource(new FileInputStream(f), f.lastModified());
         else
             return null;
     }
 
     @Override
-    protected long getResourceTimestampImpl(String resPath) throws IOException {
-        File f = file(resPath);
-        return f.lastModified();
-    }
-
-    @Override
     protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException {
         File f = file(resPath);
         f.getParentFile().mkdirs();
@@ -128,7 +122,7 @@ public class FileResourceStore extends ResourceStore {
         putResourceImpl(resPath, new ByteArrayInputStream(content), newTS);
 
         // some FS lose precision on given time stamp
-        return getResourceTimestamp(resPath);
+        return f.lastModified();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
index 1c4a7ba..2b14345 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
@@ -141,7 +141,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected boolean existsImpl(String resPath) throws IOException {
-        Result r = getByScan(resPath, null, null);
+        Result r = getByScan(resPath, false, false);
         return r != null;
     }
 
@@ -163,7 +163,7 @@ public class HBaseResourceStore extends ResourceStore {
             }
         } catch (IOException e) {
             for (RawResource rawResource : result) {
-                IOUtils.closeQuietly(rawResource.resource);
+                IOUtils.closeQuietly(rawResource.inputStream);
             }
             throw e;
         } finally {
@@ -179,7 +179,7 @@ public class HBaseResourceStore extends ResourceStore {
         byte[] value = r.getValue(B_FAMILY, B_COLUMN);
         if (value.length == 0) {
             Path redirectPath = bigCellHDFSPath(resPath);
-            Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
+            Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
             FileSystem fileSystem = FileSystem.get(hconf);
 
             return fileSystem.open(redirectPath);
@@ -197,15 +197,12 @@ public class HBaseResourceStore extends ResourceStore {
     }
 
     @Override
-    protected InputStream getResourceImpl(String resPath) throws IOException {
-        Result r = getByScan(resPath, B_FAMILY, B_COLUMN);
-        return getInputStream(resPath, r);
-    }
-
-    @Override
-    protected long getResourceTimestampImpl(String resPath) throws IOException {
-        Result r = getByScan(resPath, B_FAMILY, B_COLUMN_TS);
-        return getTimestamp(r);
+    protected RawResource getResourceImpl(String resPath) throws IOException {
+        Result r = getByScan(resPath, true, true);
+        if (r == null)
+            return null;
+        else
+            return new RawResource(getInputStream(resPath, r), getTimestamp(r));
     }
 
     @Override
@@ -236,8 +233,8 @@ public class HBaseResourceStore extends ResourceStore {
 
             boolean ok = table.checkAndPut(row, B_FAMILY, B_COLUMN_TS, bOldTS, put);
             if (!ok) {
-                long real = getResourceTimestamp(resPath);
-                throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + real + ", but it is " + oldTS);
+                long real = getTimestamp(getByScan(resPath, false, true));
+                throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + real + ", but it is " + oldTS);
             }
 
             table.flushCommits();
@@ -265,15 +262,18 @@ public class HBaseResourceStore extends ResourceStore {
         return getAllInOneTableName() + "(key='" + resPath + "')@" + kylinConfig.getMetadataUrl();
     }
 
-    private Result getByScan(String path, byte[] family, byte[] column) throws IOException {
+    private Result getByScan(String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
         byte[] startRow = Bytes.toBytes(path);
         byte[] endRow = plusZero(startRow);
 
         Scan scan = new Scan(startRow, endRow);
-        if (family == null || column == null) {
+        if (!fetchContent && !fetchTimestamp) {
             scan.setFilter(new KeyOnlyFilter());
         } else {
-            scan.addColumn(family, column);
+            if (fetchContent)
+                scan.addColumn(B_FAMILY, B_COLUMN);
+            if (fetchTimestamp)
+                scan.addColumn(B_FAMILY, B_COLUMN_TS);
         }
 
         HTableInterface table = getConnection().getTable(getAllInOneTableName());
@@ -297,7 +297,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
         Path redirectPath = bigCellHDFSPath(resPath);
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
+        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
 
         if (fileSystem.exists(redirectPath)) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/common/src/main/java/org/apache/kylin/common/persistence/RawResource.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/RawResource.java b/common/src/main/java/org/apache/kylin/common/persistence/RawResource.java
index 8125b86..4f52553 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/RawResource.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/RawResource.java
@@ -24,11 +24,11 @@ import java.io.InputStream;
  */
 public class RawResource {
 
-    public final InputStream resource;
+    public final InputStream inputStream;
     public final long timestamp;
 
-    public RawResource(InputStream resource, long timestamp) {
-        this.resource = resource;
+    public RawResource(InputStream inputStream, long timestamp) {
+        this.inputStream = inputStream;
         this.timestamp = timestamp;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
index 55a1a58..a23a4cd 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
@@ -127,22 +127,22 @@ abstract public class ResourceStore {
      */
     final public <T extends RootPersistentEntity> T getResource(String resPath, Class<T> clz, Serializer<T> serializer) throws IOException {
         resPath = norm(resPath);
-        InputStream in = getResourceImpl(resPath);
-        if (in == null)
+        RawResource res = getResourceImpl(resPath);
+        if (res == null)
             return null;
-
-        DataInputStream din = new DataInputStream(in);
+        
+        DataInputStream din = new DataInputStream(res.inputStream);
         try {
             T r = serializer.deserialize(din);
-            r.setLastModified(getResourceTimestamp(resPath));
+            r.setLastModified(res.timestamp);
             return r;
         } finally {
             IOUtils.closeQuietly(din);
-            IOUtils.closeQuietly(in);
+            IOUtils.closeQuietly(res.inputStream);
         }
     }
 
-    final public InputStream getResource(String resPath) throws IOException {
+    final public RawResource getResource(String resPath) throws IOException {
         return getResourceImpl(norm(resPath));
     }
 
@@ -154,27 +154,22 @@ abstract public class ResourceStore {
         List<T> result = Lists.newArrayList();
         try {
             for (RawResource rawResource : allResources) {
-                final T element = serializer.deserialize(new DataInputStream(rawResource.resource));
+                final T element = serializer.deserialize(new DataInputStream(rawResource.inputStream));
                 element.setLastModified(rawResource.timestamp);
                 result.add(element);
             }
             return result;
         } finally {
             for (RawResource rawResource : allResources) {
-                IOUtils.closeQuietly(rawResource.resource);
+                IOUtils.closeQuietly(rawResource.inputStream);
             }
         }
     }
 
     abstract protected List<RawResource> getAllResources(String rangeStart, String rangeEnd) throws IOException;
 
-    abstract protected InputStream getResourceImpl(String resPath) throws IOException;
-
-    final public long getResourceTimestamp(String resPath) throws IOException {
-        return getResourceTimestampImpl(norm(resPath));
-    }
-
-    abstract protected long getResourceTimestampImpl(String resPath) throws IOException;
+    /** returns null if not exists */
+    abstract protected RawResource getResourceImpl(String resPath) throws IOException;
 
     /**
      * overwrite a resource without write conflict check

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java b/common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java
index 6c15f00..0ebed3d 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.common.persistence;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 
 import org.apache.kylin.common.KylinConfig;
@@ -86,13 +85,12 @@ public class ResourceTool {
         // case of resource (not a folder)
         if (children == null) {
             if (matchExclude(path) == false) {
-                InputStream content = src.getResource(path);
-                long ts = src.getResourceTimestamp(path);
-                if (content != null) {
-                    dst.putResource(path, content, ts);
-                    content.close();
+                RawResource res = src.getResource(path);
+                if (res != null) {
+                    dst.putResource(path, res.inputStream, res.timestamp);
+                    res.inputStream.close();
                 } else {
-                    System.out.println("Null inputstream for " + path);
+                    System.out.println("Resource not exist for " + path);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java b/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java
index cd3427e..0a08709 100644
--- a/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java
+++ b/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java
@@ -193,7 +193,7 @@ public class CubeMetadataUpgrade {
         MetadataManager.getInstance(config).reload();
         CubeDescManager.clearCache();
         CubeDescManager.getInstance(config);
-        CubeManager cubeManager = CubeManager.getInstance(config);
+        CubeManager.getInstance(config);
         ProjectManager.getInstance(config);
         //cleanup();
 
@@ -267,7 +267,7 @@ public class CubeMetadataUpgrade {
 
             InputStream is = null;
             try {
-                is = store.getResource(path);
+                is = store.getResource(path).inputStream;
                 if (is == null) {
                     continue;
                 }
@@ -496,7 +496,7 @@ public class CubeMetadataUpgrade {
                         if (pkToFK.containsKey(key) && !newSeg.getDictionaries().containsKey(pkToFK.get(key))) {
                             logger.debug("Duplicate dictionary for FK " + pkToFK.get(key) + " in cube " + newInstance.getName());
                             changedCubes.add(newInstance.getName());
-                            newDictionaries.add(new Pair(pkToFK.get(key), e.getValue()));
+                            newDictionaries.add(new Pair<String, String>(pkToFK.get(key), e.getValue()));
 
                         }
                     }
@@ -617,10 +617,11 @@ public class CubeMetadataUpgrade {
         for (int i = 0, size = job.getSteps().size(); i < size; ++i) {
             final JobInstance.JobStep jobStep = job.getSteps().get(i);
             final String outputPath = ResourceStore.JOB_OUTPUT_PATH_ROOT + "/" + job.getId() + "." + i;
-            final InputStream inputStream = getStore().getResource(outputPath);
+            final InputStream inputStream = getStore().getResource(outputPath).inputStream;
 
             String output = null;
             if (inputStream != null) {
+                @SuppressWarnings("unchecked")
                 HashMap<String, String> job_output = JsonUtil.readValue(inputStream, HashMap.class);
 
                 if (job_output != null) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
index 7b3af95..a851756 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
@@ -27,7 +27,6 @@ import static org.apache.hadoop.util.StringUtils.*;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
@@ -51,6 +50,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.CliCommandExecutor;
 import org.apache.kylin.common.util.StringSplitter;
@@ -347,13 +347,11 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
         KylinConfig localConfig = KylinConfig.createInstanceFromUri(metaDir.getAbsolutePath());
         ResourceStore to = ResourceStore.getStore(localConfig);
         for (String path : dumpList) {
-            InputStream in = from.getResource(path);
-            if (in == null)
+            RawResource res = from.getResource(path);
+            if (res == null)
                 throw new IllegalStateException("No resource found at -- " + path);
-            long ts = from.getResourceTimestamp(path);
-            to.putResource(path, in, ts);
-            //The following log is duplicate with in ResourceStore
-            //log.info("Dumped resource " + path + " to " + metaDir.getAbsolutePath());
+            to.putResource(path, res.inputStream, res.timestamp);
+            res.inputStream.close();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
index b322a4b..cb601c5 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
@@ -28,6 +28,7 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
@@ -114,7 +115,9 @@ public class MetadataCleanupJob extends AbstractHadoopJob {
                     if (snapshotNames != null)
                         for (String snapshot : snapshotNames) {
                             if (!activeResourceList.contains(snapshot)) {
-                                if (isOlderThanThreshold(getStore().getResourceTimestamp(snapshot)))
+                                RawResource res = getStore().getResource(snapshot);
+                                res.inputStream.close();
+                                if (isOlderThanThreshold(res.timestamp))
                                     toDeleteResource.add(snapshot);
                             }
                         }
@@ -134,7 +137,9 @@ public class MetadataCleanupJob extends AbstractHadoopJob {
                         if (dictionaries != null)
                             for (String dict : dictionaries)
                                 if (!activeResourceList.contains(dict)) {
-                                    if (isOlderThanThreshold(getStore().getResourceTimestamp(dict)))
+                                    RawResource res = getStore().getResource(dict);
+                                    res.inputStream.close();
+                                    if (isOlderThanThreshold(res.timestamp))
                                         toDeleteResource.add(dict);
                                 }
                     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java b/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
index b07d6a9..2d4b0bf 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.job.tools;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
+import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
 import org.apache.kylin.cube.CubeInstance;
@@ -293,10 +293,9 @@ public class CubeMigrationCLI {
         }
         case COPY_FILE_IN_META: {
             String item = (String) opt.params[0];
-            InputStream inputStream = srcStore.getResource(item);
-            long ts = srcStore.getResourceTimestamp(item);
-            dstStore.putResource(item, inputStream, ts);
-            inputStream.close();
+            RawResource res = srcStore.getResource(item);
+            dstStore.putResource(item, res.inputStream, res.timestamp);
+            res.inputStream.close();
             logger.info("Item " + item + " is copied");
             break;
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/job/src/test/java/org/apache/kylin/job/DeployUtil.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/DeployUtil.java b/job/src/test/java/org/apache/kylin/job/DeployUtil.java
index 8684aa0..550bddb 100644
--- a/job/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/job/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -183,7 +183,7 @@ public class DeployUtil {
         // duplicate a copy of this fact table, with a naming convention with fact.csv.inner or fact.csv.left
         // so that later test cases can select different data files
         ResourceStore store = ResourceStore.getStore(config());
-        InputStream in = store.getResource("/data/" + factTableName + ".csv");
+        InputStream in = store.getResource("/data/" + factTableName + ".csv").inputStream;
         String factTablePathWithJoinType = "/data/" + factTableName + ".csv." + joinType.toLowerCase();
         store.deleteResource(factTablePathWithJoinType);
         store.putResource(factTablePathWithJoinType, in, System.currentTimeMillis());
@@ -203,7 +203,7 @@ public class DeployUtil {
             File localBufferFile = new File(temp.getParent() + "/" + tablename + ".csv");
             localBufferFile.createNewFile();
 
-            InputStream hbaseDataStream = metaMgr.getStore().getResource("/data/" + tablename + ".csv");
+            InputStream hbaseDataStream = metaMgr.getStore().getResource("/data/" + tablename + ".csv").inputStream;
             FileOutputStream localFileStream = new FileOutputStream(localBufferFile);
             IOUtils.copy(hbaseDataStream, localFileStream);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/job/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java b/job/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
index 2bc4dc3..c9988fc 100644
--- a/job/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
+++ b/job/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
@@ -107,8 +107,7 @@ public class FactTableGenerator {
      */
     private void loadConfig() {
         try {
-            InputStream configStream = null;
-            configStream = store.getResource("/data/data_gen_config.json");
+            InputStream configStream = store.getResource("/data/data_gen_config.json").inputStream;
             this.genConf = GenConfig.loadConfig(configStream);
 
             if (configStream != null)
@@ -136,7 +135,7 @@ public class FactTableGenerator {
             }
 
             String path = "/data/" + lookupTableName + ".csv";
-            tableStream = store.getResource(path);
+            tableStream = store.getResource(path).inputStream;
             tableReader = new BufferedReader(new InputStreamReader(tableStream));
             tableReader.mark(0);
             int rowCount = 0;
@@ -158,7 +157,7 @@ public class FactTableGenerator {
             tableStream = null;
             tableReader = null;
 
-            tableStream = store.getResource(path);
+            tableStream = store.getResource(path).inputStream;
             tableReader = new BufferedReader(new InputStreamReader(tableStream));
 
             while ((curRow = tableReader.readLine()) != null) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java b/metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
index 377fba7..b540588 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
@@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
+import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
 import org.apache.kylin.common.restclient.Broadcaster;
@@ -215,11 +216,13 @@ public class MetadataManager {
         Map<String, String> attrs = Maps.newHashMap();
 
         ResourceStore store = getStore();
-        InputStream is = store.getResource(path);
-        if (is == null) {
+        RawResource res = store.getResource(path);
+        if (res == null) {
             logger.warn("Failed to get table exd info from " + path);
             return null;
         }
+        
+        InputStream is = res.inputStream;
 
         try {
             attrs.putAll(JsonUtil.readValue(is, HashMap.class));

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/query/src/test/java/org/apache/kylin/query/test/H2Database.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/org/apache/kylin/query/test/H2Database.java b/query/src/test/java/org/apache/kylin/query/test/H2Database.java
index eea1a96..564363f 100644
--- a/query/src/test/java/org/apache/kylin/query/test/H2Database.java
+++ b/query/src/test/java/org/apache/kylin/query/test/H2Database.java
@@ -21,7 +21,6 @@ package org.apache.kylin.query.test;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -29,6 +28,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
@@ -76,18 +76,17 @@ public class H2Database {
             String normalPath = "/data/" + tableDesc.getIdentity() + ".csv";
 
             // If it's the fact table, there will be a facttable.csv.inner or
-            // facttable.csv.left in hbase
-            // otherwise just use lookup.csv
-            InputStream csvStream = metaMgr.getStore().getResource(normalPath + fileNameSuffix);
-            if (csvStream == null) {
-                csvStream = metaMgr.getStore().getResource(normalPath);
+            // facttable.csv.left in hbase, otherwise just use lookup.csv
+            RawResource res = metaMgr.getStore().getResource(normalPath + fileNameSuffix);
+            if (res == null) {
+                res = metaMgr.getStore().getResource(normalPath);
             } else {
                 logger.info("H2 decides to load " + (normalPath + fileNameSuffix) + " for table " + tableDesc.getIdentity());
             }
 
-            org.apache.commons.io.IOUtils.copy(csvStream, tempFileStream);
+            org.apache.commons.io.IOUtils.copy(res.inputStream, tempFileStream);
 
-            csvStream.close();
+            res.inputStream.close();
             tempFileStream.close();
 
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2f788e71/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java b/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
index 84f1042..148607a 100644
--- a/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
+++ b/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
@@ -43,7 +43,7 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
-@Ignore("KylinQueryTest is contained by CombinationTest")
+//@Ignore("KylinQueryTest is contained by CombinationTest")
 public class KylinQueryTest extends KylinTestBase {
 
     @BeforeClass
@@ -140,7 +140,7 @@ public class KylinQueryTest extends KylinTestBase {
     @Test
     public void testSingleRunQuery() throws Exception {
 
-        String queryFileName = "src/test/resources/query/sql/query62.sql";
+        String queryFileName = "src/test/resources/query/sql/sample.txt";
 
         File sqlFile = new File(queryFileName);
         runSQL(sqlFile, true, true);


[20/30] incubator-kylin git commit: KYLIN-702 When Kylin create the flat hive table, it generates large number of small files in HDFS

Posted by li...@apache.org.
KYLIN-702 When Kylin create the flat hive table, it generates large number of small files in HDFS


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 2432e2d6f561531f4b38ea8be6b3ca34db48f3ad
Parents: bf56038
Author: shaofengshi <sh...@apache.org>
Authored: Fri Nov 13 11:24:52 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Nov 13 11:26:50 2015 +0800

----------------------------------------------------------------------
 conf/kylin_job_conf.xml | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2432e2d6/conf/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/conf/kylin_job_conf.xml b/conf/kylin_job_conf.xml
index ad3427c..2276a7c 100644
--- a/conf/kylin_job_conf.xml
+++ b/conf/kylin_job_conf.xml
@@ -68,6 +68,23 @@ limitations under the License. See accompanying LICENSE file.
         <description>Block replication</description>
     </property>
 
+
+    <property>
+        <name>hive.merge.mapfiles</name>
+        <value>true</value>
+        <description>Enable hive file merge on mapper only job</description>
+    </property>
+    <property>
+        <name>hive.merge.mapredfiles</name>
+        <value>true</value>
+        <description>Enable hive file merge on map-reduce job</description>
+    </property>
+    <property>
+        <name>hive.merge.size.per.task</name>
+        <value>268435456</value>
+        <description>Size for the merged file: 256M</description>
+    </property>
+
     <property>
         <name>hive.support.concurrency</name>
         <value>false</value>


[05/30] incubator-kylin git commit: update pom.xml with rat exclude

Posted by li...@apache.org.
update pom.xml with rat exclude

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 824f96a15d5407ec77cedf502d97d682bd83933b
Parents: 782dd52
Author: shaofengshi <sh...@apache.org>
Authored: Wed Nov 4 15:58:26 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 11:16:22 2015 +0800

----------------------------------------------------------------------
 pom.xml | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/824f96a1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 90ba294..72bf7e7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -790,6 +790,12 @@
 
                                 <!-- MIT license -->
                                 <exclude>webapp/app/css/AdminLTE.css</exclude>
+                                
+                                <!--configuration file -->
+                                <exclude>webapp/app/routes.json</exclude>
+                                <exclude>webapp/bower.json</exclude>
+                                <exclude>webapp/grunt.json</exclude>
+                                <exclude>webapp/package.json</exclude>
 
                                 <!-- jdbc log -->
                                 <exclude>jdbc/kylin_jdbc.log*</exclude>
@@ -842,7 +848,6 @@
                                     <appendAssemblyId>true</appendAssemblyId>
                                     <descriptor>src/main/config/assemblies/source-assembly.xml</descriptor>
                                     <finalName>apache-kylin-${project.version}</finalName>
-                                    <tarLongFileMode>gnu</tarLongFileMode>
                                 </configuration>
                             </execution>
                         </executions>


[09/30] incubator-kylin git commit: KYLIN-999 Add apache license header to hadoop conf files

Posted by li...@apache.org.
KYLIN-999 Add apache license header to hadoop conf files

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: b37e026bdcbfbdda04c1fe10153584d8de847fed
Parents: d730c9a
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Nov 4 14:57:21 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 11:16:22 2015 +0800

----------------------------------------------------------------------
 .../test_case_data/sandbox/capacity-scheduler.xml  | 17 ++++++++++++++++-
 examples/test_case_data/sandbox/core-site.xml      | 17 ++++++++++++++++-
 examples/test_case_data/sandbox/hadoop-policy.xml  | 17 ++++++++++++++++-
 examples/test_case_data/sandbox/hbase-policy.xml   | 17 ++++++++++++++++-
 examples/test_case_data/sandbox/hbase-site.xml     | 17 ++++++++++++++++-
 examples/test_case_data/sandbox/hdfs-site.xml      | 17 ++++++++++++++++-
 examples/test_case_data/sandbox/hive-site.xml      | 17 ++++++++++++++++-
 examples/test_case_data/sandbox/mapred-site.xml    | 17 ++++++++++++++++-
 examples/test_case_data/sandbox/yarn-site.xml      | 17 ++++++++++++++++-
 9 files changed, 144 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b37e026b/examples/test_case_data/sandbox/capacity-scheduler.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/capacity-scheduler.xml b/examples/test_case_data/sandbox/capacity-scheduler.xml
index dcd63c1..7cb985c 100644
--- a/examples/test_case_data/sandbox/capacity-scheduler.xml
+++ b/examples/test_case_data/sandbox/capacity-scheduler.xml
@@ -1,4 +1,19 @@
-<!--Wed May 27 08:09:06 2015-->
+<!--
+  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.
+-->
 <configuration>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b37e026b/examples/test_case_data/sandbox/core-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/core-site.xml b/examples/test_case_data/sandbox/core-site.xml
index 0a80866..9aa588c 100644
--- a/examples/test_case_data/sandbox/core-site.xml
+++ b/examples/test_case_data/sandbox/core-site.xml
@@ -1,4 +1,19 @@
-<!--Wed May 27 08:08:58 2015-->
+<!--
+  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.
+-->
 <configuration>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b37e026b/examples/test_case_data/sandbox/hadoop-policy.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hadoop-policy.xml b/examples/test_case_data/sandbox/hadoop-policy.xml
index 4556120..a590979 100644
--- a/examples/test_case_data/sandbox/hadoop-policy.xml
+++ b/examples/test_case_data/sandbox/hadoop-policy.xml
@@ -1,4 +1,19 @@
-<!--Fri Apr 24 12:58:40 2015-->
+<!--
+  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.
+-->
 <configuration>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b37e026b/examples/test_case_data/sandbox/hbase-policy.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hbase-policy.xml b/examples/test_case_data/sandbox/hbase-policy.xml
index ae974bb..178b559 100644
--- a/examples/test_case_data/sandbox/hbase-policy.xml
+++ b/examples/test_case_data/sandbox/hbase-policy.xml
@@ -1,4 +1,19 @@
-<!--Wed May 27 06:14:03 2015-->
+<!--
+  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.
+-->
 <configuration>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b37e026b/examples/test_case_data/sandbox/hbase-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hbase-site.xml b/examples/test_case_data/sandbox/hbase-site.xml
index 06755a8..734908e 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -1,4 +1,19 @@
-<!--Wed May 27 06:14:00 2015-->
+<!--
+  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.
+-->
 <configuration>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b37e026b/examples/test_case_data/sandbox/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hdfs-site.xml b/examples/test_case_data/sandbox/hdfs-site.xml
index db4da75..1175fff 100644
--- a/examples/test_case_data/sandbox/hdfs-site.xml
+++ b/examples/test_case_data/sandbox/hdfs-site.xml
@@ -1,4 +1,19 @@
-<!--Wed May 27 06:14:02 2015-->
+<!--
+  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.
+-->
 <configuration>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b37e026b/examples/test_case_data/sandbox/hive-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hive-site.xml b/examples/test_case_data/sandbox/hive-site.xml
index 21fc81a..f4c7738 100644
--- a/examples/test_case_data/sandbox/hive-site.xml
+++ b/examples/test_case_data/sandbox/hive-site.xml
@@ -1,4 +1,19 @@
-<!--Fri Apr 24 13:44:42 2015-->
+<!--
+  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.
+-->
 <configuration>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b37e026b/examples/test_case_data/sandbox/mapred-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/mapred-site.xml b/examples/test_case_data/sandbox/mapred-site.xml
index f2b49c5..a8ff87f 100644
--- a/examples/test_case_data/sandbox/mapred-site.xml
+++ b/examples/test_case_data/sandbox/mapred-site.xml
@@ -1,4 +1,19 @@
-<!--Wed May 27 08:09:05 2015-->
+<!--
+  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.
+-->
 <configuration>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b37e026b/examples/test_case_data/sandbox/yarn-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/yarn-site.xml b/examples/test_case_data/sandbox/yarn-site.xml
index a8cda86..8256158 100644
--- a/examples/test_case_data/sandbox/yarn-site.xml
+++ b/examples/test_case_data/sandbox/yarn-site.xml
@@ -1,4 +1,19 @@
-<!--Wed May 27 08:09:01 2015-->
+<!--
+  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.
+-->
 <configuration>
 
     <property>


[19/30] incubator-kylin git commit: fix server/pom.xml to exclude log4j-over-slf4j

Posted by li...@apache.org.
fix server/pom.xml to exclude log4j-over-slf4j


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: bf560389048086ccf23ccf822aff4caad3180fee
Parents: 819ebd6
Author: Li, Yang <ya...@ebay.com>
Authored: Fri Nov 13 10:19:43 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Fri Nov 13 10:19:43 2015 +0800

----------------------------------------------------------------------
 server/pom.xml | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf560389/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index 8a17435..d22f47e 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -108,6 +108,10 @@
                     <artifactId>spring-boot-starter-tomcat</artifactId>
                 </exclusion>
                 <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>log4j-over-slf4j</artifactId>
+                </exclusion>
+                <exclusion>
                     <groupId>ch.qos.logback</groupId>
                     <artifactId>logback-classic</artifactId>
                 </exclusion>


[28/30] incubator-kylin git commit: KYLIN-920 & KYLIN-782 Upgrade to HBase 1.1 (with help from murkrishn )

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java b/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java
index 9f9c23c..f5f94c8 100644
--- a/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java
+++ b/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java
@@ -23,10 +23,11 @@ import java.io.IOException;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.BytesUtil;
@@ -60,11 +61,11 @@ public class HBaseRowDigestTest extends HBaseMetadataTestCase {
     @Test
     public static void test() throws IOException {
         String hbaseUrl = "hbase"; // use hbase-site.xml on classpath
-        HConnection conn = null;
-        HTableInterface table = null;
+        Connection conn = null;
+        Table table = null;
         try {
             conn = HBaseConnection.get(hbaseUrl);
-            table = conn.getTable("KYLIN_II_YTYWP3CQGJ");
+            table = conn.getTable(TableName.valueOf("KYLIN_II_YTYWP3CQGJ"));
             ResultScanner scanner = table.getScanner(CF, QN);
             StringBuffer sb = new StringBuffer();
             while (true) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/monitor/pom.xml
----------------------------------------------------------------------
diff --git a/monitor/pom.xml b/monitor/pom.xml
index dfdd12d..a131bd6 100644
--- a/monitor/pom.xml
+++ b/monitor/pom.xml
@@ -39,6 +39,12 @@
 
     <dependencies>
         <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-common</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java b/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
index 97200fc..94b3937 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
@@ -20,18 +20,21 @@ package org.apache.kylin.monitor;
 
 import java.io.IOException;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.log4j.Logger;
 
 /**
@@ -122,11 +125,10 @@ public class MonitorMetaManager {
     public static String getListWithRowkey(String table, String rowkey) throws IOException {
         Result result = getResultByRowKey(table, rowkey);
         String fileList = null;
-        if (result.list() != null) {
-            for (KeyValue kv : result.list()) {
-                fileList = Bytes.toString(kv.getValue());
+        if (result.listCells() != null) {
+            for (Cell cell : result.listCells()) {
+                fileList = Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueOffset());
             }
-
         }
         fileList = fileList == null ? "" : fileList;
         return fileList;
@@ -164,16 +166,20 @@ public class MonitorMetaManager {
      * create table in hbase
      */
     public static void creatTable(String tableName, String[] family) throws Exception {
-        HBaseAdmin admin = new HBaseAdmin(conf);
-        HTableDescriptor desc = new HTableDescriptor(tableName);
-        for (int i = 0; i < family.length; i++) {
-            desc.addFamily(new HColumnDescriptor(family[i]));
-        }
-        if (admin.tableExists(tableName)) {
-            logger.info("table Exists!");
-        } else {
-            admin.createTable(desc);
-            logger.info("create table Success!");
+        Admin admin = HBaseConnection.get().getAdmin();
+        try {
+            HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+            for (int i = 0; i < family.length; i++) {
+                desc.addFamily(new HColumnDescriptor(family[i]));
+            }
+            if (admin.tableExists(TableName.valueOf(tableName))) {
+                logger.info("table Exists!");
+            } else {
+                admin.createTable(desc);
+                logger.info("create table Success!");
+            }
+        } finally {
+            IOUtils.closeQuietly(admin);
         }
     }
 
@@ -181,13 +187,15 @@ public class MonitorMetaManager {
      * update cell in hbase
      */
     public static void updateData(String tableName, String rowKey, String family, String column, String value) throws IOException {
-        HTable table = new HTable(conf, Bytes.toBytes(tableName));
+        Table table = HBaseConnection.get().getTable(TableName.valueOf(tableName));
         Put put = new Put(rowKey.getBytes());
-        put.add(family.getBytes(), column.getBytes(), value.getBytes());
+        put.addColumn(family.getBytes(), column.getBytes(), value.getBytes());
         try {
             table.put(put);
         } catch (IOException e) {
             e.printStackTrace();
+        } finally {
+            IOUtils.closeQuietly(table);
         }
         logger.info("update table [" + tableName + "]");
         logger.info("rowKey [" + rowKey + "]");
@@ -200,9 +208,10 @@ public class MonitorMetaManager {
      * get result by rowkey
      */
     public static Result getResultByRowKey(String tableName, String rowKey) throws IOException {
-        HTable table = new HTable(conf, Bytes.toBytes(tableName));
+        Table table = HBaseConnection.get().getTable(TableName.valueOf(tableName));
         Get get = new Get(Bytes.toBytes(rowKey));
         Result result = table.get(get);
+        IOUtils.closeQuietly(table);
         return result;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 72bf7e7..1bc193d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -45,12 +45,13 @@
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
         <!-- Hadoop versions -->
-        <hadoop2.version>2.6.0</hadoop2.version>
-        <yarn.version>2.6.0</yarn.version>
+        <hadoop2.version>2.7.1</hadoop2.version>
+        <yarn.version>2.7.1</yarn.version>
         <zookeeper.version>3.4.6</zookeeper.version>
-        <hive.version>0.14.0</hive.version>
-        <hive-hcatalog.version>0.14.0</hive-hcatalog.version>
-        <hbase-hadoop2.version>0.98.8-hadoop2</hbase-hadoop2.version>
+        <hive.version>1.2.1</hive.version>
+        <hive-hcatalog.version>1.2.1</hive-hcatalog.version>
+        <hbase-hadoop2.version>1.1.1</hbase-hadoop2.version>
+        <curator.version>2.7.1</curator.version>
 
         <!-- Dependency versions -->
         <antlr.version>3.4</antlr.version>
@@ -89,9 +90,6 @@
         <!-- Calcite Version -->
         <calcite.version>1.4.0-incubating</calcite.version>
 
-        <!-- Curator.version Version -->
-        <curator.version>2.6.0</curator.version>
-
         <!-- Sonar -->
         <sonar.java.coveragePlugin>jacoco</sonar.java.coveragePlugin>
         <sonar.dynamicAnalysis>reuseReports</sonar.dynamicAnalysis>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/server/src/main/java/org/apache/kylin/rest/service/AclService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/AclService.java b/server/src/main/java/org/apache/kylin/rest/service/AclService.java
index ea2a48e..8a1cf6d 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/AclService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/AclService.java
@@ -29,13 +29,14 @@ import java.util.Map;
 import java.util.NavigableMap;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.kylin.common.KylinConfig;
@@ -130,9 +131,9 @@ public class AclService implements MutableAclService {
     @Override
     public List<ObjectIdentity> findChildren(ObjectIdentity parentIdentity) {
         List<ObjectIdentity> oids = new ArrayList<ObjectIdentity>();
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
 
             Scan scan = new Scan();
             SingleColumnValueFilter parentFilter = new SingleColumnValueFilter(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), CompareOp.EQUAL, domainObjSerializer.serialize(new DomainObjectInfo(parentIdentity)));
@@ -179,10 +180,10 @@ public class AclService implements MutableAclService {
     @Override
     public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> oids, List<Sid> sids) throws NotFoundException {
         Map<ObjectIdentity, Acl> aclMaps = new HashMap<ObjectIdentity, Acl>();
-        HTableInterface htable = null;
+        Table htable = null;
         Result result = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
 
             for (ObjectIdentity oid : oids) {
                 result = htable.get(new Get(Bytes.toBytes(String.valueOf(oid.getIdentifier()))));
@@ -231,16 +232,15 @@ public class AclService implements MutableAclService {
         Authentication auth = SecurityContextHolder.getContext().getAuthentication();
         PrincipalSid sid = new PrincipalSid(auth);
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
             Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
-            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
-            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
-            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
+            put.addColumn(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
+            put.addColumn(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
+            put.addColumn(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
 
             htable.put(put);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " created successfully.");
         } catch (IOException e) {
@@ -254,9 +254,9 @@ public class AclService implements MutableAclService {
 
     @Override
     public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
             Delete delete = new Delete(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
 
             List<ObjectIdentity> children = findChildren(objectIdentity);
@@ -269,7 +269,6 @@ public class AclService implements MutableAclService {
             }
 
             htable.delete(delete);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " deleted successfully.");
         } catch (IOException e) {
@@ -287,27 +286,26 @@ public class AclService implements MutableAclService {
             throw e;
         }
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
             Delete delete = new Delete(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
-            delete.deleteFamily(Bytes.toBytes(ACL_ACES_FAMILY));
+            delete.addFamily(Bytes.toBytes(ACL_ACES_FAMILY));
             htable.delete(delete);
 
             Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
 
             if (null != acl.getParentAcl()) {
-                put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
+                put.addColumn(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
             }
 
             for (AccessControlEntry ace : acl.getEntries()) {
                 AceInfo aceInfo = new AceInfo(ace);
-                put.add(Bytes.toBytes(ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
+                put.addColumn(Bytes.toBytes(ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
             }
 
             if (!put.isEmpty()) {
                 htable.put(put);
-                htable.flushCommits();
 
                 logger.debug("ACL of " + acl.getObjectIdentity() + " updated successfully.");
             }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
index f115d89..2770475 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -29,9 +29,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.HBaseRegionSizeCalculator;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.cube.CubeInstance;
@@ -430,33 +430,24 @@ public class CubeService extends BasicService {
      * @throws IOException Exception when HTable resource is not closed correctly.
      */
     public HBaseResponse getHTableInfo(String tableName) throws IOException {
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        HTable table = null;
+        Connection conn = HBaseConnection.get();
         HBaseResponse hr = null;
         long tableSize = 0;
         int regionCount = 0;
 
-        try {
-            table = new HTable(hconf, tableName);
-
-            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
-            Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
+        HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
+        Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
 
-            for (long s : sizeMap.values()) {
-                tableSize += s;
-            }
+        for (long s : sizeMap.values()) {
+            tableSize += s;
+        }
 
-            regionCount = sizeMap.size();
+        regionCount = sizeMap.size();
 
-            // Set response.
-            hr = new HBaseResponse();
-            hr.setTableSize(tableSize);
-            hr.setRegionCount(regionCount);
-        } finally {
-            if (null != table) {
-                table.close();
-            }
-        }
+        // Set response.
+        hr = new HBaseResponse();
+        hr.setTableSize(tableSize);
+        hr.setRegionCount(regionCount);
 
         return hr;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 764df4b..7d14021 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -42,10 +42,11 @@ import javax.sql.DataSource;
 
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
 import org.apache.kylin.common.persistence.HBaseConnection;
@@ -124,14 +125,13 @@ public class QueryService extends BasicService {
         Query[] queryArray = new Query[queries.size()];
 
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -157,14 +157,13 @@ public class QueryService extends BasicService {
 
         Query[] queryArray = new Query[queries.size()];
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -176,9 +175,9 @@ public class QueryService extends BasicService {
         }
 
         List<Query> queries = new ArrayList<Query>();
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Get get = new Get(Bytes.toBytes(creator));
             get.addFamily(Bytes.toBytes(USER_QUERY_FAMILY));
             Result result = htable.get(get);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/server/src/main/java/org/apache/kylin/rest/service/UserService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/UserService.java b/server/src/main/java/org/apache/kylin/rest/service/UserService.java
index d665ab9..d03cd55 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/UserService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/UserService.java
@@ -25,13 +25,14 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.Bytes;
@@ -75,9 +76,9 @@ public class UserService implements UserManager {
 
     @Override
     public UserDetails loadUserByUsername(String username) throws UsernameNotFoundException {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
 
             Get get = new Get(Bytes.toBytes(username));
             get.addFamily(Bytes.toBytes(USER_AUTHORITY_FAMILY));
@@ -106,15 +107,14 @@ public class UserService implements UserManager {
 
     @Override
     public void updateUser(UserDetails user) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             byte[] userAuthorities = serialize(user.getAuthorities());
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(user.getUsername()));
-            put.add(Bytes.toBytes(USER_AUTHORITY_FAMILY), Bytes.toBytes(USER_AUTHORITY_COLUMN), userAuthorities);
+            put.addColumn(Bytes.toBytes(USER_AUTHORITY_FAMILY), Bytes.toBytes(USER_AUTHORITY_COLUMN), userAuthorities);
 
             htable.put(put);
-            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -124,13 +124,12 @@ public class UserService implements UserManager {
 
     @Override
     public void deleteUser(String username) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Delete delete = new Delete(Bytes.toBytes(username));
 
             htable.delete(delete);
-            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -145,9 +144,9 @@ public class UserService implements UserManager {
 
     @Override
     public boolean userExists(String username) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Result result = htable.get(new Get(Bytes.toBytes(username)));
 
             return null != result && !result.isEmpty();
@@ -164,10 +163,10 @@ public class UserService implements UserManager {
         s.addColumn(Bytes.toBytes(USER_AUTHORITY_FAMILY), Bytes.toBytes(USER_AUTHORITY_COLUMN));
 
         List<String> authorities = new ArrayList<String>();
-        HTableInterface htable = null;
+        Table htable = null;
         ResultScanner scanner = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             scanner = htable.getScanner(s);
 
             for (Result result = scanner.next(); result != null; result = scanner.next()) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
index 9efbb79..6f08f8a 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
@@ -28,16 +28,16 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FuzzyRowFilter;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.kylin.common.persistence.StorageException;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.Bytes;
@@ -82,7 +82,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     private final Collection<RowValueDecoder> rowValueDecoders;
     private final StorageContext context;
     private final String tableName;
-    private final HTableInterface table;
+    private final Table table;
     private final RowKeyDecoder rowKeyDecoder;
     private final Iterator<HBaseKeyRange> rangeIterator;
 
@@ -94,7 +94,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     private int scanCount;
     private int scanCountDelta;
 
-    public CubeSegmentTupleIterator(CubeSegment cubeSeg, Collection<HBaseKeyRange> keyRanges, HConnection conn, Collection<TblColRef> dimensions, TupleFilter filter, Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context) {
+    public CubeSegmentTupleIterator(CubeSegment cubeSeg, Collection<HBaseKeyRange> keyRanges, Connection conn, Collection<TblColRef> dimensions, TupleFilter filter, Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context) {
         this.cube = cubeSeg.getCubeInstance();
         this.cubeSeg = cubeSeg;
         this.dimensions = dimensions;
@@ -106,7 +106,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
         this.rowKeyDecoder = new RowKeyDecoder(this.cubeSeg);
 
         try {
-            this.table = conn.getTable(tableName);
+            this.table = conn.getTable(TableName.valueOf(tableName));
         } catch (Throwable t) {
             throw new StorageException("Error when open connection to table " + tableName, t);
         }
@@ -122,12 +122,11 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
 
     private void closeScanner() {
         flushScanCountDelta();
-        
+
         if (logger.isDebugEnabled() && scan != null) {
             logger.debug("Scan " + scan.toString());
-            byte[] metricsBytes = scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA);
-            if (metricsBytes != null) {
-                ScanMetrics scanMetrics = ProtobufUtil.toScanMetrics(metricsBytes);
+            ScanMetrics scanMetrics = scan.getScanMetrics();
+            if (scanMetrics != null) {
                 logger.debug("HBase Metrics: " + "count={}, ms={}, bytes={}, remote_bytes={}, regions={}, not_serving_region={}, rpc={}, rpc_retries={}, remote_rpc={}, remote_rpc_retries={}", new Object[] { scanCount, scanMetrics.sumOfMillisSecBetweenNexts, scanMetrics.countOfBytesInResults, scanMetrics.countOfBytesInRemoteResults, scanMetrics.countOfRegions, scanMetrics.countOfNSRE, scanMetrics.countOfRPCcalls, scanMetrics.countOfRPCRetries, scanMetrics.countOfRemoteRPCcalls, scanMetrics.countOfRemoteRPCRetries });
             }
         }
@@ -254,7 +253,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
         Scan scan = new Scan();
         scan.setCaching(SCAN_CACHE);
         scan.setCacheBlocks(true);
-        scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
+        scan.setScanMetricsEnabled(true);
         for (RowValueDecoder valueDecoder : this.rowValueDecoders) {
             HBaseColumnDesc hbaseColumn = valueDecoder.getHBaseColumn();
             byte[] byteFamily = Bytes.toBytes(hbaseColumn.getColumnFamilyName());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
index ed12781..f2006b9 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
@@ -32,7 +32,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Pair;
@@ -140,7 +140,7 @@ public class CubeStorageEngine implements IStorageEngine {
         setCoprocessor(groupsCopD, valueDecoders, context); // enable coprocessor if beneficial
         setLimit(filter, context);
 
-        HConnection conn = HBaseConnection.get(context.getConnUrl());
+        Connection conn = HBaseConnection.get(context.getConnUrl());
         return new SerializedHBaseTupleIterator(conn, scans, cubeInstance, dimensionsD, filterD, groupsCopD, valueDecoders, context);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java
index 918fd4b..6a76baa 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java
@@ -1,93 +1,94 @@
-/*
- * 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.storage.hbase;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.invertedindex.model.IIDesc;
-
-/**
- * @author yangli9
- * 
- */
-public class HBaseClientKVIterator implements Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>>, Closeable {
-
-    byte[] family;
-    byte[] qualifier;
-
-    HTableInterface table;
-    ResultScanner scanner;
-    Iterator<Result> iterator;
-
-    public HBaseClientKVIterator(HConnection hconn, String tableName, byte[] family, byte[] qualifier) throws IOException {
-        this.family = family;
-        this.qualifier = qualifier;
-
-        this.table = hconn.getTable(tableName);
-        this.scanner = table.getScanner(family, qualifier);
-        this.iterator = scanner.iterator();
-    }
-
-    @Override
-    public void close() {
-        IOUtils.closeQuietly(scanner);
-        IOUtils.closeQuietly(table);
-    }
-
-    @Override
-    public Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> iterator() {
-        return new MyIterator();
-    }
-
-    private class MyIterator implements Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> {
-
-        ImmutableBytesWritable key = new ImmutableBytesWritable();
-        ImmutableBytesWritable value = new ImmutableBytesWritable();
-        Pair<ImmutableBytesWritable, ImmutableBytesWritable> pair = new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(key, value);
-
-        @Override
-        public boolean hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public Pair<ImmutableBytesWritable, ImmutableBytesWritable> next() {
-            Result r = iterator.next();
-            Cell c = r.getColumnLatestCell(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
-            key.set(c.getRowArray(), c.getRowOffset(), c.getRowLength());
-            value.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-            return pair;
-        }
-
-        public void remove() {
-            throw new UnsupportedOperationException();
-        }
-
-    }
-}
+/*
+ * 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.storage.hbase;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.invertedindex.model.IIDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class HBaseClientKVIterator implements Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>>, Closeable {
+
+    byte[] family;
+    byte[] qualifier;
+
+    Table table;
+    ResultScanner scanner;
+    Iterator<Result> iterator;
+
+    public HBaseClientKVIterator(Connection hconn, String tableName, byte[] family, byte[] qualifier) throws IOException {
+        this.family = family;
+        this.qualifier = qualifier;
+
+        this.table = hconn.getTable(TableName.valueOf(tableName));
+        this.scanner = table.getScanner(family, qualifier);
+        this.iterator = scanner.iterator();
+    }
+
+    @Override
+    public void close() {
+        IOUtils.closeQuietly(scanner);
+        IOUtils.closeQuietly(table);
+    }
+
+    @Override
+    public Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> iterator() {
+        return new MyIterator();
+    }
+
+    private class MyIterator implements Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> {
+
+        ImmutableBytesWritable key = new ImmutableBytesWritable();
+        ImmutableBytesWritable value = new ImmutableBytesWritable();
+        Pair<ImmutableBytesWritable, ImmutableBytesWritable> pair = new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(key, value);
+
+        @Override
+        public boolean hasNext() {
+            return iterator.hasNext();
+        }
+
+        @Override
+        public Pair<ImmutableBytesWritable, ImmutableBytesWritable> next() {
+            Result r = iterator.next();
+            Cell c = r.getColumnLatestCell(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
+            key.set(c.getRowArray(), c.getRowOffset(), c.getRowLength());
+            value.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
+            return pair;
+        }
+
+        public void remove() {
+            throw new UnsupportedOperationException();
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java
index afb49c0..e518a4c 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java
@@ -1,57 +1,57 @@
-/*
- * 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.storage.hbase;
-
-import java.util.ArrayList;
-
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.kylin.common.persistence.HBaseConnection;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.metadata.realization.SQLDigest;
-import org.apache.kylin.metadata.tuple.ITupleIterator;
-import org.apache.kylin.storage.IStorageEngine;
-import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.storage.hbase.coprocessor.endpoint.EndpointTupleIterator;
-
-/**
- * @author yangli9
- */
-public class InvertedIndexStorageEngine implements IStorageEngine {
-
-    private IISegment seg;
-
-    public InvertedIndexStorageEngine(IIInstance ii) {
-        this.seg = ii.getFirstSegment();
-    }
-
-    @Override
-    public ITupleIterator search(StorageContext context, SQLDigest sqlDigest) {
-        String tableName = seg.getStorageLocationIdentifier();
-
-        //HConnection is cached, so need not be closed
-        HConnection conn = HBaseConnection.get(context.getConnUrl());
-        try {
-            return new EndpointTupleIterator(seg, sqlDigest.filter, sqlDigest.groupbyColumns, new ArrayList<>(sqlDigest.aggregations), context, conn);
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw new IllegalStateException("Error when connecting to II htable " + tableName, e);
-        }
-    }
-}
+/*
+ * 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.storage.hbase;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.persistence.HBaseConnection;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.storage.IStorageEngine;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.hbase.coprocessor.endpoint.EndpointTupleIterator;
+
+/**
+ * @author yangli9
+ */
+public class InvertedIndexStorageEngine implements IStorageEngine {
+
+    private IISegment seg;
+
+    public InvertedIndexStorageEngine(IIInstance ii) {
+        this.seg = ii.getFirstSegment();
+    }
+
+    @Override
+    public ITupleIterator search(StorageContext context, SQLDigest sqlDigest) {
+        String tableName = seg.getStorageLocationIdentifier();
+
+        // Connection is cached, so need not be closed
+        Connection conn = HBaseConnection.get(context.getConnUrl());
+        try {
+            return new EndpointTupleIterator(seg, sqlDigest.filter, sqlDigest.groupbyColumns, new ArrayList<>(sqlDigest.aggregations), context, conn);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw new IllegalStateException("Error when connecting to II htable " + tableName, e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java b/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java
index d4e8529..4a9c574 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java
@@ -1,88 +1,91 @@
-/*
- * 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.storage.hbase;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.token.TokenUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.HadoopUtil;
-
-/**
- * @author yangli9
- * 
- */
-public class PingHBaseCLI {
-
-    public static void main(String[] args) throws IOException {
-        String hbaseTable = args[0];
-
-        System.out.println("Hello friend.");
-
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        if (User.isHBaseSecurityEnabled(hconf)) {
-            try {
-                System.out.println("--------------Getting kerberos credential for user " + UserGroupInformation.getCurrentUser().getUserName());
-                TokenUtil.obtainAndCacheToken(hconf, UserGroupInformation.getCurrentUser());
-            } catch (InterruptedException e) {
-                System.out.println("--------------Error while getting kerberos credential for user " + UserGroupInformation.getCurrentUser().getUserName());
-            }
-        }
-
-        Scan scan = new Scan();
-        int limit = 20;
-
-        HConnection conn = null;
-        HTableInterface table = null;
-        ResultScanner scanner = null;
-        try {
-            conn = HConnectionManager.createConnection(hconf);
-            table = conn.getTable(hbaseTable);
-            scanner = table.getScanner(scan);
-            int count = 0;
-            for (Result r : scanner) {
-                byte[] rowkey = r.getRow();
-                System.out.println(Bytes.toStringBinary(rowkey));
-                count++;
-                if (count == limit)
-                    break;
-            }
-        } finally {
-            if (scanner != null) {
-                scanner.close();
-            }
-            if (table != null) {
-                table.close();
-            }
-            if (conn != null) {
-                conn.close();
-            }
-        }
-
-    }
-}
+/*
+ * 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.storage.hbase;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.TokenUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.HadoopUtil;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class PingHBaseCLI {
+
+    public static void main(String[] args) throws IOException, InterruptedException {
+        String hbaseTable = args[0];
+
+        System.out.println("Hello friend.");
+
+        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
+        if (User.isHBaseSecurityEnabled(hconf)) {
+            Connection conn = ConnectionFactory.createConnection(hconf);
+            try {
+                UserProvider userProvider = UserProvider.instantiate(hconf);
+                TokenUtil.obtainAndCacheToken(conn, userProvider.create(UserGroupInformation.getCurrentUser()));
+            } finally {
+                conn.close();
+            }
+        }
+
+        Scan scan = new Scan();
+        int limit = 20;
+
+        Connection conn = null;
+        Table table = null;
+        ResultScanner scanner = null;
+        try {
+            conn = ConnectionFactory.createConnection(hconf);
+            table = conn.getTable(TableName.valueOf(hbaseTable));
+            scanner = table.getScanner(scan);
+            int count = 0;
+            for (Result r : scanner) {
+                byte[] rowkey = r.getRow();
+                System.out.println(Bytes.toStringBinary(rowkey));
+                count++;
+                if (count == limit)
+                    break;
+            }
+        } finally {
+            if (scanner != null) {
+                scanner.close();
+            }
+            if (table != null) {
+                table.close();
+            }
+            if (conn != null) {
+                conn.close();
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java b/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java
index e2eeed0..a07cbe4 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 
 /**
  * @author yangli9
@@ -50,7 +51,7 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
@@ -60,11 +61,16 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
     @Override
+    public int getBatch() {
+        return -1;
+    }
+
+    @Override
     public void close() throws IOException {
         scanner.close();
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
index a115753..e950e5b 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
@@ -24,7 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.kv.RowValueDecoder;
@@ -52,7 +52,7 @@ public class SerializedHBaseTupleIterator implements ITupleIterator {
     private ITupleIterator segmentIterator;
     private int scanCount;
 
-    public SerializedHBaseTupleIterator(HConnection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, Collection<TblColRef> dimensions, TupleFilter filter, Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context) {
+    public SerializedHBaseTupleIterator(Connection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, Collection<TblColRef> dimensions, TupleFilter filter, Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context) {
 
         this.context = context;
         int limit = context.getLimit();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
index 7a0ab15..a3ece7e 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
@@ -26,8 +26,9 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
@@ -79,14 +80,14 @@ public class EndpointTupleIterator implements ITupleIterator {
 
     Iterator<List<IIProtos.IIResponse.IIRow>> regionResponsesIterator = null;
     ITupleIterator tupleIterator = null;
-    HTableInterface table = null;
+    Table table = null;
 
     int rowsInAllMetric = 0;
 
-    public EndpointTupleIterator(IISegment segment, TupleFilter rootFilter, Collection<TblColRef> groupBy, List<FunctionDesc> measures, StorageContext context, HConnection conn) throws Throwable {
+    public EndpointTupleIterator(IISegment segment, TupleFilter rootFilter, Collection<TblColRef> groupBy, List<FunctionDesc> measures, StorageContext context, Connection conn) throws Throwable {
 
         String tableName = segment.getStorageLocationIdentifier();
-        table = conn.getTable(tableName);
+        table = conn.getTable(TableName.valueOf(tableName));
         factTableName = segment.getIIDesc().getFactTableName();
 
         if (rootFilter == null) {
@@ -212,7 +213,7 @@ public class EndpointTupleIterator implements ITupleIterator {
     }
 
     //TODO : async callback
-    private Iterator<List<IIProtos.IIResponse.IIRow>> getResults(final IIProtos.IIRequest request, HTableInterface table) throws Throwable {
+    private Iterator<List<IIProtos.IIResponse.IIRow>> getResults(final IIProtos.IIRequest request, Table table) throws Throwable {
         Map<byte[], List<IIProtos.IIResponse.IIRow>> results = table.coprocessorService(IIProtos.RowsService.class, null, null, new Batch.Call<IIProtos.RowsService, List<IIProtos.IIResponse.IIRow>>() {
             public List<IIProtos.IIResponse.IIRow> call(IIProtos.RowsService rowsService) throws IOException {
                 ServerRpcController controller = new ServerRpcController();
@@ -235,7 +236,7 @@ public class EndpointTupleIterator implements ITupleIterator {
         int index = 0;
 
         for (int i = 0; i < columns.size(); i++) {
-            TblColRef column = columns.get(i);
+            //            TblColRef column = columns.get(i);
             //            if (!dimensions.contains(column)) {
             //                continue;
             //            }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
index d55361b..3c91329 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
@@ -90,7 +90,7 @@ public class IIEndpoint extends IIProtos.RowsService implements Coprocessor, Cop
         RegionScanner innerScanner = null;
         HRegion region = null;
         try {
-            region = env.getRegion();
+            region = (HRegion) env.getRegion();
             innerScanner = region.getScanner(buildScan());
             region.startRegionOperation();
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java
index 2cecd5c..c21ee36 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java
@@ -93,7 +93,7 @@ public class AggregateRegionObserver extends BaseRegionObserver {
         // start/end region operation & sync on scanner is suggested by the
         // javadoc of RegionScanner.nextRaw()
         // FIXME: will the lock still work when a iterator is returned? is it safe? Is readonly attribute helping here? by mhb
-        HRegion region = ctxt.getEnvironment().getRegion();
+        HRegion region = (HRegion) ctxt.getEnvironment().getRegion();
         region.startRegionOperation();
         try {
             synchronized (innerScanner) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
index 65b616f..c1a0b15 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.metadata.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorFilter;
@@ -93,13 +94,18 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
+    public int getBatch() {
+        return outerScanner.getBatch();
+    }
+
+    @Override
     public boolean next(List<Cell> results) throws IOException {
         return outerScanner.next(results);
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
-        return outerScanner.next(result, limit);
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.next(result, scannerContext);
     }
 
     @Override
@@ -108,8 +114,8 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-        return outerScanner.nextRaw(result, limit);
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.nextRaw(result, scannerContext);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
index f609a5a..dea1d9e 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.metadata.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.coprocessor.AggregationCache;
@@ -107,7 +108,7 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -117,11 +118,16 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+        public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
         @Override
+        public int getBatch() {
+            return innerScanner.getBatch();
+        }
+
+        @Override
         public void close() throws IOException {
             // AggregateRegionObserver.LOG.info("Kylin Scanner close()");
             innerScanner.close();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
index f0f7ed5..fa2a7c1 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
@@ -23,9 +23,9 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
@@ -58,7 +58,7 @@ public class ObserverEnabler {
     static final Map<String, Boolean> CUBE_OVERRIDES = Maps.newConcurrentMap();
 
     public static ResultScanner scanWithCoprocessorIfBeneficial(CubeSegment segment, Cuboid cuboid, TupleFilter tupleFiler, //
-            Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context, HTableInterface table, Scan scan) throws IOException {
+            Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context, Table table, Scan scan) throws IOException {
 
         if (context.isCoprocessorEnabled() == false) {
             return table.getScanner(scan);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java b/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java
index f7fcef1..50069a1 100644
--- a/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java
@@ -1,115 +1,112 @@
-/*
- * 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.storage.hbase;
-
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.RawTableRecord;
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.invertedindex.index.TableRecord;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-/**
- * @author yangli9
- */
-public class InvertedIndexHBaseTest extends HBaseMetadataTestCase {
-
-    IIInstance ii;
-    IISegment seg;
-    HConnection hconn;
-
-    TableRecordInfo info;
-
-    @Before
-    public void setup() throws Exception {
-        this.createTestMetadata();
-
-        this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii");
-        this.seg = ii.getFirstSegment();
-
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        hconn = HConnectionManager.createConnection(hconf);
-
-        this.info = new TableRecordInfo(seg);
-    }
-
-    @After
-    public void after() throws Exception {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void testLoad() throws Exception {
-
-        String tableName = seg.getStorageLocationIdentifier();
-        IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest());
-
-        List<Slice> slices = Lists.newArrayList();
-        HBaseClientKVIterator kvIterator = new HBaseClientKVIterator(hconn, tableName, IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
-        try {
-            for (Slice slice : codec.decodeKeyValue(kvIterator)) {
-                slices.add(slice);
-            }
-        } finally {
-            kvIterator.close();
-        }
-
-        List<TableRecord> records = iterateRecords(slices);
-        dump(records);
-        System.out.println(records.size() + " records");
-    }
-
-    private List<TableRecord> iterateRecords(List<Slice> slices) {
-        List<TableRecord> records = Lists.newArrayList();
-        for (Slice slice : slices) {
-            for (RawTableRecord rec : slice) {
-                records.add(new TableRecord((RawTableRecord) rec.clone(), info));
-            }
-        }
-        return records;
-    }
-
-    private void dump(Iterable<TableRecord> records) {
-        for (TableRecord rec : records) {
-            System.out.println(rec.toString());
-
-            byte[] x = rec.getBytes();
-            String y = BytesUtil.toReadableText(x);
-            System.out.println(y);
-            System.out.println();
-        }
-    }
-
-}
+/*
+ * 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.storage.hbase;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.persistence.HBaseConnection;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.HBaseMetadataTestCase;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IIManager;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.invertedindex.index.RawTableRecord;
+import org.apache.kylin.invertedindex.index.Slice;
+import org.apache.kylin.invertedindex.index.TableRecord;
+import org.apache.kylin.invertedindex.index.TableRecordInfo;
+import org.apache.kylin.invertedindex.model.IIDesc;
+import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * @author yangli9
+ */
+public class InvertedIndexHBaseTest extends HBaseMetadataTestCase {
+
+    IIInstance ii;
+    IISegment seg;
+    Connection hconn;
+
+    TableRecordInfo info;
+
+    @Before
+    public void setup() throws Exception {
+        this.createTestMetadata();
+
+        this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii");
+        this.seg = ii.getFirstSegment();
+
+        this.hconn = HBaseConnection.get();
+
+        this.info = new TableRecordInfo(seg);
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testLoad() throws Exception {
+
+        String tableName = seg.getStorageLocationIdentifier();
+        IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest());
+
+        List<Slice> slices = Lists.newArrayList();
+        HBaseClientKVIterator kvIterator = new HBaseClientKVIterator(hconn, tableName, IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
+        try {
+            for (Slice slice : codec.decodeKeyValue(kvIterator)) {
+                slices.add(slice);
+            }
+        } finally {
+            kvIterator.close();
+        }
+
+        List<TableRecord> records = iterateRecords(slices);
+        dump(records);
+        System.out.println(records.size() + " records");
+    }
+
+    private List<TableRecord> iterateRecords(List<Slice> slices) {
+        List<TableRecord> records = Lists.newArrayList();
+        for (Slice slice : slices) {
+            for (RawTableRecord rec : slice) {
+                records.add(new TableRecord((RawTableRecord) rec.clone(), info));
+            }
+        }
+        return records;
+    }
+
+    private void dump(Iterable<TableRecord> records) {
+        for (TableRecord rec : records) {
+            System.out.println(rec.toString());
+
+            byte[] x = rec.getBytes();
+            String y = BytesUtil.toReadableText(x);
+            System.out.println(y);
+            System.out.println();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java b/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java
index 0454b4c..3ace91e 100644
--- a/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.kv.RowConstants;
@@ -223,102 +224,46 @@ public class AggregateRegionObserverTest {
             this.input = cellInputs;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.InternalScanner#next(java.util
-         * .List)
-         */
         @Override
         public boolean next(List<Cell> results) throws IOException {
             return nextRaw(results);
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.InternalScanner#next(java.util
-         * .List, int)
-         */
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see org.apache.hadoop.hbase.regionserver.InternalScanner#close()
-         */
         @Override
         public void close() throws IOException {
 
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#getRegionInfo()
-         */
         @Override
         public HRegionInfo getRegionInfo() {
             return null;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#isFilterDone()
-         */
         @Override
         public boolean isFilterDone() throws IOException {
             return false;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#reseek(byte[])
-         */
         @Override
         public boolean reseek(byte[] row) throws IOException {
             return false;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#getMaxResultSize()
-         */
         @Override
         public long getMaxResultSize() {
             return 0;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#getMvccReadPoint()
-         */
         @Override
         public long getMvccReadPoint() {
             return 0;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#nextRaw(java.util
-         * .List)
-         */
         @Override
         public boolean nextRaw(List<Cell> result) throws IOException {
             if (i < input.size()) {
@@ -328,18 +273,15 @@ public class AggregateRegionObserverTest {
             return i < input.size();
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#nextRaw(java.util
-         * .List, int)
-         */
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+        public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return nextRaw(result);
         }
 
+        @Override
+        public int getBatch() {
+            return -1;
+        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9d63d9a9/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java b/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java
index d17cfa6..b1f6626 100644
--- a/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java
@@ -27,7 +27,6 @@ import java.sql.SQLException;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.service.HiveInterface;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
 
@@ -47,7 +46,7 @@ public class HiveMiniClusterTest extends HiveJDBCClientTest {
     public static final File HIVE_WAREHOUSE_DIR = new File(HIVE_BASE_DIR + "/warehouse");
     public static final File HIVE_TESTDATA_DIR = new File(HIVE_BASE_DIR + "/testdata");
     public static final File HIVE_HADOOP_TMP_DIR = new File(HIVE_BASE_DIR + "/hadooptmp");
-    protected HiveInterface client;
+    //protected HiveInterface client;
 
     protected MiniDFSCluster miniDFS;
     protected MiniMRCluster miniMR;


[11/30] incubator-kylin git commit: update powered by page

Posted by li...@apache.org.
update powered by page


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 1669ed325edd510d1e045d9012768a53362e7de4
Parents: 70e3b39
Author: Luke Han <lu...@apache.org>
Authored: Thu Nov 5 17:27:17 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Thu Nov 5 17:27:17 2015 +0800

----------------------------------------------------------------------
 .gitignore                     |  6 +++++-
 website/community/index.md     |  4 ++++
 website/community/poweredby.md | 23 +++++++++++++++--------
 3 files changed, 24 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1669ed32/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 9a27b8a..147728d 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,11 +7,15 @@ tmp/
 *.bak
 *.swp
 *~.nib
-*.versionsBackup
+
 local.properties
 release.properties
 .loadpath
 
+# maven release files
+*.versionsBackup
+**/pom.xml.tag
+
 #IDEA
 *.iml
 #.settings

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1669ed32/website/community/index.md
----------------------------------------------------------------------
diff --git a/website/community/index.md b/website/community/index.md
index c573e08..02a692b 100644
--- a/website/community/index.md
+++ b/website/community/index.md
@@ -4,6 +4,10 @@ title: Community
 permalink: /community/index.html
 ---
 
+### Powered By Apache Kylin
+For information about who are using Apache Kylin, please refer to [Powered By](/community/poweredby.html) page.
+
+
 ### Apache Kylin Mailing List
 
 These are the mailing lists that have been established for this project. For each list, there is a subscribe, unsubscribe, and an archive link.

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1669ed32/website/community/poweredby.md
----------------------------------------------------------------------
diff --git a/website/community/poweredby.md b/website/community/poweredby.md
index 7623bec..586e523 100644
--- a/website/community/poweredby.md
+++ b/website/community/poweredby.md
@@ -7,13 +7,20 @@ permalink: /community/poweredby.html
 ## Powered By Apache Kylin
 
 __How to be listed here?__
-_Send a quick description of your organization and usage to the [mailing list](mailto:user@kylin.incubator.apache.org) or to [@apachekylin](https://twitter.com/apachekylin) or [@lukehq](https://twitter.com/lukehq) on twitter and we'll add you._
+Send a quick description of your organization and usage to the [mailing list](mailto:user@kylin.incubator.apache.org) or to [@apachekylin](https://twitter.com/apachekylin) or [@lukehq](https://twitter.com/lukehq) on twitter and we'll add you.
 
-__Companies__
-
-* [eBay](http://www.ebay.com), Apache Kylin is used at eBay for Big Data Analytics on Hadoop. This powers various data products including Behavior Analytics, Traffic Reporting, Account Manager Application and Streaming Dashboard.
-* [JCloud](http://www.jcloud.com), Apache Kylin is used as Data Analytics Engine to analyse JOS <http://jos.jd.com/> API access behavior and report.
-* [MiningLAMP](http://www.mininglamp.com/), Kylin is part of MiningLAMP's Big Data Analytics product offering OLAP capability 
-* [Meituan](http://www.meituan.com)
-* 
+__Companies & Organizations__
 
+* [eBay](http://www.ebay.com)  (_NASDAQ: EBAY_)
+    * Apache Kylin is used at eBay for Big Data Analytics on Hadoop. This powers various data products including Behavior Analytics, Traffic Reporting, Account Manager Application and Streaming Dashboard.
+* [JD.com, Inc.](http://www.jd.com)  (_NASDAQ: JD_)
+    * Apache Kylin as Data Analytics Engine to analysis [JOS](http://jos.jd.com) API access behavior and report in [JCloud](http://www.jcloud.com).
+* [MiningLAMP](http://www.mininglamp.com)
+    * Apache Kylin is part of MiningLAMP's Big Data Analytics product offering OLAP capability for customers from Banking, Finance and others industuies.
+* [Meituan.com](http://www.meituan.com) (_[CrunchBase:Meituan.com](https://www.crunchbase.com/organization/meituan-com)_)
+    * Apache Kylin is powering various OLAP scenarios at Meituan's Data Platform, such as traffic analysis and payment channel analysis. It's quite stable and super fast. We are very willing to try Kylin on other product as well.
+* [VIP.com](http://www.vip.com)  (_NYSE: VIPS_)
+    * Apache Kylin is used at VIP.com’s big data self-service analysis platform. Apache Kylin gives us a new option to make user experience more ad-hoc friendly and give us the opportunity to introduce Cube back to Big Data.
+* [DreamSoft](http://www.dream-it.cn/)
+    * DreamSoft is using Apache Kylin as Big Data Analytics Engine for their customers coming from Reatail, Manufacturing and others.
+    


[07/30] incubator-kylin git commit: KYLIN-999 remove ref to Admin-fonts

Posted by li...@apache.org.
KYLIN-999 remove ref to Admin-fonts

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 2feb85d0e4dc8e1cd64db95a26d27af9c566b032
Parents: 46c2163
Author: jiazhong <ji...@ebay.com>
Authored: Wed Nov 4 11:36:47 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 11:16:22 2015 +0800

----------------------------------------------------------------------
 pom.xml               | 1 -
 webapp/app/index.html | 1 -
 webapp/grunt.json     | 1 -
 3 files changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2feb85d0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1b4f862..16da25e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -651,7 +651,6 @@
                                 <exclude>dictionary/metastore_db/**</exclude>
 
                                 <!-- MIT license -->
-                                <exclude>webapp/app/css/AdminLTE-fonts.css</exclude>
                                 <exclude>webapp/app/css/AdminLTE.css</exclude>
 
                                 <!-- jdbc log -->

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2feb85d0/webapp/app/index.html
----------------------------------------------------------------------
diff --git a/webapp/app/index.html b/webapp/app/index.html
index f1087ed..1382df1 100644
--- a/webapp/app/index.html
+++ b/webapp/app/index.html
@@ -30,7 +30,6 @@
     <!-- ref:css css/styles.min.<%= buildNumber %>.css -->
     <link rel="stylesheet" type="text/css" href="components/bootstrap/dist/css/bootstrap.min.css">
     <link rel="stylesheet" type="text/css" href="components/font-awesome/css/font-awesome.css">
-    <link rel="stylesheet" type="text/css" href="css/AdminLTE-fonts.css">
     <link rel="stylesheet" type="text/css" href="components/ng-grid/ng-grid.css">
     <link rel="stylesheet" type="text/css" href="components/angular-tree-control/css/tree-control.css">
     <link rel="stylesheet" type="text/css" href="components/angular-tree-control/css/tree-control-attribute.css">

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2feb85d0/webapp/grunt.json
----------------------------------------------------------------------
diff --git a/webapp/grunt.json b/webapp/grunt.json
index fa272c9..a9b072d 100755
--- a/webapp/grunt.json
+++ b/webapp/grunt.json
@@ -49,7 +49,6 @@
             "src": [
                 "app/components/bootstrap/dist/css/bootstrap.min.css",
                 "app/components/font-awesome/css/font-awesome.min.css",
-                "app/css/AdminLTE-fonts.css",
                 "app/components/ng-grid/ng-grid.css",
                 "app/components/angular-tree-control/css/tree-control.css",
                 "app/components/angular-tree-control/css/tree-control-attribute.css",


[15/30] incubator-kylin git commit: KYLIN-1099 Fix test case for windows

Posted by li...@apache.org.
KYLIN-1099 Fix test case for windows


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 6b6be7397872a51606a3dca8b6b67bbd43058493
Parents: a441237
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Nov 11 10:49:25 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Nov 11 10:49:25 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/util/HadoopUtil.java | 19 +++++++++++++++----
 .../kylin/dict/lookup/FileTableReader.java       |  2 ++
 2 files changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6b6be739/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java b/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
index 3255b32..483a7a1 100644
--- a/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
+++ b/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
@@ -65,12 +65,23 @@ public class HadoopUtil {
 
     public static URI makeURI(String filePath) {
         try {
-            return new URI(filePath);
+            return new URI(fixWindowsPath(filePath));
         } catch (URISyntaxException e) {
             throw new IllegalArgumentException("Cannot create FileSystem from URI: " + filePath, e);
         }
     }
 
+    public static String fixWindowsPath(String path) {
+        // fix windows path
+        if (path.startsWith("file://") && !path.startsWith("file:///") && path.contains(":\\")) {
+            path = path.replace("file://", "file:///");
+        }
+        if (path.startsWith("file:///")) {
+            path = path.replace('\\', '/');
+        }
+        return path;
+    }
+
     public static String makeQualifiedPathInHadoopCluster(String path) {
         try {
             FileSystem fs = FileSystem.get(getCurrentConfiguration());
@@ -93,10 +104,10 @@ public class HadoopUtil {
         Configuration conf = new Configuration();
         return healSickConfig(conf);
     }
-    
+
     public static Configuration newHBaseConfiguration(String url) {
         Configuration conf = HBaseConfiguration.create(getCurrentConfiguration());
-        
+
         // using a hbase:xxx URL is deprecated, instead hbase config is always loaded from hbase-site.xml in classpath
         if (!(StringUtils.isEmpty(url) || "hbase".equals(url)))
             throw new IllegalArgumentException("to use hbase storage, pls set 'kylin.storage.url=hbase' in kylin.properties");
@@ -106,7 +117,7 @@ public class HadoopUtil {
         if (StringUtils.isNotEmpty(hbaseClusterFs)) {
             conf.set(FileSystem.FS_DEFAULT_NAME_KEY, hbaseClusterFs);
         }
-        
+
         // reduce rpc retry
         conf.set(HConstants.HBASE_CLIENT_PAUSE, "3000");
         conf.set(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "5");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6b6be739/dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTableReader.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTableReader.java b/dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTableReader.java
index 5ef7731..a6de5df 100644
--- a/dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTableReader.java
+++ b/dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTableReader.java
@@ -172,6 +172,7 @@ public class FileTableReader implements TableReader {
         Text value;
 
         SeqRowReader(Configuration hconf, FileSystem fs, String path) throws IOException {
+            path = HadoopUtil.fixWindowsPath(path);
             reader = new Reader(hconf, SequenceFile.Reader.file(new Path(path)));
             key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), hconf);
             value = new Text();
@@ -196,6 +197,7 @@ public class FileTableReader implements TableReader {
         BufferedReader reader;
 
         CsvRowReader(FileSystem fs, String path) throws IOException {
+            path = HadoopUtil.fixWindowsPath(path);
             FSDataInputStream in = fs.open(new Path(path));
             reader = new BufferedReader(new InputStreamReader(in, "UTF-8"));
         }


[24/30] incubator-kylin git commit: KYLIN-1152 ResourceStore add getResourceTimestamp() API

Posted by li...@apache.org.
KYLIN-1152 ResourceStore add getResourceTimestamp() API


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: e9219d7ba42379d929c5c2ebf183b3919381cc05
Parents: 2f788e7
Author: Li, Yang <ya...@ebay.com>
Authored: Tue Nov 17 13:54:19 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Nov 17 13:54:19 2015 +0800

----------------------------------------------------------------------
 .../kylin/common/persistence/FileResourceStore.java      |  9 +++++++++
 .../kylin/common/persistence/HBaseResourceStore.java     |  7 ++++++-
 .../apache/kylin/common/persistence/ResourceStore.java   |  9 ++++++++-
 .../apache/kylin/job/hadoop/cube/MetadataCleanupJob.java | 11 ++++-------
 4 files changed, 27 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e9219d7b/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
index 98c6b18..646cd80 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
@@ -100,6 +100,15 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
+    protected long getResourceTimestampImpl(String resPath) throws IOException {
+        File f = file(resPath);
+        if (f.exists() && f.isFile())
+            return f.lastModified();
+        else
+            return 0;
+    }
+    
+    @Override
     protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException {
         File f = file(resPath);
         f.getParentFile().mkdirs();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e9219d7b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
index 2b14345..35a62b5 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
@@ -206,6 +206,11 @@ public class HBaseResourceStore extends ResourceStore {
     }
 
     @Override
+    protected long getResourceTimestampImpl(String resPath) throws IOException {
+        return getTimestamp(getByScan(resPath, false, true));
+    }
+    
+    @Override
     protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException {
         ByteArrayOutputStream bout = new ByteArrayOutputStream();
         IOUtils.copy(content, bout);
@@ -233,7 +238,7 @@ public class HBaseResourceStore extends ResourceStore {
 
             boolean ok = table.checkAndPut(row, B_FAMILY, B_COLUMN_TS, bOldTS, put);
             if (!ok) {
-                long real = getTimestamp(getByScan(resPath, false, true));
+                long real = getResourceTimestampImpl(resPath);
                 throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + real + ", but it is " + oldTS);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e9219d7b/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
index a23a4cd..db70997 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
@@ -146,6 +146,10 @@ abstract public class ResourceStore {
         return getResourceImpl(norm(resPath));
     }
 
+    final public long getResourceTimestamp(String resPath) throws IOException {
+        return getResourceTimestampImpl(norm(resPath));
+    }
+    
     final public <T extends RootPersistentEntity> List<T> getAllResources(String rangeStart, String rangeEnd, Class<T> clazz, Serializer<T> serializer) throws IOException {
         final List<RawResource> allResources = getAllResources(rangeStart, rangeEnd);
         if (allResources.isEmpty()) {
@@ -170,7 +174,10 @@ abstract public class ResourceStore {
 
     /** returns null if not exists */
     abstract protected RawResource getResourceImpl(String resPath) throws IOException;
-
+    
+    /** returns 0 if not exists */
+    abstract protected long getResourceTimestampImpl(String resPath) throws IOException;
+    
     /**
      * overwrite a resource without write conflict check
      */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e9219d7b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
index cb601c5..6d06dcc 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
@@ -28,7 +28,6 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
@@ -115,9 +114,8 @@ public class MetadataCleanupJob extends AbstractHadoopJob {
                     if (snapshotNames != null)
                         for (String snapshot : snapshotNames) {
                             if (!activeResourceList.contains(snapshot)) {
-                                RawResource res = getStore().getResource(snapshot);
-                                res.inputStream.close();
-                                if (isOlderThanThreshold(res.timestamp))
+                                long ts = getStore().getResourceTimestamp(snapshot);
+                                if (isOlderThanThreshold(ts))
                                     toDeleteResource.add(snapshot);
                             }
                         }
@@ -137,9 +135,8 @@ public class MetadataCleanupJob extends AbstractHadoopJob {
                         if (dictionaries != null)
                             for (String dict : dictionaries)
                                 if (!activeResourceList.contains(dict)) {
-                                    RawResource res = getStore().getResource(dict);
-                                    res.inputStream.close();
-                                    if (isOlderThanThreshold(res.timestamp))
+                                    long ts = getStore().getResourceTimestamp(dict);
+                                    if (isOlderThanThreshold(ts))
                                         toDeleteResource.add(dict);
                                 }
                     }


[12/30] incubator-kylin git commit: update powered by page

Posted by li...@apache.org.
update powered by page


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: c5ada2995b6750934114e629dbde675e23221b42
Parents: 70e3b39
Author: Luke Han <lu...@apache.org>
Authored: Thu Nov 5 17:27:17 2015 +0800
Committer: lukehan <lu...@apache.org>
Committed: Thu Nov 5 21:57:24 2015 +0800

----------------------------------------------------------------------
 .gitignore                     |  6 +++++-
 website/community/index.md     |  4 ++++
 website/community/poweredby.md | 25 +++++++++++++++++--------
 3 files changed, 26 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c5ada299/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 9a27b8a..147728d 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,11 +7,15 @@ tmp/
 *.bak
 *.swp
 *~.nib
-*.versionsBackup
+
 local.properties
 release.properties
 .loadpath
 
+# maven release files
+*.versionsBackup
+**/pom.xml.tag
+
 #IDEA
 *.iml
 #.settings

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c5ada299/website/community/index.md
----------------------------------------------------------------------
diff --git a/website/community/index.md b/website/community/index.md
index c573e08..02a692b 100644
--- a/website/community/index.md
+++ b/website/community/index.md
@@ -4,6 +4,10 @@ title: Community
 permalink: /community/index.html
 ---
 
+### Powered By Apache Kylin
+For information about who are using Apache Kylin, please refer to [Powered By](/community/poweredby.html) page.
+
+
 ### Apache Kylin Mailing List
 
 These are the mailing lists that have been established for this project. For each list, there is a subscribe, unsubscribe, and an archive link.

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c5ada299/website/community/poweredby.md
----------------------------------------------------------------------
diff --git a/website/community/poweredby.md b/website/community/poweredby.md
index 7623bec..42210a9 100644
--- a/website/community/poweredby.md
+++ b/website/community/poweredby.md
@@ -7,13 +7,22 @@ permalink: /community/poweredby.html
 ## Powered By Apache Kylin
 
 __How to be listed here?__
-_Send a quick description of your organization and usage to the [mailing list](mailto:user@kylin.incubator.apache.org) or to [@apachekylin](https://twitter.com/apachekylin) or [@lukehq](https://twitter.com/lukehq) on twitter and we'll add you._
+Send a quick description of your organization and usage to the [mailing list](mailto:user@kylin.incubator.apache.org) or to [@apachekylin](https://twitter.com/apachekylin) or [@lukehq](https://twitter.com/lukehq) on twitter and we'll add you.
 
-__Companies__
-
-* [eBay](http://www.ebay.com), Apache Kylin is used at eBay for Big Data Analytics on Hadoop. This powers various data products including Behavior Analytics, Traffic Reporting, Account Manager Application and Streaming Dashboard.
-* [JCloud](http://www.jcloud.com), Apache Kylin is used as Data Analytics Engine to analyse JOS <http://jos.jd.com/> API access behavior and report.
-* [MiningLAMP](http://www.mininglamp.com/), Kylin is part of MiningLAMP's Big Data Analytics product offering OLAP capability 
-* [Meituan](http://www.meituan.com)
-* 
+__Companies & Organizations__
 
+* [eBay](http://www.ebay.com)  (_NASDAQ: EBAY_)
+    * Apache Kylin is used at eBay for Big Data Analytics on Hadoop. This powers various data products including Behavior Analytics, Traffic Reporting, Account Manager Application and Streaming Dashboard.
+* [JD.com, Inc.](http://www.jd.com)  (_NASDAQ: JD_)
+    * Apache Kylin as Data Analytics Engine to analysis [JOS](http://jos.jd.com) API access behavior and report in [JCloud](http://www.jcloud.com).
+* [MiningLAMP](http://www.mininglamp.com)
+    * Apache Kylin is part of MiningLAMP's Big Data Analytics product offering OLAP capability for customers from Banking, Finance and others industuies.
+* [Meituan.com](http://www.meituan.com) (_[CrunchBase:Meituan.com](https://www.crunchbase.com/organization/meituan-com)_)
+    * Apache Kylin is powering various OLAP scenarios at Meituan's Data Platform, such as traffic analysis and payment channel analysis. It's quite stable and super fast. We are very willing to try Kylin on other product as well.
+* [VIP.com](http://www.vip.com)  (_NYSE: VIPS_)
+    * Apache Kylin is used at VIP.com’s big data self-service analysis platform. Apache Kylin gives us a new option to make user experience more ad-hoc friendly and give us the opportunity to introduce Cube back to Big Data.
+* [DreamSoft](http://www.dream-it.cn/)
+    * DreamSoft is using Apache Kylin as Big Data Analytics Engine for their customers coming from Retail, Manufacturing, Finance and others.
+* [Exponential](http://www.exponential.com)
+	* Using Kylin as a component in it's data platform LAMBDA to power inventory, campaign, behavior and demand analysis for advertising. It has been great at handling the size and speed constraints of old gen systems. With the first version already in production, we are looking forward to pushing more data into kylin and achieve near real time updates. 
+    


[25/30] incubator-kylin git commit: KYLIN-993 Support functions in where clause

Posted by li...@apache.org.
KYLIN-993 Support functions in where clause

Signed-off-by: Li, Yang <ya...@ebay.com>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: bed15abbdd70305f5bd998eb7fb1775e6900fcee
Parents: e9219d7
Author: lidongsjtu <do...@ebay.com>
Authored: Mon Nov 16 22:39:56 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Nov 17 16:51:12 2015 +0800

----------------------------------------------------------------------
 .../metadata/filter/CompareTupleFilter.java     |  23 +--
 .../metadata/filter/FunctionTupleFilter.java    | 154 +++++++++++++++++
 .../metadata/filter/ITupleFilterTranslator.java |  26 +++
 .../kylin/metadata/filter/TupleFilter.java      |   2 +-
 .../metadata/filter/util/BuiltInMethod.java     |  56 +++++++
 .../kylin/query/relnode/OLAPFilterRel.java      |  45 ++---
 query/src/test/resources/query/sql/query85.sql  |  22 +++
 .../kylin/storage/hbase/CubeStorageEngine.java  |   2 +
 .../hbase/coprocessor/CoprocessorFilter.java    |   9 +-
 .../CoprocessorTupleFilterTranslator.java       | 166 +++++++++++++++++++
 10 files changed, 457 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
index d6368dd..6754ff7 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
@@ -19,12 +19,9 @@
 package org.apache.kylin.metadata.filter;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
+import java.util.*;
 
+import org.apache.calcite.sql.SqlFunction;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
@@ -39,6 +36,7 @@ public class CompareTupleFilter extends TupleFilter {
     private String firstCondValue;
     private Map<String, String> dynamicVariables;
     private String nullString;
+    private FunctionTupleFilter functionTupleFilter;
 
     public CompareTupleFilter(FilterOperatorEnum op) {
         super(new ArrayList<TupleFilter>(2), op);
@@ -49,7 +47,7 @@ public class CompareTupleFilter extends TupleFilter {
                 || op == FilterOperatorEnum.GT || op == FilterOperatorEnum.GTE //
                 || op == FilterOperatorEnum.IN || op == FilterOperatorEnum.NOTIN //
                 || op == FilterOperatorEnum.ISNULL || op == FilterOperatorEnum.ISNOTNULL);
-        if (opGood == false)
+        if (!opGood)
             throw new IllegalArgumentException("Unsupported operator " + op);
     }
 
@@ -81,6 +79,8 @@ public class CompareTupleFilter extends TupleFilter {
         } else if (child instanceof DynamicTupleFilter) {
             DynamicTupleFilter dynamicFilter = (DynamicTupleFilter) child;
             this.dynamicVariables.put(dynamicFilter.getVariableName(), null);
+        } else if (child instanceof FunctionTupleFilter) {
+            this.functionTupleFilter = (FunctionTupleFilter)child;
         }
         //TODO
         //        else if (child instanceof ExtractTupleFilter) {
@@ -105,6 +105,10 @@ public class CompareTupleFilter extends TupleFilter {
         return column;
     }
 
+    public FunctionTupleFilter getFunctionTupleFilter() {
+        return functionTupleFilter;
+    }
+
     public Map<String, String> getVariables() {
         return dynamicVariables;
     }
@@ -137,7 +141,7 @@ public class CompareTupleFilter extends TupleFilter {
 
     @Override
     public String toString() {
-        return "CompareFilter [" + column + " " + operator + " " + conditionValues + ", children=" + children + "]";
+        return "CompareFilter [" + (functionTupleFilter == null ? column : functionTupleFilter) + " " + operator + " " + conditionValues + ", children=" + children + "]";
     }
 
     // TODO requires generalize, currently only evaluates COLUMN {op} CONST
@@ -146,7 +150,7 @@ public class CompareTupleFilter extends TupleFilter {
         // extract tuple value
         String tupleValue = null;
         for (TupleFilter filter : this.children) {
-            if (isConstant(filter) == false) {
+            if (!isConstant(filter)) {
                 filter.evaluate(tuple);
                 tupleValue = filter.getValues().iterator().next();
             }
@@ -209,7 +213,7 @@ public class CompareTupleFilter extends TupleFilter {
 
     @Override
     public boolean isEvaluable() {
-        return column != null && !conditionValues.isEmpty();
+        return (functionTupleFilter != null || column != null) && !conditionValues.isEmpty();
     }
 
     @Override
@@ -239,5 +243,4 @@ public class CompareTupleFilter extends TupleFilter {
         }
         this.nullString = BytesUtil.readAsciiString(buffer);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
new file mode 100644
index 0000000..62ab42f
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
@@ -0,0 +1,154 @@
+/*
+ * 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.metadata.filter;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Primitives;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.kylin.metadata.filter.util.BuiltInMethod;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.tuple.ITuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Created by dongli on 11/11/15.
+ */
+public class FunctionTupleFilter extends TupleFilter {
+    public static final Logger logger = LoggerFactory.getLogger(FunctionTupleFilter.class);
+
+    private SqlOperator sqlOperator;
+    // FIXME Only supports single parameter functions currently
+    private TupleFilter columnContainerFilter;
+    private int colPosition;
+    private Method method;
+    private List<Object> methodParams;
+    private boolean isValid = false;
+
+    public FunctionTupleFilter(SqlOperator sqlOperator) {
+        super(Lists.<TupleFilter>newArrayList(), FilterOperatorEnum.FUNCTION);
+        this.methodParams = Lists.newArrayList();
+        this.sqlOperator = sqlOperator;
+
+        String opName = sqlOperator.getName().toUpperCase();
+        if (BuiltInMethod.MAP.containsKey(opName)) {
+            this.method = BuiltInMethod.MAP.get(opName).method;
+            isValid = true;
+        }
+    }
+
+    public SqlOperator getSqlOperator() {
+        return sqlOperator;
+    }
+
+    public TblColRef getColumn() {
+        if (columnContainerFilter == null)
+            return null;
+
+        if (columnContainerFilter instanceof ColumnTupleFilter)
+            return ((ColumnTupleFilter)columnContainerFilter).getColumn();
+        else if (columnContainerFilter instanceof FunctionTupleFilter)
+            return ((FunctionTupleFilter)columnContainerFilter).getColumn();
+
+        throw new UnsupportedOperationException("Wrong type TupleFilter in FunctionTupleFilter.");
+    }
+
+    public Object invokeFunction(Object input) throws InvocationTargetException, IllegalAccessException {
+        if (columnContainerFilter instanceof ColumnTupleFilter)
+            methodParams.set(colPosition, input);
+        else if (columnContainerFilter instanceof FunctionTupleFilter)
+            methodParams.set(colPosition, ((FunctionTupleFilter) columnContainerFilter).invokeFunction(input));
+        return method.invoke(null, (Object[])(methodParams.toArray()));
+    }
+
+    public boolean isValid() {
+        return isValid && method != null && methodParams.size() == children.size();
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addChild(TupleFilter child) {
+        if (child instanceof ColumnTupleFilter || child instanceof FunctionTupleFilter) {
+            columnContainerFilter = child;
+            colPosition = methodParams.size();
+            methodParams.add(null);
+        } else if (child instanceof ConstantTupleFilter) {
+            String constVal = child.getValues().iterator().next();
+            try {
+                Class clazz = Primitives.wrap(method.getParameterTypes()[methodParams.size()]);
+                if (!Primitives.isWrapperType(clazz))
+                    methodParams.add(constVal);
+                else
+                    methodParams.add(clazz.cast(clazz.getDeclaredMethod("valueOf", String.class).invoke(null, constVal)));
+            } catch (Exception e) {
+                logger.debug(e.getMessage());
+                isValid = false;
+            }
+        }
+        super.addChild(child);
+    }
+
+    @Override
+    public boolean isEvaluable() {
+        return false;
+    }
+
+    @Override
+    public boolean evaluate(ITuple tuple) {
+        throw new UnsupportedOperationException("Function filter cannot be evaluated immediately");
+    }
+
+    @Override
+    public Collection<String> getValues() {
+        throw new UnsupportedOperationException("Function filter cannot be evaluated immediately");
+    }
+
+    @Override
+    public String toString(){
+        StringBuilder sb = new StringBuilder();
+        sb.append(sqlOperator.getName());
+        sb.append("(");
+        for (int i = 0; i < methodParams.size(); i++) {
+            if (colPosition == i) {
+                sb.append(columnContainerFilter);
+            } else {
+                sb.append(methodParams.get(i));
+            }
+            if (i < methodParams.size() - 1)
+                sb.append(",");
+        }
+        sb.append(")");
+        return sb.toString();
+    }
+
+    @Override
+    public byte[] serialize() {
+        throw new UnsupportedOperationException("Method serialize() is not supported for FunctionTupleFilter.");
+    }
+
+    @Override
+    public void deserialize(byte[] bytes) {
+        throw new UnsupportedOperationException("Method deserialize(byte[] bytes) is not supported for FunctionTupleFilter.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/metadata/src/main/java/org/apache/kylin/metadata/filter/ITupleFilterTranslator.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/ITupleFilterTranslator.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/ITupleFilterTranslator.java
new file mode 100644
index 0000000..7f1aed6
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/ITupleFilterTranslator.java
@@ -0,0 +1,26 @@
+/*
+ * 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.metadata.filter;
+
+/**
+ * Created by dongli on 11/11/15.
+ */
+public interface ITupleFilterTranslator {
+    TupleFilter translate(TupleFilter tupleFilter);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
index 66dc0db..25cae20 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
@@ -37,7 +37,7 @@ import com.google.common.collect.Maps;
 public abstract class TupleFilter {
 
     public enum FilterOperatorEnum {
-        EQ(1), NEQ(2), GT(3), LT(4), GTE(5), LTE(6), ISNULL(7), ISNOTNULL(8), IN(9), NOTIN(10), AND(20), OR(21), NOT(22), COLUMN(30), CONSTANT(31), DYNAMIC(32), EXTRACT(33), CASE(34);
+        EQ(1), NEQ(2), GT(3), LT(4), GTE(5), LTE(6), ISNULL(7), ISNOTNULL(8), IN(9), NOTIN(10), AND(20), OR(21), NOT(22), COLUMN(30), CONSTANT(31), DYNAMIC(32), EXTRACT(33), CASE(34), FUNCTION(35);
 
         private final int value;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/metadata/src/main/java/org/apache/kylin/metadata/filter/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/util/BuiltInMethod.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/util/BuiltInMethod.java
new file mode 100644
index 0000000..1f15c9c
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/util/BuiltInMethod.java
@@ -0,0 +1,56 @@
+/*
+ * 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.metadata.filter.util;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.runtime.SqlFunctions;
+
+import java.lang.reflect.Method;
+
+/**
+ * Created by dongli on 11/13/15.
+ */
+public enum BuiltInMethod {
+    UPPER(SqlFunctions.class, "upper", String.class),
+    LOWER(SqlFunctions.class, "lower", String.class),
+    SUBSTRING(SqlFunctions.class, "substring", String.class, int.class, int.class),
+    CHAR_LENGTH(SqlFunctions.class, "charLength", String.class),
+    LIKE(SqlFunctions.class, "like", String.class, String.class),
+    INITCAP(SqlFunctions.class, "initcap", String.class);
+    public final Method method;
+
+    public static final ImmutableMap<String, BuiltInMethod> MAP;
+
+    static {
+        final ImmutableMap.Builder<String, BuiltInMethod> builder =
+                ImmutableMap.builder();
+        for (BuiltInMethod value : BuiltInMethod.values()) {
+            if (value.method != null) {
+                builder.put(value.name(), value);
+            }
+        }
+        MAP = builder.build();
+    }
+
+    BuiltInMethod(Class clazz, String methodName, Class... argumentTypes) {
+        this.method = Types.lookupMethod(clazz, methodName, argumentTypes);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
index cfd5e3b..583af9a 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
@@ -18,50 +18,25 @@
 
 package org.apache.kylin.query.relnode;
 
-import java.util.Calendar;
-import java.util.GregorianCalendar;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
 import org.apache.calcite.adapter.enumerable.EnumerableCalc;
 import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.adapter.enumerable.EnumerableRel;
 import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTrait;
-import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.*;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexDynamicParam;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexLocalRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.calcite.rex.RexProgramBuilder;
-import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.rex.*;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.util.NlsString;
-import org.apache.kylin.metadata.filter.CaseTupleFilter;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.DynamicTupleFilter;
-import org.apache.kylin.metadata.filter.ExtractTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.*;
 import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
+import java.util.*;
 
 /**
  */
@@ -127,9 +102,13 @@ public class OLAPFilterRel extends Filter implements OLAPRel {
                 if (op.getName().equalsIgnoreCase("extract_date")) {
                     filter = new ExtractTupleFilter(FilterOperatorEnum.EXTRACT);
                 } else {
-                    throw new UnsupportedOperationException(op.getName());
+                    filter = new FunctionTupleFilter(op);
                 }
                 break;
+            case LIKE:
+            case OTHER_FUNCTION:
+                filter = new FunctionTupleFilter(op);
+                break;
             default:
                 throw new UnsupportedOperationException(op.getName());
             }
@@ -286,6 +265,8 @@ public class OLAPFilterRel extends Filter implements OLAPRel {
     }
 
     private void collectColumnsRecursively(TupleFilter filter, Set<TblColRef> collector) {
+        if (filter == null) return;
+
         if (filter instanceof ColumnTupleFilter) {
             collector.add(((ColumnTupleFilter) filter).getColumn());
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/query/src/test/resources/query/sql/query85.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query85.sql b/query/src/test/resources/query/sql/query85.sql
new file mode 100644
index 0000000..4e80d59
--- /dev/null
+++ b/query/src/test/resources/query/sql/query85.sql
@@ -0,0 +1,22 @@
+--
+-- 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.
+--
+
+select upper(lstg_format_name) as lstg_format_name, count(*) as cnt from test_kylin_fact
+where lower(lstg_format_name)='abin' and substring(lstg_format_name,1,3) in ('ABI') and upper(lstg_format_name) > 'AAAA' and
+lower(lstg_format_name) like '%b%' and char_length(lstg_format_name) < 10 and char_length(lstg_format_name) > 3 and lstg_format_name||'a'='ABINa'
+group by lstg_format_name
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
index 8eb7bcb..fdb8986 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
@@ -267,6 +267,8 @@ public class CubeStorageEngine implements IStorageEngine {
     }
 
     private void collectColumnsRecursively(TupleFilter filter, Set<TblColRef> collector) {
+        if (filter == null) return;
+
         if (filter instanceof ColumnTupleFilter) {
             collectColumns(((ColumnTupleFilter) filter).getColumn(), collector);
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java
index 65fddd2..b3e2d31 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java
@@ -26,11 +26,7 @@ import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.kv.RowKeyColumnIO;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.ISegment;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilterSerializer;
+import org.apache.kylin.metadata.filter.*;
 import org.apache.kylin.metadata.filter.TupleFilterSerializer.Decorator;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
@@ -61,6 +57,9 @@ public class CoprocessorFilter {
             if (filter == null)
                 return null;
 
+            ITupleFilterTranslator translator = new CoprocessorTupleFilterTranslator(columnIO);
+            filter = translator.translate(filter);
+
             // un-evaluatable filter is replaced with TRUE
             if (!filter.isEvaluable()) {
                 TupleFilter.collectColumns(filter, unstrictlyFilteredColumns);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bed15abb/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorTupleFilterTranslator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorTupleFilterTranslator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorTupleFilterTranslator.java
new file mode 100644
index 0000000..aae945d
--- /dev/null
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorTupleFilterTranslator.java
@@ -0,0 +1,166 @@
+/*
+ * 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.storage.hbase.coprocessor;
+
+import com.google.common.primitives.Primitives;
+import org.apache.kylin.cube.kv.RowKeyColumnIO;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.metadata.filter.*;
+import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.InvocationTargetException;
+
+/**
+ * Created by dongli on 11/11/15.
+ */
+public class CoprocessorTupleFilterTranslator implements ITupleFilterTranslator {
+    public static final Logger logger = LoggerFactory.getLogger(CoprocessorTupleFilterTranslator.class);
+
+    private RowKeyColumnIO rowKeyColumnIO;
+
+    public CoprocessorTupleFilterTranslator(RowKeyColumnIO rowKeyColumnIO) {
+        this.rowKeyColumnIO = rowKeyColumnIO;
+    }
+
+    @Override
+    public TupleFilter translate(TupleFilter tupleFilter) {
+        TupleFilter translated = null;
+        if (tupleFilter instanceof CompareTupleFilter) {
+            logger.info("Translation to IN clause: " + tupleFilter);
+            translated = translateCompareTupleFilter((CompareTupleFilter) tupleFilter);
+            logger.info(translated == null ? "Failed, will use Calcite to handle computed comparison." : "Succeed: " + translated);
+        } else if (tupleFilter instanceof FunctionTupleFilter) {
+            logger.info("Translation to IN clause: " + tupleFilter);
+            translated = translateFunctionTupleFilter((FunctionTupleFilter) tupleFilter);
+            logger.info(translated == null ? "Failed, will use Calcite to handle computed column." : "Succeed: " + translated);
+        }
+        return translated == null ? tupleFilter : translated;
+    }
+
+    private TupleFilter translateFunctionTupleFilter(FunctionTupleFilter functionTupleFilter) {
+        if (!functionTupleFilter.isValid())
+            return null;
+
+        TblColRef columnRef = functionTupleFilter.getColumn();
+        Dictionary<String> dict = rowKeyColumnIO.getDictionary(columnRef);
+        if (dict == null)
+            return null;
+
+        CompareTupleFilter translated = new CompareTupleFilter(FilterOperatorEnum.IN);
+        translated.addChild(new ColumnTupleFilter(columnRef));
+
+        try {
+            for (int i = dict.getMinId(); i <= dict.getMaxId(); i++) {
+                String dictVal = dict.getValueFromId(i);
+                if ((Boolean)functionTupleFilter.invokeFunction(dictVal)) {
+                    translated.addChild(new ConstantTupleFilter(dictVal));
+                }
+            }
+        } catch (IllegalAccessException e) {
+            logger.debug(e.getMessage());
+            return null;
+        } catch (InvocationTargetException e) {
+            logger.debug(e.getMessage());
+            return null;
+        } catch (IllegalArgumentException e) {
+            logger.debug(e.getMessage());
+            return null;
+        }
+        return translated;
+    }
+
+    @SuppressWarnings("unchecked")
+    private TupleFilter translateCompareTupleFilter(CompareTupleFilter compTupleFilter){
+        if (compTupleFilter.getFunctionTupleFilter() == null)
+            return null;
+
+        FunctionTupleFilter functionTupleFilter = compTupleFilter.getFunctionTupleFilter();
+        if (!functionTupleFilter.isValid())
+            return null;
+
+        TblColRef columnRef = functionTupleFilter.getColumn();
+        Dictionary<String> dict = rowKeyColumnIO.getDictionary(columnRef);
+        if (dict == null)
+            return null;
+
+        CompareTupleFilter translated = new CompareTupleFilter(FilterOperatorEnum.IN);
+        translated.addChild(new ColumnTupleFilter(columnRef));
+
+        try {
+            for (int i = dict.getMinId(); i <= dict.getMaxId(); i++) {
+                String dictVal = dict.getValueFromId(i);
+                Object computedVal = functionTupleFilter.invokeFunction(dictVal);
+                Class clazz = Primitives.wrap(computedVal.getClass());
+                Object targetVal = compTupleFilter.getFirstValue();
+                if (Primitives.isWrapperType(clazz))
+                    targetVal = clazz.cast(clazz.getDeclaredMethod("valueOf", String.class).invoke(null, compTupleFilter.getFirstValue()));
+
+                int comp = ((Comparable)computedVal).compareTo(targetVal);
+                boolean compResult = false;
+                switch (compTupleFilter.getOperator()) {
+                    case EQ:
+                        compResult = comp == 0;
+                        break;
+                    case NEQ:
+                        compResult = comp != 0;
+                        break;
+                    case LT:
+                        compResult = comp < 0;
+                        break;
+                    case LTE:
+                        compResult = comp <= 0;
+                        break;
+                    case GT:
+                        compResult = comp > 0;
+                        break;
+                    case GTE:
+                        compResult = comp >= 0;
+                        break;
+                    case IN:
+                        compResult = compTupleFilter.getValues().contains(computedVal.toString());
+                        break;
+                    case NOTIN:
+                        compResult = !compTupleFilter.getValues().contains(computedVal.toString());
+                        break;
+                    default:
+                        break;
+                }
+                if (compResult) {
+                    translated.addChild(new ConstantTupleFilter(dictVal));
+                }
+            }
+        } catch (IllegalAccessException e) {
+            logger.debug(e.getMessage());
+            return null;
+        } catch (InvocationTargetException e) {
+            logger.debug(e.getMessage());
+            return null;
+        } catch (IllegalArgumentException e) {
+            logger.debug(e.getMessage());
+            return null;
+        } catch (NoSuchMethodException e) {
+            logger.debug(e.getMessage());
+            return null;
+        }
+        return translated;
+    }
+}


[06/30] incubator-kylin git commit: update source-assembly.xml to add zip format

Posted by li...@apache.org.
update source-assembly.xml to add zip format

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: d730c9a94f9fefa62e53e9a4d1b5bfe25f466add
Parents: 2feb85d
Author: shaofengshi <sh...@apache.org>
Authored: Wed Nov 4 14:34:47 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 11:16:22 2015 +0800

----------------------------------------------------------------------
 src/main/config/assemblies/source-assembly.xml | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d730c9a9/src/main/config/assemblies/source-assembly.xml
----------------------------------------------------------------------
diff --git a/src/main/config/assemblies/source-assembly.xml b/src/main/config/assemblies/source-assembly.xml
index 31c783c..9c28851 100644
--- a/src/main/config/assemblies/source-assembly.xml
+++ b/src/main/config/assemblies/source-assembly.xml
@@ -26,7 +26,7 @@ limitations under the License.
         <!-- main project directory structure -->
         <fileSet>
             <directory>.</directory>
-            <outputDirectory>/</outputDirectory>
+            <outputDirectory>.</outputDirectory>
             <useDefaultExcludes>true</useDefaultExcludes>
             <excludes>
                 <!-- build output -->
@@ -104,14 +104,14 @@ limitations under the License.
         <fileSet>
             <directory>${project.build.directory}/maven-shared-archive-resources/META-INF
             </directory>
-            <outputDirectory>/</outputDirectory>
+            <outputDirectory>.</outputDirectory>
         </fileSet>
         <fileSet>
             <directory>${project.build.directory}</directory>
             <includes>
                 <include>git.properties</include>
             </includes>
-            <outputDirectory>/</outputDirectory>
+            <outputDirectory>.</outputDirectory>
         </fileSet>
     </fileSets>
 </assembly>
\ No newline at end of file


[04/30] incubator-kylin git commit: KYLIN-999 fix for google fonts license

Posted by li...@apache.org.
KYLIN-999 fix for google fonts license

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: efd234ec15e74b7247ee3a0c4ef8bcf171b02d0f
Parents: 4c889cd
Author: Luke Han <lu...@apache.org>
Authored: Wed Nov 4 10:16:23 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 11:16:21 2015 +0800

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 webapp/app/css/AdminLTE-fonts.css               |  23 -------------------
 webapp/app/css/AdminLTE.css                     |   4 ++--
 ...1aHBYDBqgeIAH2zlPytCVloACqnDHJwh-em2k8.woff2 | Bin 11400 -> 0 bytes
 ...t5NIMarvNW-hkYWvT3rGVtsTkPsbDajuO5ueQw.woff2 | Bin 11044 -> 0 bytes
 5 files changed, 3 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/efd234ec/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index b016aa2..9a27b8a 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,6 +7,7 @@ tmp/
 *.bak
 *.swp
 *~.nib
+*.versionsBackup
 local.properties
 release.properties
 .loadpath

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/efd234ec/webapp/app/css/AdminLTE-fonts.css
----------------------------------------------------------------------
diff --git a/webapp/app/css/AdminLTE-fonts.css b/webapp/app/css/AdminLTE-fonts.css
deleted file mode 100644
index a8e0244..0000000
--- a/webapp/app/css/AdminLTE-fonts.css
+++ /dev/null
@@ -1,23 +0,0 @@
-/*!
- *   AdminLTE v1.2
- *   Author: AlmsaeedStudio.com
- *   License: Open source - MIT
- *           Please visit http://opensource.org/licenses/MIT for more information
-!*/
-
-/* latin */
-@font-face {
-    font-family: 'Source Sans Pro';
-    font-style: normal;
-    font-weight: 400;
-    src: local('Source Sans Pro'), local('SourceSansPro-Regular'), url(../fonts/ODelI1aHBYDBqgeIAH2zlPytCVloACqnDHJwh-em2k8.woff2) format('woff2');
-    unicode-range: U+0000-00FF, U+0131, U+0152-0153, U+02C6, U+02DA, U+02DC, U+2000-206F, U+2074, U+20AC, U+2212, U+2215, U+E0FF, U+EFFD, U+F000;
-}
-/* latin */
-@font-face {
-    font-family: 'Source Sans Pro';
-    font-style: normal;
-    font-weight: 700;
-    src: local('Source Sans Pro Bold'), local('SourceSansPro-Bold'), url(../fonts/toadOcfmlt9b38dHJxOBGKUt5Ot5NIMarvNW-hkYWvT3rGVtsTkPsbDajuO5ueQw.woff2) format('woff2');
-    unicode-range: U+0000-00FF, U+0131, U+0152-0153, U+02C6, U+02DA, U+02DC, U+2000-206F, U+2074, U+20AC, U+2212, U+2215, U+E0FF, U+EFFD, U+F000;
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/efd234ec/webapp/app/css/AdminLTE.css
----------------------------------------------------------------------
diff --git a/webapp/app/css/AdminLTE.css b/webapp/app/css/AdminLTE.css
index e91795e..d86593d 100644
--- a/webapp/app/css/AdminLTE.css
+++ b/webapp/app/css/AdminLTE.css
@@ -21,7 +21,7 @@ body {
 body {
   -webkit-font-smoothing: antialiased;
   -moz-osx-font-smoothing: grayscale;
-  font-family: 'Source Sans Pro', 'Helvetica Neue', Helvetica, Arial, sans-serif;
+  font-family: 'Helvetica Neue', Helvetica, Arial, sans-serif;
   font-weight: 400;
   overflow-x: hidden;
   overflow-y: auto;
@@ -151,7 +151,7 @@ h6,
 .h4,
 .h5,
 .h6 {
-  font-family: 'Source Sans Pro', sans-serif;
+  font-family: sans-serif;
 }
 /* General Links */
 a {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/efd234ec/webapp/app/fonts/ODelI1aHBYDBqgeIAH2zlPytCVloACqnDHJwh-em2k8.woff2
----------------------------------------------------------------------
diff --git a/webapp/app/fonts/ODelI1aHBYDBqgeIAH2zlPytCVloACqnDHJwh-em2k8.woff2 b/webapp/app/fonts/ODelI1aHBYDBqgeIAH2zlPytCVloACqnDHJwh-em2k8.woff2
deleted file mode 100644
index da49c62..0000000
Binary files a/webapp/app/fonts/ODelI1aHBYDBqgeIAH2zlPytCVloACqnDHJwh-em2k8.woff2 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/efd234ec/webapp/app/fonts/toadOcfmlt9b38dHJxOBGKUt5Ot5NIMarvNW-hkYWvT3rGVtsTkPsbDajuO5ueQw.woff2
----------------------------------------------------------------------
diff --git a/webapp/app/fonts/toadOcfmlt9b38dHJxOBGKUt5Ot5NIMarvNW-hkYWvT3rGVtsTkPsbDajuO5ueQw.woff2 b/webapp/app/fonts/toadOcfmlt9b38dHJxOBGKUt5Ot5NIMarvNW-hkYWvT3rGVtsTkPsbDajuO5ueQw.woff2
deleted file mode 100644
index 2486c1d..0000000
Binary files a/webapp/app/fonts/toadOcfmlt9b38dHJxOBGKUt5Ot5NIMarvNW-hkYWvT3rGVtsTkPsbDajuO5ueQw.woff2 and /dev/null differ


[13/30] incubator-kylin git commit: Merge branch '1.x-staging' of https://git-wip-us.apache.org/repos/asf/incubator-kylin into 1.x-staging

Posted by li...@apache.org.
Merge branch '1.x-staging' of https://git-wip-us.apache.org/repos/asf/incubator-kylin into 1.x-staging


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 882ca61bccb3bd28c306ed6b393216ee4c4d78b7
Parents: c5ada29 1669ed3
Author: lukehan <lu...@apache.org>
Authored: Thu Nov 5 22:00:21 2015 +0800
Committer: lukehan <lu...@apache.org>
Committed: Thu Nov 5 22:00:21 2015 +0800

----------------------------------------------------------------------

----------------------------------------------------------------------



[21/30] incubator-kylin git commit: KYLIN-1149 When yarn return an incomplete job tracking URL, Kylin will fail to get job status

Posted by li...@apache.org.
KYLIN-1149 When yarn return an incomplete job tracking URL, Kylin will fail to get job status

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

Branch: refs/heads/1.x-HBase1.1.3
Commit: db7f8be0fe2929edd72fff0ea4b0fcfe937816e5
Parents: 2432e2d
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 16 16:29:16 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Nov 16 16:29:16 2015 +0800

----------------------------------------------------------------------
 .../kylin/job/tools/HadoopStatusGetter.java     | 21 ++++++++++++++++++++
 1 file changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/db7f8be0/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusGetter.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusGetter.java b/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusGetter.java
index 398d15a..0cd4d43 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusGetter.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusGetter.java
@@ -19,6 +19,7 @@
 package org.apache.kylin.job.tools;
 
 import java.io.IOException;
+import java.net.MalformedURLException;
 
 import org.apache.commons.httpclient.Header;
 import org.apache.commons.httpclient.HttpClient;
@@ -26,6 +27,7 @@ import org.apache.commons.httpclient.HttpMethod;
 import org.apache.commons.httpclient.methods.GetMethod;
 import org.apache.commons.httpclient.protocol.Protocol;
 import org.apache.commons.httpclient.protocol.ProtocolSocketFactory;
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
@@ -84,6 +86,11 @@ public class HadoopStatusGetter {
                     int cut = s.indexOf("url=");
                     if (cut >= 0) {
                         redirect = s.substring(cut + 4);
+                        
+                        if (isValidURL(redirect) == false) {
+                            log.info("Get invalid redirect url, skip it: " + redirect);
+                            continue;
+                        }
                     }
                 }
 
@@ -112,4 +119,18 @@ public class HadoopStatusGetter {
         }
     }
 
+    private static boolean isValidURL(String value) {
+        if (StringUtils.isNotEmpty(value)) {
+            java.net.URL url;
+            try {
+                url = new java.net.URL(value);
+            } catch (MalformedURLException var5) {
+                return false;
+            }
+
+            return StringUtils.isNotEmpty(url.getProtocol()) && StringUtils.isNotEmpty(url.getHost());
+        }
+
+        return false;
+    }
 }


[17/30] incubator-kylin git commit: KYLIN-1140 cube couldn’t be saved if the cube name != cubeDesc name

Posted by li...@apache.org.
KYLIN-1140 cube couldn’t be saved if the cube name != cubeDesc name

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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 6ad0786c6a22d9013510bfd20d5737f9c8db731a
Parents: 9e8a717
Author: shaofengshi <sh...@apache.org>
Authored: Thu Nov 12 17:12:42 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 12 17:12:42 2015 +0800

----------------------------------------------------------------------
 examples/sample_cube/metadata/cube/kylin_sales_cube.json          | 3 ++-
 .../java/org/apache/kylin/rest/controller/CubeController.java     | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6ad0786c/examples/sample_cube/metadata/cube/kylin_sales_cube.json
----------------------------------------------------------------------
diff --git a/examples/sample_cube/metadata/cube/kylin_sales_cube.json b/examples/sample_cube/metadata/cube/kylin_sales_cube.json
index ef1fdf6..5c36aa1 100644
--- a/examples/sample_cube/metadata/cube/kylin_sales_cube.json
+++ b/examples/sample_cube/metadata/cube/kylin_sales_cube.json
@@ -4,7 +4,8 @@
   "name" : "kylin_sales_cube",
   "owner" : null,
   "version" : null,
-  "descriptor" : "kylin_sales_cube_desc",
+  "descriptor" : "kylin_sales_cube_desc",  
+  "status" : "DISABLED",
   "segments" : [ ],
   "auto_merge_time_ranges" : [604800000, 2419200000],
   "retention_range": 63072000000,

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6ad0786c/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index 8193f5e..2f87c4f 100644
--- a/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -343,7 +343,7 @@ public class CubeController extends BasicController {
             return cubeRequest;
         }
 
-        final String cubeName = desc.getName();
+        final String cubeName = cubeRequest.getCubeName();
         if (StringUtils.isEmpty(cubeName)) {
             return errorRequest(cubeRequest, "Missing cubeName");
         }


[26/30] incubator-kylin git commit: KYLIN-993 Code review, minor formattings

Posted by li...@apache.org.
KYLIN-993 Code review, minor formattings


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: 3285a42ab4d376cda314b06237f5f78de54016b7
Parents: bed15ab
Author: Li, Yang <ya...@ebay.com>
Authored: Tue Nov 17 17:47:13 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Nov 17 17:47:13 2015 +0800

----------------------------------------------------------------------
 .../metadata/filter/CompareTupleFilter.java     | 22 +++--
 .../metadata/filter/FunctionTupleFilter.java    | 13 ++-
 .../metadata/filter/util/BuiltInMethod.java     |  8 +-
 .../kylin/storage/hbase/CubeStorageEngine.java  |  3 +-
 .../storage/hbase/FuzzyValueCombination.java    |  2 -
 .../CoprocessorTupleFilterTranslator.java       | 93 +++++++++-----------
 6 files changed, 67 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3285a42a/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
index 6754ff7..73fd0fb 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
@@ -19,9 +19,12 @@
 package org.apache.kylin.metadata.filter;
 
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
 
-import org.apache.calcite.sql.SqlFunction;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
@@ -31,12 +34,15 @@ import org.apache.kylin.metadata.tuple.ITuple;
  */
 public class CompareTupleFilter extends TupleFilter {
 
+    // operand 1 is either a column or a function
     private TblColRef column;
+    private FunctionTupleFilter function;
+    
+    // operand 2 is constants
     private Collection<String> conditionValues;
     private String firstCondValue;
     private Map<String, String> dynamicVariables;
     private String nullString;
-    private FunctionTupleFilter functionTupleFilter;
 
     public CompareTupleFilter(FilterOperatorEnum op) {
         super(new ArrayList<TupleFilter>(2), op);
@@ -80,7 +86,7 @@ public class CompareTupleFilter extends TupleFilter {
             DynamicTupleFilter dynamicFilter = (DynamicTupleFilter) child;
             this.dynamicVariables.put(dynamicFilter.getVariableName(), null);
         } else if (child instanceof FunctionTupleFilter) {
-            this.functionTupleFilter = (FunctionTupleFilter)child;
+            this.function = (FunctionTupleFilter)child;
         }
         //TODO
         //        else if (child instanceof ExtractTupleFilter) {
@@ -105,8 +111,8 @@ public class CompareTupleFilter extends TupleFilter {
         return column;
     }
 
-    public FunctionTupleFilter getFunctionTupleFilter() {
-        return functionTupleFilter;
+    public FunctionTupleFilter getFunction() {
+        return function;
     }
 
     public Map<String, String> getVariables() {
@@ -141,7 +147,7 @@ public class CompareTupleFilter extends TupleFilter {
 
     @Override
     public String toString() {
-        return "CompareFilter [" + (functionTupleFilter == null ? column : functionTupleFilter) + " " + operator + " " + conditionValues + ", children=" + children + "]";
+        return "CompareFilter [" + (function == null ? column : function) + " " + operator + " " + conditionValues + ", children=" + children + "]";
     }
 
     // TODO requires generalize, currently only evaluates COLUMN {op} CONST
@@ -213,7 +219,7 @@ public class CompareTupleFilter extends TupleFilter {
 
     @Override
     public boolean isEvaluable() {
-        return (functionTupleFilter != null || column != null) && !conditionValues.isEmpty();
+        return (function != null || column != null) && !conditionValues.isEmpty();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3285a42a/metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
index 62ab42f..731e5ab 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
@@ -47,7 +47,7 @@ public class FunctionTupleFilter extends TupleFilter {
     private boolean isValid = false;
 
     public FunctionTupleFilter(SqlOperator sqlOperator) {
-        super(Lists.<TupleFilter>newArrayList(), FilterOperatorEnum.FUNCTION);
+        super(Lists.<TupleFilter> newArrayList(), FilterOperatorEnum.FUNCTION);
         this.methodParams = Lists.newArrayList();
         this.sqlOperator = sqlOperator;
 
@@ -67,9 +67,9 @@ public class FunctionTupleFilter extends TupleFilter {
             return null;
 
         if (columnContainerFilter instanceof ColumnTupleFilter)
-            return ((ColumnTupleFilter)columnContainerFilter).getColumn();
+            return ((ColumnTupleFilter) columnContainerFilter).getColumn();
         else if (columnContainerFilter instanceof FunctionTupleFilter)
-            return ((FunctionTupleFilter)columnContainerFilter).getColumn();
+            return ((FunctionTupleFilter) columnContainerFilter).getColumn();
 
         throw new UnsupportedOperationException("Wrong type TupleFilter in FunctionTupleFilter.");
     }
@@ -79,7 +79,7 @@ public class FunctionTupleFilter extends TupleFilter {
             methodParams.set(colPosition, input);
         else if (columnContainerFilter instanceof FunctionTupleFilter)
             methodParams.set(colPosition, ((FunctionTupleFilter) columnContainerFilter).invokeFunction(input));
-        return method.invoke(null, (Object[])(methodParams.toArray()));
+        return method.invoke(null, (Object[]) (methodParams.toArray()));
     }
 
     public boolean isValid() {
@@ -87,7 +87,6 @@ public class FunctionTupleFilter extends TupleFilter {
     }
 
     @Override
-    @SuppressWarnings("unchecked")
     public void addChild(TupleFilter child) {
         if (child instanceof ColumnTupleFilter || child instanceof FunctionTupleFilter) {
             columnContainerFilter = child;
@@ -96,7 +95,7 @@ public class FunctionTupleFilter extends TupleFilter {
         } else if (child instanceof ConstantTupleFilter) {
             String constVal = child.getValues().iterator().next();
             try {
-                Class clazz = Primitives.wrap(method.getParameterTypes()[methodParams.size()]);
+                Class<?> clazz = Primitives.wrap(method.getParameterTypes()[methodParams.size()]);
                 if (!Primitives.isWrapperType(clazz))
                     methodParams.add(constVal);
                 else
@@ -125,7 +124,7 @@ public class FunctionTupleFilter extends TupleFilter {
     }
 
     @Override
-    public String toString(){
+    public String toString() {
         StringBuilder sb = new StringBuilder();
         sb.append(sqlOperator.getName());
         sb.append("(");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3285a42a/metadata/src/main/java/org/apache/kylin/metadata/filter/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/util/BuiltInMethod.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/util/BuiltInMethod.java
index 1f15c9c..3a36385 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/filter/util/BuiltInMethod.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/util/BuiltInMethod.java
@@ -18,12 +18,12 @@
 
 package org.apache.kylin.metadata.filter.util;
 
-import com.google.common.collect.ImmutableMap;
-import org.apache.calcite.avatica.util.DateTimeUtils;
+import java.lang.reflect.Method;
+
 import org.apache.calcite.linq4j.tree.Types;
 import org.apache.calcite.runtime.SqlFunctions;
 
-import java.lang.reflect.Method;
+import com.google.common.collect.ImmutableMap;
 
 /**
  * Created by dongli on 11/13/15.
@@ -50,7 +50,7 @@ public enum BuiltInMethod {
         MAP = builder.build();
     }
 
-    BuiltInMethod(Class clazz, String methodName, Class... argumentTypes) {
+    BuiltInMethod(Class<?> clazz, String methodName, Class<?>... argumentTypes) {
         this.method = Types.lookupMethod(clazz, methodName, argumentTypes);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3285a42a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
index fdb8986..ed12781 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
@@ -267,7 +267,8 @@ public class CubeStorageEngine implements IStorageEngine {
     }
 
     private void collectColumnsRecursively(TupleFilter filter, Set<TblColRef> collector) {
-        if (filter == null) return;
+        if (filter == null)
+            return;
 
         if (filter instanceof ColumnTupleFilter) {
             collectColumns(((ColumnTupleFilter) filter).getColumn(), collector);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3285a42a/storage/src/main/java/org/apache/kylin/storage/hbase/FuzzyValueCombination.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/FuzzyValueCombination.java b/storage/src/main/java/org/apache/kylin/storage/hbase/FuzzyValueCombination.java
index 616a232..ce14b2b 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/FuzzyValueCombination.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/FuzzyValueCombination.java
@@ -18,9 +18,7 @@
 
 package org.apache.kylin.storage.hbase;
 
-import java.util.Arrays;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3285a42a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorTupleFilterTranslator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorTupleFilterTranslator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorTupleFilterTranslator.java
index aae945d..6affc18 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorTupleFilterTranslator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorTupleFilterTranslator.java
@@ -18,16 +18,20 @@
 
 package org.apache.kylin.storage.hbase.coprocessor;
 
-import com.google.common.primitives.Primitives;
 import org.apache.kylin.cube.kv.RowKeyColumnIO;
 import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.filter.*;
+import org.apache.kylin.metadata.filter.ColumnTupleFilter;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.ConstantTupleFilter;
+import org.apache.kylin.metadata.filter.FunctionTupleFilter;
+import org.apache.kylin.metadata.filter.ITupleFilterTranslator;
+import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.lang.reflect.InvocationTargetException;
+import com.google.common.primitives.Primitives;
 
 /**
  * Created by dongli on 11/11/15.
@@ -71,17 +75,11 @@ public class CoprocessorTupleFilterTranslator implements ITupleFilterTranslator
         try {
             for (int i = dict.getMinId(); i <= dict.getMaxId(); i++) {
                 String dictVal = dict.getValueFromId(i);
-                if ((Boolean)functionTupleFilter.invokeFunction(dictVal)) {
+                if ((Boolean) functionTupleFilter.invokeFunction(dictVal)) {
                     translated.addChild(new ConstantTupleFilter(dictVal));
                 }
             }
-        } catch (IllegalAccessException e) {
-            logger.debug(e.getMessage());
-            return null;
-        } catch (InvocationTargetException e) {
-            logger.debug(e.getMessage());
-            return null;
-        } catch (IllegalArgumentException e) {
+        } catch (Exception e) {
             logger.debug(e.getMessage());
             return null;
         }
@@ -89,11 +87,11 @@ public class CoprocessorTupleFilterTranslator implements ITupleFilterTranslator
     }
 
     @SuppressWarnings("unchecked")
-    private TupleFilter translateCompareTupleFilter(CompareTupleFilter compTupleFilter){
-        if (compTupleFilter.getFunctionTupleFilter() == null)
+    private TupleFilter translateCompareTupleFilter(CompareTupleFilter compTupleFilter) {
+        if (compTupleFilter.getFunction() == null)
             return null;
 
-        FunctionTupleFilter functionTupleFilter = compTupleFilter.getFunctionTupleFilter();
+        FunctionTupleFilter functionTupleFilter = compTupleFilter.getFunction();
         if (!functionTupleFilter.isValid())
             return null;
 
@@ -109,55 +107,46 @@ public class CoprocessorTupleFilterTranslator implements ITupleFilterTranslator
             for (int i = dict.getMinId(); i <= dict.getMaxId(); i++) {
                 String dictVal = dict.getValueFromId(i);
                 Object computedVal = functionTupleFilter.invokeFunction(dictVal);
-                Class clazz = Primitives.wrap(computedVal.getClass());
+                Class<?> clazz = Primitives.wrap(computedVal.getClass());
                 Object targetVal = compTupleFilter.getFirstValue();
                 if (Primitives.isWrapperType(clazz))
                     targetVal = clazz.cast(clazz.getDeclaredMethod("valueOf", String.class).invoke(null, compTupleFilter.getFirstValue()));
 
-                int comp = ((Comparable)computedVal).compareTo(targetVal);
+                int comp = ((Comparable<Object>) computedVal).compareTo(targetVal);
                 boolean compResult = false;
                 switch (compTupleFilter.getOperator()) {
-                    case EQ:
-                        compResult = comp == 0;
-                        break;
-                    case NEQ:
-                        compResult = comp != 0;
-                        break;
-                    case LT:
-                        compResult = comp < 0;
-                        break;
-                    case LTE:
-                        compResult = comp <= 0;
-                        break;
-                    case GT:
-                        compResult = comp > 0;
-                        break;
-                    case GTE:
-                        compResult = comp >= 0;
-                        break;
-                    case IN:
-                        compResult = compTupleFilter.getValues().contains(computedVal.toString());
-                        break;
-                    case NOTIN:
-                        compResult = !compTupleFilter.getValues().contains(computedVal.toString());
-                        break;
-                    default:
-                        break;
+                case EQ:
+                    compResult = comp == 0;
+                    break;
+                case NEQ:
+                    compResult = comp != 0;
+                    break;
+                case LT:
+                    compResult = comp < 0;
+                    break;
+                case LTE:
+                    compResult = comp <= 0;
+                    break;
+                case GT:
+                    compResult = comp > 0;
+                    break;
+                case GTE:
+                    compResult = comp >= 0;
+                    break;
+                case IN:
+                    compResult = compTupleFilter.getValues().contains(computedVal.toString());
+                    break;
+                case NOTIN:
+                    compResult = !compTupleFilter.getValues().contains(computedVal.toString());
+                    break;
+                default:
+                    break;
                 }
                 if (compResult) {
                     translated.addChild(new ConstantTupleFilter(dictVal));
                 }
             }
-        } catch (IllegalAccessException e) {
-            logger.debug(e.getMessage());
-            return null;
-        } catch (InvocationTargetException e) {
-            logger.debug(e.getMessage());
-            return null;
-        } catch (IllegalArgumentException e) {
-            logger.debug(e.getMessage());
-            return null;
-        } catch (NoSuchMethodException e) {
+        } catch (Exception e) {
             logger.debug(e.getMessage());
             return null;
         }


[27/30] incubator-kylin git commit: minor, fix a NPE in SnapshotTable.equals()

Posted by li...@apache.org.
minor, fix a NPE in SnapshotTable.equals()


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: ca1e695b7514599e142365fd25f05ff89c45c4c9
Parents: 3285a42
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Nov 18 10:19:41 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Nov 18 10:19:41 2015 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ca1e695b/dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java b/dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java
index 71dbab7..96a7cd6 100644
--- a/dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java
+++ b/dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java
@@ -23,6 +23,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.commons.lang.ArrayUtils;
@@ -50,7 +51,7 @@ public class SnapshotTable extends RootPersistentEntity implements ReadableTable
     @JsonProperty("useDictionary")
     private boolean useDictionary;
 
-    private ArrayList<int[]> rowIndices;
+    private List<int[]> rowIndices = Collections.emptyList();
     private Dictionary<String> dict;
 
     // default constructor for JSON serialization