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/07/22 12:50:26 UTC

[1/6] incubator-kylin git commit: KYLIN-875 Refactor core-common, remove dependency on hadoop/hbase

Repository: incubator-kylin
Updated Branches:
  refs/heads/0.8 dd5f2b2c6 -> d2456215c


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HBaseMiniclusterHelper.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HBaseMiniclusterHelper.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HBaseMiniclusterHelper.java
new file mode 100644
index 0000000..1402530
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HBaseMiniclusterHelper.java
@@ -0,0 +1,167 @@
+/*
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.storage.hbase.HBaseResourceStore;
+
+/**
+ * a helper class to start and shutdown hbase mini cluster
+ *
+ * @author shaoshi
+ */
+public class HBaseMiniclusterHelper {
+
+    public static final String SHARED_STORAGE_PREFIX = "KYLIN_";
+    public static final String CUBE_STORAGE_PREFIX = "KYLIN_";
+    public static final String II_STORAGE_PREFIX = "KYLIN_II_";
+    public static final String TEST_METADATA_TABLE = "kylin_metadata";
+
+    private static final String hbaseTarLocation = "../examples/test_case_data/minicluster/hbase-export.tar.gz";
+    private static final String iiEndpointClassName = "org.apache.kylin.storage.hbase.coprocessor.endpoint.IIEndpoint";
+
+    public static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+    private static volatile boolean clusterStarted = false;
+    private static String hbaseconnectionUrl = "";
+
+    private static final Log logger = LogFactory.getLog(HBaseMiniclusterHelper.class);
+
+    static {
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                shutdownMiniCluster();
+            }
+        });
+    }
+
+    /**
+     * Start the minicluster; Sub-classes should invoke this in BeforeClass method.
+     *
+     * @throws Exception
+     */
+    public static void startupMinicluster() throws Exception {
+
+        if (!clusterStarted) {
+            synchronized (HBaseMiniclusterHelper.class) {
+                if (!clusterStarted) {
+                    startupMiniClusterAndImportData();
+                    clusterStarted = true;
+                }
+            }
+        } else {
+            updateKylinConfigWithMinicluster();
+        }
+    }
+
+    private static void updateKylinConfigWithMinicluster() {
+
+        KylinConfig.getInstanceFromEnv().setMetadataUrl(TEST_METADATA_TABLE + "@" + hbaseconnectionUrl);
+        KylinConfig.getInstanceFromEnv().setStorageUrl(hbaseconnectionUrl);
+    }
+
+    private static void startupMiniClusterAndImportData() throws Exception {
+
+        logger.info("Going to start mini cluster.");
+
+        if (existInClassPath(iiEndpointClassName)) {
+            HBaseMiniclusterHelper.UTIL.getConfiguration().setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, iiEndpointClassName);
+        }
+
+        //https://issues.apache.org/jira/browse/HBASE-11711
+        UTIL.getConfiguration().setInt("hbase.master.info.port", -1);//avoid port clobbering
+
+        MiniHBaseCluster hbaseCluster = UTIL.startMiniCluster();
+
+        Configuration config = hbaseCluster.getConf();
+        String host = config.get(HConstants.ZOOKEEPER_QUORUM);
+        String port = config.get(HConstants.ZOOKEEPER_CLIENT_PORT);
+        String parent = config.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+
+        // see in: https://hbase.apache.org/book.html#trouble.rs.runtime.zkexpired
+        config.set("zookeeper.session.timeout", "1200000");
+        config.set("hbase.zookeeper.property.tickTime", "6000");
+        // reduce rpc retry
+        config.set(HConstants.HBASE_CLIENT_PAUSE, "3000");
+        config.set(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "1");
+        config.set(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, "60000");
+
+        hbaseconnectionUrl = "hbase:" + host + ":" + port + ":" + parent;
+        updateKylinConfigWithMinicluster();
+
+        UTIL.startMiniMapReduceCluster();
+
+        // create the metadata htables;
+        @SuppressWarnings("unused")
+        HBaseResourceStore store = new HBaseResourceStore(KylinConfig.getInstanceFromEnv());
+
+        // import the table content
+        HbaseImporter.importHBaseData(hbaseTarLocation, UTIL.getConfiguration());
+
+    }
+
+    private static boolean existInClassPath(String className) {
+        try {
+            Class.forName(className);
+        } catch (ClassNotFoundException e) {
+            return false;
+        }
+        return true;
+    }
+
+    /**
+     * Shutdown the minicluster; 
+     */
+    public static void shutdownMiniCluster() {
+
+        logger.info("Going to shutdown mini cluster.");
+
+        try {
+            UTIL.shutdownMiniMapReduceCluster();
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+
+        try {
+            UTIL.shutdownMiniCluster();
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
+    public static void main(String[] args) {
+        HBaseMiniclusterHelper t = new HBaseMiniclusterHelper();
+        logger.info(t);
+        try {
+            HBaseMetadataTestCase.staticCreateTestMetadata(HBaseMetadataTestCase.MINICLUSTER_TEST_DATA);
+            HBaseMiniclusterHelper.startupMinicluster();
+        } catch (Exception e) {
+            e.printStackTrace();
+        } finally {
+            HBaseMiniclusterHelper.shutdownMiniCluster();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HbaseImporter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HbaseImporter.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HbaseImporter.java
new file mode 100644
index 0000000..00285ab
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HbaseImporter.java
@@ -0,0 +1,121 @@
+/*
+ * 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.File;
+import java.io.IOException;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.mapreduce.Import;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.storage.hbase.HBaseConnection;
+
+import com.google.common.base.Preconditions;
+
+/**
+ */
+public class HbaseImporter {
+
+    private static final Log logger = LogFactory.getLog(HbaseImporter.class);
+
+    public static void importHBaseData(String hbaseTarLocation, Configuration conf) throws IOException, ClassNotFoundException, InterruptedException {
+
+        if (System.getenv("JAVA_HOME") == null) {
+            logger.error("Didn't find $JAVA_HOME, this will cause HBase data import failed. Please set $JAVA_HOME.");
+            logger.error("Skipping table import...");
+            return;
+        }
+
+        File exportFile = new File(hbaseTarLocation);
+        if (!exportFile.exists()) {
+            logger.error("Didn't find the export achieve file on " + exportFile.getAbsolutePath());
+            return;
+        }
+
+        File folder = new File("/tmp/hbase-export/");
+        if (folder.exists()) {
+            FileUtils.deleteDirectory(folder);
+        }
+        folder.mkdirs();
+        folder.deleteOnExit();
+
+        //TarGZUtil.uncompressTarGZ(exportFile, folder);
+        FileUtil.unTar(exportFile, folder);
+        String[] child = folder.list();
+        Preconditions.checkState(child.length == 1);
+        String backupFolderName = child[0];
+        File backupFolder = new File(folder, backupFolderName);
+        String[] tableNames = backupFolder.list();
+
+        for (String table : tableNames) {
+
+            if (!(table.equalsIgnoreCase(HBaseMiniclusterHelper.TEST_METADATA_TABLE) || table.startsWith(HBaseMiniclusterHelper.SHARED_STORAGE_PREFIX))) {
+                continue;
+            }
+
+            // create the htable; otherwise the import will fail.
+            if (table.startsWith(HBaseMiniclusterHelper.II_STORAGE_PREFIX)) {
+                HBaseConnection.createHTableIfNeeded(KylinConfig.getInstanceFromEnv().getStorageUrl(), table, "f");
+            } else if (table.startsWith(HBaseMiniclusterHelper.CUBE_STORAGE_PREFIX)) {
+                HBaseConnection.createHTableIfNeeded(KylinConfig.getInstanceFromEnv().getStorageUrl(), table, "F1", "F2");
+            }
+
+            // directly import from local fs, no need to copy to hdfs
+            String importLocation = "file://" + backupFolder.getAbsolutePath() + "/" + table;
+            String[] args = new String[] { table, importLocation };
+            boolean result = runImport(args, conf);
+            logger.info("importing table '" + table + "' with result:" + result);
+
+            if (!result)
+                break;
+        }
+
+    }
+
+    private static boolean runImport(String[] args, Configuration configuration) throws IOException, InterruptedException, ClassNotFoundException {
+        // need to make a copy of the configuration because to make sure different temp dirs are used.
+        GenericOptionsParser opts = new GenericOptionsParser(new Configuration(configuration), args);
+        Configuration newConf = opts.getConfiguration();
+        args = opts.getRemainingArgs();
+        Job job = Import.createSubmittableJob(newConf, args);
+        job.waitForCompletion(false);
+        return job.isSuccessful();
+    }
+
+    public static void main(String[] args) throws IOException, InterruptedException, ClassNotFoundException {
+        if (args.length != 1) {
+            logger.error("Usage: HbaseImporter hbase_tar_lcoation");
+            System.exit(-1);
+        }
+
+        logger.info("The KylinConfig being used:");
+        logger.info("=================================================");
+        KylinConfig.getInstanceFromEnv().printProperties();
+        logger.info("=================================================");
+
+        importHBaseData(args[0], HBaseConfiguration.create());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
new file mode 100644
index 0000000..2560bc2
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
@@ -0,0 +1,211 @@
+/*
+ * 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 org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.persistence.RootPersistentEntity;
+import org.apache.kylin.common.persistence.Serializer;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.storage.hbase.HBaseResourceStore;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import static org.junit.Assert.*;
+
+public class ITHBaseResourceStoreTest extends HBaseMetadataTestCase {
+
+    @Before
+    public void setup() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testHBaseStore() throws Exception {
+        testAStore(ResourceStore.getStore(KylinConfig.getInstanceFromEnv()));
+    }
+
+    @Test
+    public void testHBaseStoreWithLargeCell() throws Exception {
+        String path = "/cube/_test_large_cell.json";
+        String largeContent = "THIS_IS_A_LARGE_CELL";
+        StringEntity content = new StringEntity(largeContent);
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        int origSize = config.getHBaseKeyValueSize();
+        ResourceStore store = ResourceStore.getStore(KylinConfig.getInstanceFromEnv());
+
+        try {
+            config.setProperty("kylin.hbase.client.keyvalue.maxsize", String.valueOf(largeContent.length() - 1));
+
+            store.deleteResource(path);
+
+            store.putResource(path, content, StringEntity.serializer);
+            assertTrue(store.exists(path));
+            StringEntity t = store.getResource(path, StringEntity.class, StringEntity.serializer);
+            assertEquals(content, t);
+
+            Path redirectPath = ((HBaseResourceStore) store).bigCellHDFSPath(path);
+            Configuration hconf = HadoopUtil.getCurrentConfiguration();
+            FileSystem fileSystem = FileSystem.get(hconf);
+            assertTrue(fileSystem.exists(redirectPath));
+
+            FSDataInputStream in = fileSystem.open(redirectPath);
+            assertEquals(largeContent, in.readUTF());
+            in.close();
+
+            store.deleteResource(path);
+        } finally {
+            config.setProperty("kylin.hbase.client.keyvalue.maxsize", "" + origSize);
+            store.deleteResource(path);
+        }
+    }
+
+    void testAStore(ResourceStore store) throws IOException {
+        String dir1 = "/cube";
+        String path1 = "/cube/_test.json";
+        StringEntity content1 = new StringEntity("anything");
+        String dir2 = "/table";
+        String path2 = "/table/_test.json";
+        StringEntity content2 = new StringEntity("something");
+
+        // cleanup legacy if any
+        store.deleteResource(path1);
+        store.deleteResource(path2);
+
+        StringEntity t;
+
+        // put/get
+        store.putResource(path1, content1, StringEntity.serializer);
+        assertTrue(store.exists(path1));
+        t = store.getResource(path1, StringEntity.class, StringEntity.serializer);
+        assertEquals(content1, t);
+
+        store.putResource(path2, content2, StringEntity.serializer);
+        assertTrue(store.exists(path2));
+        t = store.getResource(path2, StringEntity.class, StringEntity.serializer);
+        assertEquals(content2, t);
+
+        // overwrite
+        t.str = "new string";
+        store.putResource(path2, t, StringEntity.serializer);
+
+        // write conflict
+        try {
+            t.setLastModified(t.getLastModified() - 1);
+            store.putResource(path2, t, StringEntity.serializer);
+            fail("write conflict should trigger IllegalStateException");
+        } catch (IllegalStateException e) {
+            // expected
+        }
+
+        // list
+        ArrayList<String> list;
+
+        list = store.listResources(dir1);
+        assertTrue(list.contains(path1));
+        assertTrue(list.contains(path2) == false);
+
+        list = store.listResources(dir2);
+        assertTrue(list.contains(path2));
+        assertTrue(list.contains(path1) == false);
+
+        list = store.listResources("/");
+        assertTrue(list.contains(dir1));
+        assertTrue(list.contains(dir2));
+        assertTrue(list.contains(path1) == false);
+        assertTrue(list.contains(path2) == false);
+
+        list = store.listResources(path1);
+        assertNull(list);
+        list = store.listResources(path2);
+        assertNull(list);
+
+        // delete/exist
+        store.deleteResource(path1);
+        assertTrue(store.exists(path1) == false);
+        list = store.listResources(dir1);
+        assertTrue(list == null || list.contains(path1) == false);
+
+        store.deleteResource(path2);
+        assertTrue(store.exists(path2) == false);
+        list = store.listResources(dir2);
+        assertTrue(list == null || list.contains(path2) == false);
+    }
+
+    public static class StringEntity extends RootPersistentEntity {
+
+        static final Serializer<StringEntity> serializer = new Serializer<StringEntity>() {
+            @Override
+            public void serialize(StringEntity obj, DataOutputStream out) throws IOException {
+                out.writeUTF(obj.str);
+            }
+
+            @Override
+            public StringEntity deserialize(DataInputStream in) throws IOException {
+                String str = in.readUTF();
+                return new StringEntity(str);
+            }
+        };
+
+        String str;
+
+        public StringEntity(String str) {
+            this.str = str;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = super.hashCode();
+            result = prime * result + ((str == null) ? 0 : str.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == this)
+                return true;
+            if (!(obj instanceof StringEntity))
+                return false;
+            return StringUtils.equals(this.str, ((StringEntity) obj).str);
+        }
+
+        @Override
+        public String toString() {
+            return str;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage/pom.xml
----------------------------------------------------------------------
diff --git a/storage/pom.xml b/storage/pom.xml
index a720329..e557363 100644
--- a/storage/pom.xml
+++ b/storage/pom.xml
@@ -32,31 +32,35 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-common</artifactId>
-            <type>test-jar</type>
-            <scope>test</scope>
-            <version>${project.parent.version}</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-cube</artifactId>
+            <artifactId>kylin-invertedindex</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-invertedindex</artifactId>
+            <artifactId>kylin-storage-hbase</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-
-
-        <!-- Env & Test -->
         <dependency>
             <groupId>net.sf.ehcache</groupId>
             <artifactId>ehcache</artifactId>
             <version>2.8.1</version>
         </dependency>
 
+        <!-- Env & Test -->
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-storage-hbase</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java b/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java
index 261e85d..e4b8657 100644
--- a/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java
+++ b/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java
@@ -11,7 +11,6 @@ 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.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.ImmutableBitSet;
@@ -28,6 +27,7 @@ import org.apache.kylin.storage.gridtable.GTRowBlock;
 import org.apache.kylin.storage.gridtable.GTRowBlock.Writer;
 import org.apache.kylin.storage.gridtable.GTScanRequest;
 import org.apache.kylin.storage.gridtable.IGTStore;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/HadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/HadoopFileSystem.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/HadoopFileSystem.java
index 9546db7..429a30b 100644
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/HadoopFileSystem.java
+++ b/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/HadoopFileSystem.java
@@ -1,7 +1,7 @@
 package org.apache.kylin.storage.gridtable.diskstore;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java
index 1d76bd4..889783a 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java
@@ -22,8 +22,8 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
+
 import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageQuery.java b/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageQuery.java
index 762258d..d203078 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageQuery.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageQuery.java
@@ -19,8 +19,8 @@
 package org.apache.kylin.storage.hbase;
 
 import com.google.common.collect.Range;
+
 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;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 1d4ec68..b69d5b6 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
@@ -29,10 +29,8 @@ 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.kylin.common.util.Bytes;
 import org.apache.hadoop.security.UserGroupInformation;
-
-import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.common.util.Bytes;
 
 /**
  * @author yangli9
@@ -46,7 +44,7 @@ public class PingHBaseCLI {
 
         System.out.println("Hello friend.");
 
-        Configuration hconf = HadoopUtil.newHBaseConfiguration(metadataUrl);
+        Configuration hconf = HBaseConnection.newHBaseConfiguration(metadataUrl);
         if (User.isHBaseSecurityEnabled(hconf)) {
             try {
                 System.out.println("--------------Getting kerberos credential for user " + UserGroupInformation.getCurrentUser().getUserName());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage/src/test/java/org/apache/kylin/storage/hbase/ITInvertedIndexHBaseTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/hbase/ITInvertedIndexHBaseTest.java b/storage/src/test/java/org/apache/kylin/storage/hbase/ITInvertedIndexHBaseTest.java
index c92921b..e3f3bc4 100644
--- a/storage/src/test/java/org/apache/kylin/storage/hbase/ITInvertedIndexHBaseTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/hbase/ITInvertedIndexHBaseTest.java
@@ -18,14 +18,13 @@
 
 package org.apache.kylin.storage.hbase;
 
-import com.google.common.collect.Lists;
+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.KylinConfig;
 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;
@@ -39,7 +38,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.List;
+import com.google.common.collect.Lists;
 
 /**
  * @author yangli9
@@ -60,7 +59,7 @@ public class ITInvertedIndexHBaseTest extends HBaseMetadataTestCase {
         this.seg = ii.getFirstSegment();
 
         String hbaseUrl = KylinConfig.getInstanceFromEnv().getStorageUrl();
-        Configuration hconf = HadoopUtil.newHBaseConfiguration(hbaseUrl);
+        Configuration hconf = HBaseConnection.newHBaseConfiguration(hbaseUrl);
         hconn = HConnectionManager.createConnection(hconf);
 
         this.info = new TableRecordInfo(seg);
@@ -102,6 +101,7 @@ public class ITInvertedIndexHBaseTest extends HBaseMetadataTestCase {
         return records;
     }
 
+    @SuppressWarnings("unused")
     private void dump(Iterable<TableRecord> records) {
         for (TableRecord rec : records) {
             byte[] x = rec.getBytes();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage/src/test/java/org/apache/kylin/storage/test/ITStorageTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/test/ITStorageTest.java b/storage/src/test/java/org/apache/kylin/storage/test/ITStorageTest.java
index 4408b55..19c9bce 100644
--- a/storage/src/test/java/org/apache/kylin/storage/test/ITStorageTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/test/ITStorageTest.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.storage.test;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.metadata.filter.TupleFilter;
@@ -31,6 +30,7 @@ import org.apache.kylin.metadata.tuple.ITupleIterator;
 import org.apache.kylin.storage.IStorageQuery;
 import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.StorageFactory;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.apache.kylin.storage.hbase.ScanOutOfLimitException;
 import org.junit.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
index fd8e973..4791684 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -14,49 +14,52 @@
 
     <dependencies>
 
-
         <dependency>
-            <groupId>org.apache.mrunit</groupId>
-            <artifactId>mrunit</artifactId>
-            <classifier>hadoop2</classifier>
-            <scope>test</scope>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-cube</artifactId>
+            <version>${project.parent.version}</version>
         </dependency>
-
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-invertedindex</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-storage</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
 
+        <!-- Test & Env -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-cube</artifactId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-storage</artifactId>
+            <artifactId>kylin-storage-hbase</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.kafka</groupId>
             <artifactId>kafka_2.10</artifactId>
             <version>${kafka.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-client</artifactId>
-            <scope>provided</scope>
-            <version>${hbase-hadoop2.version}</version>
-        </dependency>
 
         <dependency>
             <groupId>org.apache.kylin</groupId>
@@ -65,6 +68,12 @@
             <scope>test</scope>
             <version>${project.parent.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.mrunit</groupId>
+            <artifactId>mrunit</artifactId>
+            <classifier>hadoop2</classifier>
+            <scope>test</scope>
+        </dependency>
 
     </dependencies>
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/IIStreamConsumer.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/IIStreamConsumer.java b/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/IIStreamConsumer.java
index 421daef..fa89b2b 100644
--- a/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/IIStreamConsumer.java
+++ b/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/IIStreamConsumer.java
@@ -36,15 +36,16 @@ package org.apache.kylin.streaming.invertedindex;
 
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
+
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.invertedindex.index.Slice;
 import org.apache.kylin.invertedindex.model.IIDesc;
 import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
 import org.apache.kylin.invertedindex.model.IIRow;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.streaming.MicroStreamBatch;
 import org.apache.kylin.streaming.MicroStreamBatchConsumer;
 import org.apache.kylin.streaming.StreamingManager;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/SliceBuilder.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/SliceBuilder.java b/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/SliceBuilder.java
index 604bf9c..290f51a 100644
--- a/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/SliceBuilder.java
+++ b/streaming/src/main/java/org/apache/kylin/streaming/invertedindex/SliceBuilder.java
@@ -1,35 +1,18 @@
 /*
- *
- *
- *  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.
- *
- * /
+ * 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.streaming.invertedindex;



[3/6] incubator-kylin git commit: KYLIN-875 Refactor core-common, remove dependency on hadoop/hbase

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/invertedindex/src/main/java/org/apache/kylin/invertedindex/tools/IICLI.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/tools/IICLI.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/tools/IICLI.java
deleted file mode 100644
index 068af07..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/tools/IICLI.java
+++ /dev/null
@@ -1,108 +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.invertedindex.tools;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-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.IIKeyValueCodec;
-import org.apache.kylin.invertedindex.model.IIRow;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-/**
- * @author yangli9
- */
-public class IICLI {
-
-	public static void main(String[] args) throws IOException {
-		Configuration hconf = HadoopUtil.getCurrentConfiguration();
-		IIManager mgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
-
-		String iiName = args[0];
-		IIInstance ii = mgr.getII(iiName);
-
-		String path = args[1];
-		System.out.println("Reading from " + path + " ...");
-
-		TableRecordInfo info = new TableRecordInfo(ii.getFirstSegment());
-		IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest());
-		int count = 0;
-		for (Slice slice : codec.decodeKeyValue(readSequenceKVs(hconf, path))) {
-			for (RawTableRecord rec : slice) {
-				System.out.printf(new TableRecord(rec, info).toString());
-				count++;
-			}
-		}
-		System.out.println("Total " + count + " records");
-	}
-
-	public static Iterable<IIRow> readSequenceKVs(
-			Configuration hconf, String path) throws IOException {
-		final Reader reader = new Reader(hconf,
-				SequenceFile.Reader.file(new Path(path)));
-		return new Iterable<IIRow>() {
-			@Override
-			public Iterator<IIRow> iterator() {
-				return new Iterator<IIRow>() {
-					ImmutableBytesWritable k = new ImmutableBytesWritable();
-					ImmutableBytesWritable v = new ImmutableBytesWritable();
-                    IIRow pair = new IIRow(k, v, null);
-
-					@Override
-					public boolean hasNext() {
-						boolean hasNext = false;
-						try {
-							hasNext = reader.next(k, v);
-						} catch (IOException e) {
-							throw new RuntimeException(e);
-						} finally {
-							if (hasNext == false) {
-								IOUtils.closeQuietly(reader);
-							}
-						}
-						return hasNext;
-					}
-
-					@Override
-					public IIRow next() {
-						return pair;
-					}
-
-					@Override
-					public void remove() {
-						throw new UnsupportedOperationException();
-					}
-				};
-			}
-		};
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/dependency-reduced-pom.xml
----------------------------------------------------------------------
diff --git a/job/dependency-reduced-pom.xml b/job/dependency-reduced-pom.xml
deleted file mode 100644
index 4073e53..0000000
--- a/job/dependency-reduced-pom.xml
+++ /dev/null
@@ -1,594 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one
- or more contributor license agreements.  See the NOTICE file
- distributed with this work for additional information
- regarding copyright ownership.  The ASF licenses this file
- to you under the Apache License, Version 2.0 (the
- "License"); you may not use this file except in compliance
- with the License.  You may obtain a copy of the License at
- 
-     http://www.apache.org/licenses/LICENSE-2.0
- 
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <parent>
-    <artifactId>kylin</artifactId>
-    <groupId>org.apache.kylin</groupId>
-    <version>0.6.4-SNAPSHOT</version>
-  </parent>
-  <modelVersion>4.0.0</modelVersion>
-  <artifactId>kylin-job</artifactId>
-  <name>Kylin:Job</name>
-  <url>http://maven.apache.org</url>
-  <build>
-    <plugins>
-      <plugin>
-        <artifactId>maven-shade-plugin</artifactId>
-        <version>2.3</version>
-        <executions>
-          <execution>
-            <phase>package</phase>
-            <goals>
-              <goal>shade</goal>
-            </goals>
-            <configuration>
-              <minimizeJar>true</minimizeJar>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.curator</groupId>
-      <artifactId>curator-framework</artifactId>
-      <version>2.6.0</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>zookeeper</artifactId>
-          <groupId>org.apache.zookeeper</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>curator-client</artifactId>
-          <groupId>org.apache.curator</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.curator</groupId>
-      <artifactId>curator-recipes</artifactId>
-      <version>2.6.0</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>zookeeper</artifactId>
-          <groupId>org.apache.zookeeper</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <version>2.6.0</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>servlet-api</artifactId>
-          <groupId>javax.servlet</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>commons-math3</artifactId>
-          <groupId>org.apache.commons</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>xmlenc</artifactId>
-          <groupId>xmlenc</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>commons-net</artifactId>
-          <groupId>commons-net</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jetty</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jetty-util</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-core</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-json</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-server</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jasper-compiler</artifactId>
-          <groupId>tomcat</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jasper-runtime</artifactId>
-          <groupId>tomcat</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jsp-api</artifactId>
-          <groupId>javax.servlet.jsp</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>commons-el</artifactId>
-          <groupId>commons-el</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jets3t</artifactId>
-          <groupId>net.java.dev.jets3t</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>slf4j-log4j12</artifactId>
-          <groupId>org.slf4j</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jackson-core-asl</artifactId>
-          <groupId>org.codehaus.jackson</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jackson-mapper-asl</artifactId>
-          <groupId>org.codehaus.jackson</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>avro</artifactId>
-          <groupId>org.apache.avro</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>gson</artifactId>
-          <groupId>com.google.code.gson</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-auth</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>htrace-core</artifactId>
-          <groupId>org.htrace</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>zookeeper</artifactId>
-          <groupId>org.apache.zookeeper</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>commons-compress</artifactId>
-          <groupId>org.apache.commons</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>curator-client</artifactId>
-          <groupId>org.apache.curator</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-annotations</artifactId>
-      <version>2.6.0</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>jdk.tools</artifactId>
-          <groupId>jdk.tools</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-core</artifactId>
-      <version>2.6.0</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>hadoop-yarn-common</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>guice-servlet</artifactId>
-          <groupId>com.google.inject.extensions</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>netty</artifactId>
-          <groupId>io.netty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>avro</artifactId>
-          <groupId>org.apache.avro</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>slf4j-log4j12</artifactId>
-          <groupId>org.slf4j</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-minicluster</artifactId>
-      <version>2.6.0</version>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>hadoop-common</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-hdfs</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-yarn-server-tests</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-app</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-hs</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.mrunit</groupId>
-      <artifactId>mrunit</artifactId>
-      <version>1.0.0</version>
-      <classifier>hadoop2</classifier>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>mockito-all</artifactId>
-          <groupId>org.mockito</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-hadoop2-compat</artifactId>
-      <version>0.98.4-hadoop2</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>hbase-hadoop-compat</artifactId>
-          <groupId>org.apache.hbase</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>metrics-core</artifactId>
-          <groupId>com.yammer.metrics</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-client</artifactId>
-      <version>0.98.4-hadoop2</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>hbase-protocol</artifactId>
-          <groupId>org.apache.hbase</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>htrace-core</artifactId>
-          <groupId>org.cloudera.htrace</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>netty</artifactId>
-          <groupId>io.netty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>zookeeper</artifactId>
-          <groupId>org.apache.zookeeper</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jackson-mapper-asl</artifactId>
-          <groupId>org.codehaus.jackson</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-auth</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-server</artifactId>
-      <version>0.98.4-hadoop2</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>hbase-prefix-tree</artifactId>
-          <groupId>org.apache.hbase</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>high-scale-lib</artifactId>
-          <groupId>com.github.stephenc.high-scale-lib</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>commons-math</artifactId>
-          <groupId>org.apache.commons</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jetty-sslengine</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jsp-2.1</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jsp-api-2.1</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>servlet-api-2.5</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jackson-jaxrs</artifactId>
-          <groupId>org.codehaus.jackson</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jamon-runtime</artifactId>
-          <groupId>org.jamon</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jaxb-api</artifactId>
-          <groupId>javax.xml.bind</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-client</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hbase-protocol</artifactId>
-          <groupId>org.apache.hbase</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hbase-hadoop-compat</artifactId>
-          <groupId>org.apache.hbase</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>metrics-core</artifactId>
-          <groupId>com.yammer.metrics</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jetty</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jetty-util</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jackson-core-asl</artifactId>
-          <groupId>org.codehaus.jackson</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jasper-compiler</artifactId>
-          <groupId>tomcat</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jasper-runtime</artifactId>
-          <groupId>tomcat</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-core</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-json</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-server</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>htrace-core</artifactId>
-          <groupId>org.cloudera.htrace</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>zookeeper</artifactId>
-          <groupId>org.apache.zookeeper</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jackson-mapper-asl</artifactId>
-          <groupId>org.codehaus.jackson</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-auth</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
-      <version>2.6.0</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>servlet-api</artifactId>
-          <groupId>javax.servlet</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>guice</artifactId>
-          <groupId>com.google.inject</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-guice</artifactId>
-          <groupId>com.sun.jersey.contribs</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jettison</artifactId>
-          <groupId>org.codehaus.jettison</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-client</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-yarn-server-common</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-yarn-server-web-proxy</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>guice-servlet</artifactId>
-          <groupId>com.google.inject.extensions</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-yarn-common</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jaxb-api</artifactId>
-          <groupId>javax.xml.bind</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-json</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-core</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jetty-util</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>zookeeper</artifactId>
-          <groupId>org.apache.zookeeper</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>4.11</version>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>hamcrest-core</artifactId>
-          <groupId>org.hamcrest</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.maven</groupId>
-      <artifactId>maven-model</artifactId>
-      <version>3.1.1</version>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>plexus-utils</artifactId>
-          <groupId>org.codehaus.plexus</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-      <version>2.6.0</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <version>2.6.0</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>xercesImpl</artifactId>
-          <groupId>xerces</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jsp-api</artifactId>
-          <groupId>javax.servlet.jsp</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>servlet-api</artifactId>
-          <groupId>javax.servlet</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>xmlenc</artifactId>
-          <groupId>xmlenc</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>htrace-core</artifactId>
-          <groupId>org.htrace</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jetty</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-server</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jackson-core-asl</artifactId>
-          <groupId>org.codehaus.jackson</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jasper-runtime</artifactId>
-          <groupId>tomcat</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>netty</artifactId>
-          <groupId>io.netty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jetty-util</artifactId>
-          <groupId>org.mortbay.jetty</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-core</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jackson-mapper-asl</artifactId>
-          <groupId>org.codehaus.jackson</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-  </dependencies>
-  <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-  </properties>
-</project>
-

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/pom.xml
----------------------------------------------------------------------
diff --git a/job/pom.xml b/job/pom.xml
index 8bb5f3d..cbb0772 100644
--- a/job/pom.xml
+++ b/job/pom.xml
@@ -35,21 +35,17 @@
 
     <dependencies>
 
+        <!--Kylin Jar -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-common</artifactId>
-            <type>test-jar</type>
-            <scope>test</scope>
+            <artifactId>kylin-core-job</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-
-        <!--Kylin Jar -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-cube</artifactId>
+            <artifactId>kylin-engine-mr</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-storage</artifactId>
@@ -57,58 +53,26 @@
             <scope>test</scope>
             <version>${project.parent.version}</version>
         </dependency>
-
-
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-invertedindex</artifactId>
+            <artifactId>kylin-source-hive</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-streaming</artifactId>
+            <artifactId>kylin-invertedindex</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-
-        <dependency>
-            <groupId>commons-cli</groupId>
-            <artifactId>commons-cli</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-lang</groupId>
-            <artifactId>commons-lang</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-configuration</groupId>
-            <artifactId>commons-configuration</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-databind</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-httpclient</groupId>
-            <artifactId>commons-httpclient</artifactId>
-        </dependency>
         <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-streaming</artifactId>
+            <version>${project.parent.version}</version>
         </dependency>
-
+        
         <dependency>
             <groupId>commons-daemon</groupId>
             <artifactId>commons-daemon</artifactId>
         </dependency>
-
         <dependency>
             <groupId>org.apache.curator</groupId>
             <artifactId>curator-framework</artifactId>
@@ -129,19 +93,22 @@
                 </exclusion>
             </exclusions>
         </dependency>
+
+        <!-- Env & Test -->
         <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-email</artifactId>
-            <version>1.1</version>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
         </dependency>
         <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-common</artifactId>
-            <scope>provided</scope>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-storage-hbase</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
         </dependency>
-
-        <!-- Env & Test -->
-
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>
@@ -170,6 +137,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-hadoop2-compat</artifactId>
             <scope>provided</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
index 5ecd26a..ff6ffe5 100644
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
+++ b/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
@@ -16,7 +16,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -24,6 +23,7 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.DimensionDesc;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
 import org.apache.kylin.engine.mr.MRUtil;
 import org.apache.kylin.job.constant.BatchConstants;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
index 1b3e2b2..45da2c8 100644
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
+++ b/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
@@ -4,7 +4,6 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinReducer;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -12,6 +11,7 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
 import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
 import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.KylinReducer;
 import org.apache.kylin.engine.mr.IMROutput2.IMRStorageOutputFormat;
 import org.apache.kylin.engine.mr.MRUtil;
 import org.apache.kylin.job.constant.BatchConstants;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
index 6535777..1162f14 100644
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
+++ b/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
@@ -25,7 +25,6 @@ import java.util.HashMap;
 import java.util.List;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.SplittedBytes;
@@ -41,6 +40,7 @@ import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.DictionaryManager;
 import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.IMROutput2.IMRStorageInputFormat;
 import org.apache.kylin.engine.mr.MRUtil;
 import org.apache.kylin.job.constant.BatchConstants;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
index 9a7f761..8bd6ea2 100644
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
+++ b/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
@@ -20,6 +20,7 @@ package org.apache.kylin.engine.mr.steps;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -35,10 +36,10 @@ import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.ByteArray;
 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.engine.mr.HadoopUtil;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
index bdaf689..14eef1a 100644
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
+++ b/job/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
@@ -26,10 +26,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
-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.engine.mr.HadoopUtil;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/Scheduler.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/Scheduler.java b/job/src/main/java/org/apache/kylin/job/Scheduler.java
index fe6fe95..2ed2fc2 100644
--- a/job/src/main/java/org/apache/kylin/job/Scheduler.java
+++ b/job/src/main/java/org/apache/kylin/job/Scheduler.java
@@ -18,10 +18,10 @@
 
 package org.apache.kylin.job;
 
-import org.apache.kylin.common.lock.JobLock;
 import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.exception.SchedulerException;
 import org.apache.kylin.job.execution.Executable;
+import org.apache.kylin.job.lock.JobLock;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/common/HqlExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/common/HqlExecutable.java b/job/src/main/java/org/apache/kylin/job/common/HqlExecutable.java
index 32d6ec2..ffe45ed 100644
--- a/job/src/main/java/org/apache/kylin/job/common/HqlExecutable.java
+++ b/job/src/main/java/org/apache/kylin/job/common/HqlExecutable.java
@@ -27,11 +27,12 @@ import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableContext;
 import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.source.hive.HiveClient;
 import org.datanucleus.store.types.backed.HashMap;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.google.common.collect.Lists;
-import org.apache.kylin.common.util.HiveClient;
+
 import org.apache.kylin.common.util.JsonUtil;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 a27866c..c10d4e0 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
@@ -43,10 +43,10 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.CliCommandExecutor;
-import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.common.util.StringSplitter;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.job.JobInstance;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java
index 071554f..7f549d0 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java
@@ -29,10 +29,10 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.MRUtil;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityReducer.java
index 2b36bee..09baa75 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityReducer.java
@@ -25,10 +25,9 @@ import java.util.*;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
-
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.mr.KylinReducer;
 import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.engine.mr.KylinReducer;
 
 /**
  * @author Jack

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidMapperBase.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidMapperBase.java
index 3081b7d..dc4c746 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidMapperBase.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidMapperBase.java
@@ -1,10 +1,10 @@
 package org.apache.kylin.job.hadoop.cube;
 
 import com.google.common.collect.Lists;
+
 import org.apache.kylin.common.util.Bytes;
 import org.apache.hadoop.io.Text;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.common.util.BytesSplitter;
 import org.apache.kylin.common.util.SplittedBytes;
 import org.apache.kylin.cube.CubeInstance;
@@ -15,6 +15,7 @@ import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.measure.MeasureCodec;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper.java
index 56799ad..f6e6182 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper.java
@@ -26,11 +26,11 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
 import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.measure.MeasureCodec;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidReducer.java
index 8bfbdf5..7f41b8d 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidReducer.java
@@ -22,17 +22,16 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.kylin.common.mr.KylinReducer;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.metadata.measure.MeasureAggregators;
 import org.apache.kylin.metadata.measure.MeasureCodec;
 import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.KylinReducer;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsCombiner.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsCombiner.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsCombiner.java
index 1391f15..12c8713 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsCombiner.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsCombiner.java
@@ -22,11 +22,10 @@ import java.io.IOException;
 import java.util.HashSet;
 
 import org.apache.hadoop.io.LongWritable;
-import org.apache.kylin.common.mr.KylinReducer;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.hadoop.io.Text;
-
 import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.engine.mr.KylinReducer;
 
 /**
  * @author yangli9

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsMapperBase.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsMapperBase.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsMapperBase.java
index 09dd3bf..d30cfc0 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsMapperBase.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsMapperBase.java
@@ -9,7 +9,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -18,6 +17,7 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.RowKeyDesc;
 import org.apache.kylin.dict.DictionaryManager;
 import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.MRUtil;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsReducer.java
index a43b0fb..4b73739 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsReducer.java
@@ -20,6 +20,7 @@ package org.apache.kylin.job.hadoop.cube;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -27,7 +28,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.*;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.mr.KylinReducer;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.CubeInstance;
@@ -35,6 +35,7 @@ import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.KylinReducer;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapper.java
index f776029..0e7a89f 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapper.java
@@ -27,9 +27,7 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -41,6 +39,7 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapper.java
index dcc77ea..c4978bf 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapper.java
@@ -23,13 +23,10 @@ import java.util.Collection;
 
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
-
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -38,6 +35,7 @@ import org.apache.kylin.common.util.SplittedBytes;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.job.constant.BatchConstants;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
index 20f2474..97b5de8 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
@@ -29,7 +29,6 @@ import java.util.List;
 
 import org.apache.hadoop.io.Text;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Bytes;
@@ -45,6 +44,7 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.DimensionDesc;
 import org.apache.kylin.dict.lookup.LookupBytesTable;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.MetadataManager;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapper.java
index 43ca888..e49f090 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapper.java
@@ -20,7 +20,7 @@ package org.apache.kylin.job.hadoop.cube;
 
 import java.io.IOException;
 
-import org.apache.kylin.common.mr.KylinMapper;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
index f41a1e8..a580a9e 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
@@ -27,9 +27,8 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import org.apache.kylin.common.mr.KylinReducer;
 import org.apache.kylin.cube.model.v1.CubeDesc.CubeCapacity;
+import org.apache.kylin.engine.mr.KylinReducer;
 import org.apache.kylin.job.constant.BatchConstants;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
index 0189315..de57562 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
@@ -29,8 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.util.ReflectionUtils;
-
-import org.apache.kylin.common.mr.KylinMapper;
+import org.apache.kylin.engine.mr.KylinMapper;
 
 /**
  * @author ysong1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
index 324e718..c010a33 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
@@ -22,8 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-
-import org.apache.kylin.common.mr.KylinReducer;
+import org.apache.kylin.engine.mr.KylinReducer;
 
 /**
  * @author ysong1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateDictionaryJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateDictionaryJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateDictionaryJob.java
index a2d3190..1a5e690 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateDictionaryJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateDictionaryJob.java
@@ -20,10 +20,13 @@ package org.apache.kylin.job.hadoop.dict;
 
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.util.ToolRunner;
-
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.cli.DictionaryGeneratorCLI;
+import org.apache.kylin.dict.DistinctColumnValuesProvider;
+import org.apache.kylin.engine.mr.DFSFileTable;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.source.ReadableTable;
 
 /**
  * @author ysong1
@@ -44,13 +47,18 @@ public class CreateDictionaryJob extends AbstractHadoopJob {
             options.addOption(OPTION_INPUT_PATH);
             parseOptions(options, args);
 
-            String cubeName = getOptionValue(OPTION_CUBE_NAME);
-            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
-            String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
+            final String cubeName = getOptionValue(OPTION_CUBE_NAME);
+            final String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+            final String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
 
             KylinConfig config = KylinConfig.getInstanceFromEnv();
 
-            DictionaryGeneratorCLI.processSegment(config, cubeName, segmentName, factColumnsInputPath);
+            DictionaryGeneratorCLI.processSegment(config, cubeName, segmentName, new DistinctColumnValuesProvider() {
+                @Override
+                public ReadableTable getDistinctValuesFor(TblColRef col) {
+                    return new DFSFileTable(factColumnsInputPath + "/" + col.getName(), -1);
+                }
+            });
         } catch (Exception e) {
             printUsage(options);
             throw e;
@@ -58,7 +66,7 @@ public class CreateDictionaryJob extends AbstractHadoopJob {
 
         return returnCode;
     }
-
+    
     public static void main(String[] args) throws Exception {
         int exitCode = ToolRunner.run(new CreateDictionaryJob(), args);
         System.exit(exitCode);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
index 478ae3d..66a1106 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
@@ -21,13 +21,15 @@ package org.apache.kylin.job.hadoop.dict;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.dict.DistinctColumnValuesProvider;
+import org.apache.kylin.engine.mr.DFSFileTable;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.source.ReadableTable;
 
 /**
- * @author ysong1
- * 
  */
 public class CreateInvertedIndexDictionaryJob extends AbstractHadoopJob {
 
@@ -40,14 +42,19 @@ public class CreateInvertedIndexDictionaryJob extends AbstractHadoopJob {
             options.addOption(OPTION_INPUT_PATH);
             parseOptions(options, args);
 
-            String iiname = getOptionValue(OPTION_II_NAME);
-            String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
-            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            final String iiname = getOptionValue(OPTION_II_NAME);
+            final String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
+            final KylinConfig config = KylinConfig.getInstanceFromEnv();
 
             IIManager mgr = IIManager.getInstance(config);
             IIInstance ii = mgr.getII(iiname);
 
-            mgr.buildInvertedIndexDictionary(ii.getFirstSegment(), factColumnsInputPath);
+            mgr.buildInvertedIndexDictionary(ii.getFirstSegment(), new DistinctColumnValuesProvider() {
+                @Override
+                public ReadableTable getDistinctValuesFor(TblColRef col) {
+                    return new DFSFileTable(factColumnsInputPath + "/" + col.getName(), -1);
+                }
+            });
             return 0;
         } catch (Exception e) {
             printUsage(options);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 f133608..304fa04 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
@@ -46,13 +46,13 @@ import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.ByteArray;
 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.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.DataType;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
index f636af4..8575f89 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
@@ -18,12 +18,6 @@
 
 package org.apache.kylin.job.hadoop.invertedindex;
 
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,7 +25,13 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
-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.model.IIDesc;
+import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 
 /**
  * @author ysong1
@@ -76,7 +76,7 @@ public class IIBulkLoadJob extends AbstractHadoopJob {
 
     public static void main(String[] args) throws Exception {
         IIBulkLoadJob job = new IIBulkLoadJob();
-        job.setConf(HadoopUtil.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl()));
+        job.setConf(HBaseConnection.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl()));
         int exitCode = ToolRunner.run(job, args);
         System.exit(exitCode);
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 c479b68..d55900b 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
@@ -30,8 +30,8 @@ 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.util.HadoopUtil;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,7 +83,7 @@ public class IICreateHFileJob extends AbstractHadoopJob {
 
     public static void main(String[] args) throws Exception {
         IICreateHFileJob job = new IICreateHFileJob();
-        job.setConf(HadoopUtil.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl()));
+        job.setConf(HBaseConnection.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl()));
         int exitCode = ToolRunner.run(job, args);
         System.exit(exitCode);
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileMapper.java
index 882a70a..1adf8d6 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileMapper.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.mr.KylinMapper;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.invertedindex.model.IIDesc;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 23a4930..3690e48 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.model.IIDesc;
@@ -41,6 +40,7 @@ import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.job.tools.DeployCoprocessorCLI;
 import org.apache.kylin.job.tools.LZOSupportnessChecker;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 
 /**
  * @author George Song (ysong1)
@@ -128,7 +128,7 @@ public class IICreateHTableJob extends AbstractHadoopJob {
 
     public static void main(String[] args) throws Exception {
         IICreateHTableJob job = new IICreateHTableJob();
-        job.setConf(HadoopUtil.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl()));
+        job.setConf(HBaseConnection.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl()));
         ToolRunner.run(job, args);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
index fc086cb..8c67538 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
@@ -24,9 +24,8 @@ import java.util.HashSet;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
-
-import org.apache.kylin.common.mr.KylinReducer;
 import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.engine.mr.KylinReducer;
 
 /**
  * @author yangli9

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
index 1a29a4a..b1ff6f1 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.job.constant.BatchConstants;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsMapper.java
index 6406ad4..aaec89f 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsMapper.java
@@ -27,8 +27,7 @@ import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-
-import org.apache.kylin.common.mr.KylinMapper;
+import org.apache.kylin.engine.mr.KylinMapper;
 
 /**
  * @author yangli9

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
index d7b2735..1676429 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
@@ -29,9 +29,8 @@ import org.apache.kylin.common.util.Bytes;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
-
-import org.apache.kylin.common.mr.KylinReducer;
 import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.engine.mr.KylinReducer;
 import org.apache.kylin.job.constant.BatchConstants;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
index 6a0532e..16b8ca1 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
index 85abc41..1d30ee7 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
@@ -29,7 +29,7 @@ import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinMapper;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
index 36c048a..d9b5aee 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.mr.KylinReducer;
+import org.apache.kylin.engine.mr.KylinReducer;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
index 7394567..e3e743e 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
@@ -26,9 +26,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.ReflectionUtils;
-
-import org.apache.kylin.common.mr.KylinMapper;
 import org.apache.kylin.common.util.RandomSampler;
+import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.job.constant.BatchConstants;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
index a16a2af..bc6d379 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
@@ -26,8 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.ReflectionUtils;
-
-import org.apache.kylin.common.mr.KylinReducer;
+import org.apache.kylin.engine.mr.KylinReducer;
 import org.apache.kylin.job.constant.BatchConstants;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java b/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
index 6c93da6..8a83870 100644
--- a/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
+++ b/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
@@ -19,10 +19,10 @@
 package org.apache.kylin.job.impl.threadpool;
 
 import com.google.common.collect.Maps;
+
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
-import org.apache.kylin.common.lock.JobLock;
 import org.apache.kylin.job.Scheduler;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
@@ -32,6 +32,7 @@ import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.Executable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.execution.Output;
+import org.apache.kylin.job.lock.JobLock;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java b/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java
index b90dd4a..5fc445c 100644
--- a/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java
+++ b/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java
@@ -23,11 +23,9 @@ import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.CubeInstance;
@@ -43,6 +41,7 @@ import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.DictionaryGenerator;
 import org.apache.kylin.dict.DictionaryInfo;
 import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.cube.FactDistinctColumnsReducer;
 import org.apache.kylin.job.hadoop.hbase.CubeHTableUtil;
@@ -53,6 +52,7 @@ import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.source.ReadableTable.TableSignature;
 import org.apache.kylin.storage.cube.CuboidToGridTableMapping;
 import org.apache.kylin.storage.gridtable.GTRecord;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.InMemKeyValueCreator;
 import org.apache.kylin.streaming.MicroStreamBatch;
 import org.apache.kylin.streaming.MicroStreamBatchConsumer;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java b/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java
index 6b9e7dd..7bcdaa5 100644
--- a/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java
+++ b/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java
@@ -38,11 +38,12 @@ import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+
 import kafka.api.OffsetRequest;
 import kafka.cluster.Broker;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.DaemonThreadFactory;
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.Pair;
@@ -58,12 +59,14 @@ import org.apache.kylin.job.monitor.MonitorCLI;
 import org.apache.kylin.job.monitor.StreamingMonitor;
 import org.apache.kylin.metadata.model.IntermediateColumnDesc;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.streaming.*;
 import org.apache.kylin.streaming.invertedindex.IIStreamConsumer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
+
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 196381e..578c7eb 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
@@ -33,15 +33,16 @@ 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.engine.mr.HadoopUtil;
 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.apache.kylin.metadata.realization.RealizationStatusEnum;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -63,7 +64,7 @@ public class DeployCoprocessorCLI {
 
     public static void main(String[] args) throws IOException {
         KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        Configuration hconf = HadoopUtil.newHBaseConfiguration(kylinConfig.getStorageUrl());
+        Configuration hconf = HBaseConnection.newHBaseConfiguration(kylinConfig.getStorageUrl());
         FileSystem fileSystem = FileSystem.get(hconf);
         HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 4e84bf3..b8a452c 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
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.persistence.HBaseConnection;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 
 import com.google.common.collect.Lists;
 


[4/6] incubator-kylin git commit: KYLIN-875 Refactor core-common, remove dependency on hadoop/hbase

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/test/java/org/apache/kylin/common/util/HBaseMiniclusterHelper.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/HBaseMiniclusterHelper.java b/core-common/src/test/java/org/apache/kylin/common/util/HBaseMiniclusterHelper.java
deleted file mode 100644
index fb0d313..0000000
--- a/core-common/src/test/java/org/apache/kylin/common/util/HBaseMiniclusterHelper.java
+++ /dev/null
@@ -1,167 +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.common.util;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.HBaseResourceStore;
-
-/**
- * a helper class to start and shutdown hbase mini cluster
- *
- * @author shaoshi
- */
-public class HBaseMiniclusterHelper {
-
-    public static final String SHARED_STORAGE_PREFIX = "KYLIN_";
-    public static final String CUBE_STORAGE_PREFIX = "KYLIN_";
-    public static final String II_STORAGE_PREFIX = "KYLIN_II_";
-    public static final String TEST_METADATA_TABLE = "kylin_metadata";
-
-    private static final String hbaseTarLocation = "../examples/test_case_data/minicluster/hbase-export.tar.gz";
-    private static final String iiEndpointClassName = "org.apache.kylin.storage.hbase.coprocessor.endpoint.IIEndpoint";
-
-    public static HBaseTestingUtility UTIL = new HBaseTestingUtility();
-    private static volatile boolean clusterStarted = false;
-    private static String hbaseconnectionUrl = "";
-
-    private static final Log logger = LogFactory.getLog(HBaseMiniclusterHelper.class);
-
-    static {
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            @Override
-            public void run() {
-                shutdownMiniCluster();
-            }
-        });
-    }
-
-    /**
-     * Start the minicluster; Sub-classes should invoke this in BeforeClass method.
-     *
-     * @throws Exception
-     */
-    public static void startupMinicluster() throws Exception {
-
-        if (!clusterStarted) {
-            synchronized (HBaseMiniclusterHelper.class) {
-                if (!clusterStarted) {
-                    startupMiniClusterAndImportData();
-                    clusterStarted = true;
-                }
-            }
-        } else {
-            updateKylinConfigWithMinicluster();
-        }
-    }
-
-    private static void updateKylinConfigWithMinicluster() {
-
-        KylinConfig.getInstanceFromEnv().setMetadataUrl(TEST_METADATA_TABLE + "@" + hbaseconnectionUrl);
-        KylinConfig.getInstanceFromEnv().setStorageUrl(hbaseconnectionUrl);
-    }
-
-    private static void startupMiniClusterAndImportData() throws Exception {
-
-        logger.info("Going to start mini cluster.");
-
-        if (existInClassPath(iiEndpointClassName)) {
-            HBaseMiniclusterHelper.UTIL.getConfiguration().setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, iiEndpointClassName);
-        }
-
-        //https://issues.apache.org/jira/browse/HBASE-11711
-        UTIL.getConfiguration().setInt("hbase.master.info.port", -1);//avoid port clobbering
-
-        MiniHBaseCluster hbaseCluster = UTIL.startMiniCluster();
-
-        Configuration config = hbaseCluster.getConf();
-        String host = config.get(HConstants.ZOOKEEPER_QUORUM);
-        String port = config.get(HConstants.ZOOKEEPER_CLIENT_PORT);
-        String parent = config.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
-
-        // see in: https://hbase.apache.org/book.html#trouble.rs.runtime.zkexpired
-        config.set("zookeeper.session.timeout", "1200000");
-        config.set("hbase.zookeeper.property.tickTime", "6000");
-        // reduce rpc retry
-        config.set(HConstants.HBASE_CLIENT_PAUSE, "3000");
-        config.set(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "1");
-        config.set(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, "60000");
-
-        hbaseconnectionUrl = "hbase:" + host + ":" + port + ":" + parent;
-        updateKylinConfigWithMinicluster();
-
-        UTIL.startMiniMapReduceCluster();
-
-        // create the metadata htables;
-        @SuppressWarnings("unused")
-        HBaseResourceStore store = new HBaseResourceStore(KylinConfig.getInstanceFromEnv());
-
-        // import the table content
-        HbaseImporter.importHBaseData(hbaseTarLocation, UTIL.getConfiguration());
-
-    }
-
-    private static boolean existInClassPath(String className) {
-        try {
-            Class.forName(className);
-        } catch (ClassNotFoundException e) {
-            return false;
-        }
-        return true;
-    }
-
-    /**
-     * Shutdown the minicluster; 
-     */
-    public static void shutdownMiniCluster() {
-
-        logger.info("Going to shutdown mini cluster.");
-
-        try {
-            UTIL.shutdownMiniMapReduceCluster();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-
-        try {
-            UTIL.shutdownMiniCluster();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-
-    public static void main(String[] args) {
-        HBaseMiniclusterHelper t = new HBaseMiniclusterHelper();
-        logger.info(t);
-        try {
-            HBaseMetadataTestCase.staticCreateTestMetadata(HBaseMetadataTestCase.MINICLUSTER_TEST_DATA);
-            HBaseMiniclusterHelper.startupMinicluster();
-        } catch (Exception e) {
-            e.printStackTrace();
-        } finally {
-            HBaseMiniclusterHelper.shutdownMiniCluster();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/test/java/org/apache/kylin/common/util/HbaseImporter.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/HbaseImporter.java b/core-common/src/test/java/org/apache/kylin/common/util/HbaseImporter.java
deleted file mode 100644
index 1647d54..0000000
--- a/core-common/src/test/java/org/apache/kylin/common/util/HbaseImporter.java
+++ /dev/null
@@ -1,121 +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.common.util;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.mapreduce.Import;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.util.GenericOptionsParser;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.HBaseConnection;
-
-import com.google.common.base.Preconditions;
-
-/**
- */
-public class HbaseImporter {
-
-    private static final Log logger = LogFactory.getLog(HbaseImporter.class);
-
-    public static void importHBaseData(String hbaseTarLocation, Configuration conf) throws IOException, ClassNotFoundException, InterruptedException {
-
-        if (System.getenv("JAVA_HOME") == null) {
-            logger.error("Didn't find $JAVA_HOME, this will cause HBase data import failed. Please set $JAVA_HOME.");
-            logger.error("Skipping table import...");
-            return;
-        }
-
-        File exportFile = new File(hbaseTarLocation);
-        if (!exportFile.exists()) {
-            logger.error("Didn't find the export achieve file on " + exportFile.getAbsolutePath());
-            return;
-        }
-
-        File folder = new File("/tmp/hbase-export/");
-        if (folder.exists()) {
-            FileUtils.deleteDirectory(folder);
-        }
-        folder.mkdirs();
-        folder.deleteOnExit();
-
-        //TarGZUtil.uncompressTarGZ(exportFile, folder);
-        FileUtil.unTar(exportFile, folder);
-        String[] child = folder.list();
-        Preconditions.checkState(child.length == 1);
-        String backupFolderName = child[0];
-        File backupFolder = new File(folder, backupFolderName);
-        String[] tableNames = backupFolder.list();
-
-        for (String table : tableNames) {
-
-            if (!(table.equalsIgnoreCase(HBaseMiniclusterHelper.TEST_METADATA_TABLE) || table.startsWith(HBaseMiniclusterHelper.SHARED_STORAGE_PREFIX))) {
-                continue;
-            }
-
-            // create the htable; otherwise the import will fail.
-            if (table.startsWith(HBaseMiniclusterHelper.II_STORAGE_PREFIX)) {
-                HBaseConnection.createHTableIfNeeded(KylinConfig.getInstanceFromEnv().getStorageUrl(), table, "f");
-            } else if (table.startsWith(HBaseMiniclusterHelper.CUBE_STORAGE_PREFIX)) {
-                HBaseConnection.createHTableIfNeeded(KylinConfig.getInstanceFromEnv().getStorageUrl(), table, "F1", "F2");
-            }
-
-            // directly import from local fs, no need to copy to hdfs
-            String importLocation = "file://" + backupFolder.getAbsolutePath() + "/" + table;
-            String[] args = new String[] { table, importLocation };
-            boolean result = runImport(args, conf);
-            logger.info("importing table '" + table + "' with result:" + result);
-
-            if (!result)
-                break;
-        }
-
-    }
-
-    private static boolean runImport(String[] args, Configuration configuration) throws IOException, InterruptedException, ClassNotFoundException {
-        // need to make a copy of the configuration because to make sure different temp dirs are used.
-        GenericOptionsParser opts = new GenericOptionsParser(new Configuration(configuration), args);
-        Configuration newConf = opts.getConfiguration();
-        args = opts.getRemainingArgs();
-        Job job = Import.createSubmittableJob(newConf, args);
-        job.waitForCompletion(false);
-        return job.isSuccessful();
-    }
-
-    public static void main(String[] args) throws IOException, InterruptedException, ClassNotFoundException {
-        if (args.length != 1) {
-            logger.error("Usage: HbaseImporter hbase_tar_lcoation");
-            System.exit(-1);
-        }
-
-        logger.info("The KylinConfig being used:");
-        logger.info("=================================================");
-        KylinConfig.getInstanceFromEnv().printProperties();
-        logger.info("=================================================");
-
-        importHBaseData(args[0], HBaseConfiguration.create());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/test/java/org/apache/kylin/common/util/IdentityUtilTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/IdentityUtilTest.java b/core-common/src/test/java/org/apache/kylin/common/util/IdentityUtilTest.java
index 7e802a4..452c5e6 100644
--- a/core-common/src/test/java/org/apache/kylin/common/util/IdentityUtilTest.java
+++ b/core-common/src/test/java/org/apache/kylin/common/util/IdentityUtilTest.java
@@ -1,11 +1,11 @@
 package org.apache.kylin.common.util;
 
-import com.google.common.collect.Lists;
-import org.apache.commons.collections.CollectionUtils;
+import java.util.List;
+
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.util.List;
+import com.google.common.collect.Lists;
 
 /**
  */
@@ -20,9 +20,6 @@ public class IdentityUtilTest {
         List<String> c2 = Lists.newArrayList(s2);
         List<String> c3 = Lists.newArrayList(s2);
 
-        Assert.assertTrue(CollectionUtils.isEqualCollection(c1,c2));
-        Assert.assertTrue(CollectionUtils.isEqualCollection(c3,c2));
-
         Assert.assertFalse(IdentityUtils.collectionReferenceEquals(c1,c2));
         Assert.assertTrue(IdentityUtils.collectionReferenceEquals(c3,c2));
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/test/java/org/apache/kylin/common/util/RandomSamplerTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/RandomSamplerTest.java b/core-common/src/test/java/org/apache/kylin/common/util/RandomSamplerTest.java
index 89d5701..1cf1e8b 100644
--- a/core-common/src/test/java/org/apache/kylin/common/util/RandomSamplerTest.java
+++ b/core-common/src/test/java/org/apache/kylin/common/util/RandomSamplerTest.java
@@ -23,24 +23,21 @@ import static org.junit.Assert.*;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 /**
- * @author ysong1
- * 
  */
 public class RandomSamplerTest {
 
     @Test
     public void test() {
-        RandomSampler<Text> s = new RandomSampler<Text>();
-        List<Text> data = new ArrayList<Text>();
+        RandomSampler<String> s = new RandomSampler<String>();
+        List<String> data = new ArrayList<String>();
         for (int i = 0; i < 1000; i++) {
-            data.add(new Text(String.valueOf(i)));
+            data.add(String.valueOf(i));
         }
 
-        List<Text> result = s.sample(data, 50);
+        List<String> result = s.sample(data, 50);
         System.out.println(result);
         assertEquals(50, result.size());
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/test/java/org/apache/kylin/common/util/RangeTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/RangeTest.java b/core-common/src/test/java/org/apache/kylin/common/util/RangeTest.java
index 2a9ce40..e18b3e7 100644
--- a/core-common/src/test/java/org/apache/kylin/common/util/RangeTest.java
+++ b/core-common/src/test/java/org/apache/kylin/common/util/RangeTest.java
@@ -9,6 +9,7 @@ import com.google.common.collect.Ranges;
 
 /**
  */
+@SuppressWarnings({ "rawtypes", "unchecked" })
 public class RangeTest {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/test/java/org/apache/kylin/common/util/SSHClientTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/SSHClientTest.java b/core-common/src/test/java/org/apache/kylin/common/util/SSHClientTest.java
index 5680234..3f32b86 100644
--- a/core-common/src/test/java/org/apache/kylin/common/util/SSHClientTest.java
+++ b/core-common/src/test/java/org/apache/kylin/common/util/SSHClientTest.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.*;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.hadoop.fs.FileUtil;
+import org.apache.commons.io.FileUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.junit.After;
 import org.junit.Before;
@@ -80,7 +80,9 @@ public class SSHClientTest extends LocalFileMetadataTestCase {
             return;
 
         SSHClient ssh = new SSHClient(this.hostname, this.port, this.username, this.password);
-        File tmpFile = FileUtil.createLocalTempFile(new File("/tmp/test_scp"), "temp_", false);
+        File tmpFile = File.createTempFile("test_scp", "", new File("/tmp"));
+        tmpFile.deleteOnExit();
+        FileUtils.write(tmpFile, "test_scp");
         ssh.scpFileToRemote(tmpFile.getAbsolutePath(), "/tmp");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-cube/pom.xml
----------------------------------------------------------------------
diff --git a/core-cube/pom.xml b/core-cube/pom.xml
index 711e4db..8751a78 100644
--- a/core-cube/pom.xml
+++ b/core-cube/pom.xml
@@ -43,6 +43,8 @@
             <artifactId>kylin-core-dictionary</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
+
+        <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-core-common</artifactId>
@@ -50,8 +52,6 @@
             <scope>test</scope>
             <version>${project.parent.version}</version>
         </dependency>
-
-        <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index cc61dac..6ed4100 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -44,11 +44,11 @@ import org.apache.kylin.cube.model.DimensionDesc;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.DictionaryInfo;
 import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.dict.DistinctColumnValuesProvider;
 import org.apache.kylin.dict.lookup.LookupStringTable;
 import org.apache.kylin.dict.lookup.SnapshotManager;
 import org.apache.kylin.dict.lookup.SnapshotTable;
 import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.PartitionDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -161,13 +161,13 @@ public class CubeManager implements IRealizationProvider {
         return result;
     }
 
-    public DictionaryInfo buildDictionary(CubeSegment cubeSeg, TblColRef col, String factColumnsPath) throws IOException {
+    public DictionaryInfo buildDictionary(CubeSegment cubeSeg, TblColRef col, DistinctColumnValuesProvider factTableValueProvider) throws IOException {
         CubeDesc cubeDesc = cubeSeg.getCubeDesc();
         if (!cubeDesc.getRowkey().isUseDictionary(col))
             return null;
 
         DictionaryManager dictMgr = getDictionaryManager();
-        DictionaryInfo dictInfo = dictMgr.buildDictionary(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, factColumnsPath);
+        DictionaryInfo dictInfo = dictMgr.buildDictionary(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, factTableValueProvider);
 
         if (dictInfo != null) {
             cubeSeg.putDictResPath(col, dictInfo.getResourcePath());
@@ -708,8 +708,6 @@ public class CubeManager implements IRealizationProvider {
      * - Favors big segments over the small
      */
     private List<CubeSegment> calculateToBeSegments(CubeInstance cube, boolean strictChecking, CubeSegment... newSegments) {
-        CubeDesc cubeDesc = cube.getDescriptor();
-        PartitionDesc partDesc = cubeDesc.getModel().getPartitionDesc();
 
         List<CubeSegment> tobe = Lists.newArrayList(cube.getSegments());
         if (newSegments != null)

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java b/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
index feb0f1a..e375a19 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
@@ -23,11 +23,11 @@ import java.io.IOException;
 import org.apache.kylin.cube.model.DimensionDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.dict.DistinctColumnValuesProvider;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -35,14 +35,14 @@ public class DictionaryGeneratorCLI {
 
     private static final Logger logger = LoggerFactory.getLogger(DictionaryGeneratorCLI.class);
 
-    public static void processSegment(KylinConfig config, String cubeName, String segmentName, String factColumnsPath) throws IOException {
+    public static void processSegment(KylinConfig config, String cubeName, String segmentName, DistinctColumnValuesProvider factTableValueProvider) throws IOException {
         CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
         CubeSegment segment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
 
-        processSegment(config, segment, factColumnsPath);
+        processSegment(config, segment, factTableValueProvider);
     }
 
-    private static void processSegment(KylinConfig config, CubeSegment cubeSeg, String factColumnsPath) throws IOException {
+    private static void processSegment(KylinConfig config, CubeSegment cubeSeg, DistinctColumnValuesProvider factTableValueProvider) throws IOException {
         CubeManager cubeMgr = CubeManager.getInstance(config);
 
         for (DimensionDesc dim : cubeSeg.getCubeDesc().getDimensions()) {
@@ -50,7 +50,7 @@ public class DictionaryGeneratorCLI {
             for (TblColRef col : dim.getColumnRefs()) {
                 if (cubeSeg.getCubeDesc().getRowkey().isUseDictionary(col)) {
                     logger.info("Building dictionary for " + col);
-                    cubeMgr.buildDictionary(cubeSeg, col, factColumnsPath);
+                    cubeMgr.buildDictionary(cubeSeg, col, factTableValueProvider);
                 }
             }
 
@@ -60,8 +60,7 @@ public class DictionaryGeneratorCLI {
                 logger.info("Building snapshot of " + dim.getTable());
                 cubeMgr.buildSnapshotTable(cubeSeg, dim.getTable());
                 logger.info("Checking snapshot of " + dim.getTable());
-                cubeMgr.getLookupTable(cubeSeg, dim); // load the table for
-                                                      // sanity check
+                cubeMgr.getLookupTable(cubeSeg, dim); // load the table for sanity check
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-dictionary/pom.xml
----------------------------------------------------------------------
diff --git a/core-dictionary/pom.xml b/core-dictionary/pom.xml
index 14b54a0..649f58b 100644
--- a/core-dictionary/pom.xml
+++ b/core-dictionary/pom.xml
@@ -40,6 +40,8 @@
             <artifactId>kylin-core-metadata</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
+
+        <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-core-common</artifactId>
@@ -47,8 +49,6 @@
             <scope>test</scope>
             <version>${project.parent.version}</version>
         </dependency>
-
-        <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
index 611251a..a579abf 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
@@ -18,23 +18,14 @@
 
 package org.apache.kylin.dict;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.commons.compress.utils.IOUtils;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.dict.lookup.FileTable;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.DataType;
@@ -195,11 +186,11 @@ public class DictionaryManager {
         }
     }
 
-    public DictionaryInfo buildDictionary(DataModelDesc model, String dict, TblColRef col, String factColumnsPath) throws IOException {
+    public DictionaryInfo buildDictionary(DataModelDesc model, String dict, TblColRef col, DistinctColumnValuesProvider factTableValueProvider) throws IOException {
 
         logger.info("building dictionary for " + col);
 
-        Object[] tmp = decideSourceData(model, dict, col, factColumnsPath);
+        Object[] tmp = decideSourceData(model, dict, col, factTableValueProvider);
         String srcTable = (String) tmp[0];
         String srcCol = (String) tmp[1];
         int srcColIdx = (Integer) tmp[2];
@@ -232,93 +223,42 @@ public class DictionaryManager {
      * 4. ReadableTable object
      */
 
-    public Object[] decideSourceData(DataModelDesc model, String dict, TblColRef col, String factColumnsPath) throws IOException {
+    public Object[] decideSourceData(DataModelDesc model, String dict, TblColRef col, DistinctColumnValuesProvider factTableValueProvider) throws IOException {
         String srcTable;
         String srcCol;
         int srcColIdx;
         ReadableTable table;
         MetadataManager metaMgr = MetadataManager.getInstance(config);
 
-        // case of full table (dict on fact table)
-        if (model == null) {
-            srcTable = col.getTable();
-            srcCol = col.getName();
-            srcColIdx = col.getColumnDesc().getZeroBasedIndex();
-            int nColumns = metaMgr.getTableDesc(col.getTable()).getColumnCount();
-            table = new FileTable(factColumnsPath + "/" + col.getName(), nColumns);
-            return new Object[] { srcTable, srcCol, srcColIdx, table };
-        }
-
         // Decide source data of dictionary:
         // 1. If 'useDict' specifies pre-defined data set, use that
         // 2. Otherwise find a lookup table to scan through
 
         // Note FK on fact table is supported by scan the related PK on lookup table
 
-        //String useDict = cube.getRowkey().getDictionary(col);
-
         // normal case, source from lookup table
         if ("true".equals(dict) || "string".equals(dict) || "number".equals(dict) || "any".equals(dict)) {
             // FK on fact table and join type is inner, use PK from lookup instead
             if (model.isFactTable(col.getTable())) {
                 TblColRef pkCol = model.findPKByFK(col, "inner");
                 if (pkCol != null)
-                    col = pkCol; // scan the counterparty PK on lookup table
-                // instead
+                    col = pkCol; // scan the counterparty PK on lookup table instead
             }
             srcTable = col.getTable();
             srcCol = col.getName();
             srcColIdx = col.getColumnDesc().getZeroBasedIndex();
             if (model.isFactTable(col.getTable())) {
-                table = new FileTable(factColumnsPath + "/" + col.getName(), -1);
+                table = factTableValueProvider.getDistinctValuesFor(col);
             } else {
                 table = TableSourceFactory.createReadableTable(metaMgr.getTableDesc(col.getTable()));
             }
         }
-        // otherwise could refer to a data set, e.g. common_indicators.txt
-        // (LEGACY PATH, since distinct values are collected from fact table)
-        else {
-            String dictDataSetPath = unpackDataSet(this.config.getTempHDFSDir(), dict);
-            if (dictDataSetPath == null)
-                throw new IllegalArgumentException("Unknown dictionary data set '" + dict + "', referred from " + col);
-            srcTable = "PREDEFINED";
-            srcCol = dict;
-            srcColIdx = 0;
-            table = new FileTable(dictDataSetPath, -1);
-        }
+        else
+            throw new IllegalArgumentException("Unknown dictionary value: " + dict);
 
         return new Object[] { srcTable, srcCol, srcColIdx, table };
     }
 
-    private String unpackDataSet(String tempHDFSDir, String dataSetName) throws IOException {
-
-        InputStream in = this.getClass().getResourceAsStream("/org/apache/kylin/dict/" + dataSetName + ".txt");
-        if (in == null) // data set resource not found
-            return null;
-
-        ByteArrayOutputStream buf = new ByteArrayOutputStream();
-        IOUtils.copy(in, buf);
-        in.close();
-        byte[] bytes = buf.toByteArray();
-
-        Path tmpDataSetPath = new Path(tempHDFSDir + "/dict/temp_dataset/" + dataSetName + "_" + bytes.length + ".txt");
-
-        FileSystem fs = HadoopUtil.getFileSystem(tempHDFSDir);
-        boolean writtenNewFile = false;
-        if (fs.exists(tmpDataSetPath) == false || fs.getFileStatus(tmpDataSetPath).getLen() != bytes.length) {
-            fs.mkdirs(tmpDataSetPath.getParent());
-            FSDataOutputStream out = fs.create(tmpDataSetPath);
-            IOUtils.copy(new ByteArrayInputStream(bytes), out);
-            out.close();
-            writtenNewFile = true;
-        }
-
-        String qualifiedPath = tmpDataSetPath.makeQualified(fs.getUri(), new Path("/")).toString();
-        if (writtenNewFile)
-            logger.info("Dictionary temp data set file written to " + qualifiedPath);
-        return qualifiedPath;
-    }
-
     private String checkDupByInfo(DictionaryInfo dictInfo) throws IOException {
         ResourceStore store = MetadataManager.getInstance(config).getStore();
         ArrayList<String> existings = store.listResources(dictInfo.getResourceDir());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-dictionary/src/main/java/org/apache/kylin/dict/DistinctColumnValuesProvider.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DistinctColumnValuesProvider.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DistinctColumnValuesProvider.java
new file mode 100644
index 0000000..66511be
--- /dev/null
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DistinctColumnValuesProvider.java
@@ -0,0 +1,17 @@
+package org.apache.kylin.dict;
+
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.source.ReadableTable;
+
+/**
+ * To build dictionary, we need a list of distinct values on a column.
+ * For column on lookup table, simply scan the whole table since the table is small.
+ * For column on fact table, the fact table is too big to iterate. So the build
+ * engine will first extract distinct values (by a MR job for example), and
+ * implement this interface to provide the result to DictionaryManager.
+ */
+public interface DistinctColumnValuesProvider {
+
+    /** Return a ReadableTable contains only one column, each row being a distinct value. */
+    public ReadableTable getDistinctValuesFor(TblColRef col);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTable.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTable.java b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTable.java
deleted file mode 100644
index 59eca4a..0000000
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTable.java
+++ /dev/null
@@ -1,100 +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 java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.source.ReadableTable;
-
-/**
- */
-public class FileTable implements ReadableTable {
-
-    public static final String DELIM_AUTO = "auto";
-    public static final String DELIM_COMMA = ",";
-
-    String path;
-    String delim;
-    int nColumns;
-
-    public FileTable(String path, int nColumns) {
-        this(path, DELIM_AUTO, nColumns);
-    }
-
-    public FileTable(String path, String delim, int nColumns) {
-        this.path = path;
-        this.delim = delim;
-        this.nColumns = nColumns;
-    }
-
-    public String getColumnDelimeter() {
-        return delim;
-    }
-
-    @Override
-    public TableReader getReader() throws IOException {
-        return new FileTableReader(path, delim, nColumns);
-    }
-
-    @Override
-    public TableSignature getSignature() throws IOException {
-        try {
-            Pair<Long, Long> sizeAndLastModified = getSizeAndLastModified(path);
-            return new TableSignature(path, sizeAndLastModified.getFirst(), sizeAndLastModified.getSecond());
-        } catch (FileNotFoundException ex) {
-            return null;
-        }
-    }
-
-    @Override
-    public String toString() {
-        return path;
-    }
-
-    public static Pair<Long, Long> getSizeAndLastModified(String path) throws IOException {
-        FileSystem fs = HadoopUtil.getFileSystem(path);
-
-        // get all contained files if path is directory
-        ArrayList<FileStatus> allFiles = new ArrayList<>();
-        FileStatus status = fs.getFileStatus(new Path(path));
-        if (status.isFile()) {
-            allFiles.add(status);
-        } else {
-            FileStatus[] listStatus = fs.listStatus(new Path(path));
-            allFiles.addAll(Arrays.asList(listStatus));
-        }
-
-        long size = 0;
-        long lastModified = 0;
-        for (FileStatus file : allFiles) {
-            size += file.getLen();
-            lastModified = Math.max(lastModified, file.getModificationTime());
-        }
-
-        return new Pair<Long, Long>(size, lastModified);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTableReader.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTableReader.java b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTableReader.java
deleted file mode 100644
index 4e04c93..0000000
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/FileTableReader.java
+++ /dev/null
@@ -1,220 +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 java.io.BufferedReader;
-import java.io.Closeable;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStreamReader;
-
-import org.apache.commons.lang.StringEscapeUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.common.util.StringSplitter;
-import org.apache.kylin.source.ReadableTable.TableReader;
-
-/**
- * Tables are typically CSV or SEQ file.
- * 
- * @author yangli9
- */
-public class FileTableReader implements TableReader {
-
-    private static final Logger logger = LoggerFactory.getLogger(FileTableReader.class);
-    private static final char CSV_QUOTE = '"';
-    private static final String[] DETECT_DELIMS = new String[] { "\177", "|", "\t", "," };
-
-    private String filePath;
-    private String delim;
-    private RowReader reader;
-
-    private String curLine;
-    private String[] curColumns;
-    private int expectedColumnNumber = -1; // helps delimiter detection
-
-    public FileTableReader(String filePath, int expectedColumnNumber) throws IOException {
-        this(filePath, FileTable.DELIM_AUTO, expectedColumnNumber);
-    }
-    
-    public FileTableReader(String filePath, String delim, int expectedColumnNumber) throws IOException {
-        filePath = HadoopUtil.fixWindowsPath(filePath);
-        this.filePath = filePath;
-        this.delim = delim;
-        this.expectedColumnNumber = expectedColumnNumber;
-
-        FileSystem fs = HadoopUtil.getFileSystem(filePath);
-
-        try {
-            this.reader = new SeqRowReader(HadoopUtil.getCurrentConfiguration(), fs, filePath);
-
-        } catch (IOException e) {
-            if (isExceptionSayingNotSeqFile(e) == false)
-                throw e;
-
-            this.reader = new CsvRowReader(fs, filePath);
-        }
-    }
-
-    private boolean isExceptionSayingNotSeqFile(IOException e) {
-        if (e.getMessage() != null && e.getMessage().contains("not a SequenceFile"))
-            return true;
-
-        if (e instanceof EOFException) // in case the file is very very small
-            return true;
-
-        return false;
-    }
-
-    @Override
-    public boolean next() throws IOException {
-        curLine = reader.nextLine();
-        curColumns = null;
-        return curLine != null;
-    }
-
-    public String getLine() {
-        return curLine;
-    }
-
-    @Override
-    public String[] getRow() {
-        if (curColumns == null) {
-            if (FileTable.DELIM_AUTO.equals(delim))
-                delim = autoDetectDelim(curLine);
-
-            if (delim == null)
-                curColumns = new String[] { curLine };
-            else
-                curColumns = split(curLine, delim);
-        }
-        return curColumns;
-    }
-
-    private String[] split(String line, String delim) {
-        // FIXME CVS line should be parsed considering escapes
-        String str[] = StringSplitter.split(line, delim);
-
-        // un-escape CSV
-        if (FileTable.DELIM_COMMA.equals(delim)) {
-            for (int i = 0; i < str.length; i++) {
-                str[i] = unescapeCsv(str[i]);
-            }
-        }
-
-        return str;
-    }
-
-    private String unescapeCsv(String str) {
-        if (str == null || str.length() < 2)
-            return str;
-
-        str = StringEscapeUtils.unescapeCsv(str);
-
-        // unescapeCsv may not remove the outer most quotes
-        if (str.charAt(0) == CSV_QUOTE && str.charAt(str.length() - 1) == CSV_QUOTE)
-            str = str.substring(1, str.length() - 1);
-
-        return str;
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (reader != null)
-            reader.close();
-    }
-
-    private String autoDetectDelim(String line) {
-        if (expectedColumnNumber > 0) {
-            for (String delim : DETECT_DELIMS) {
-                if (StringSplitter.split(line, delim).length == expectedColumnNumber) {
-                    logger.info("Auto detect delim to be '" + delim + "', split line to " + expectedColumnNumber + " columns -- " + line);
-                    return delim;
-                }
-            }
-        }
-
-        logger.info("Auto detect delim to be null, will take THE-WHOLE-LINE as a single value, for " + filePath);
-        return null;
-    }
-
-    // ============================================================================
-
-    private interface RowReader extends Closeable {
-        String nextLine() throws IOException; // return null on EOF
-    }
-
-    private class SeqRowReader implements RowReader {
-        Reader reader;
-        Writable key;
-        Text value;
-
-        SeqRowReader(Configuration hconf, FileSystem fs, String path) throws IOException {
-            reader = new Reader(hconf, SequenceFile.Reader.file(new Path(path)));
-            key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), hconf);
-            value = new Text();
-        }
-
-        @Override
-        public String nextLine() throws IOException {
-            boolean hasNext = reader.next(key, value);
-            if (hasNext)
-                return Bytes.toString(value.getBytes(), 0, value.getLength());
-            else
-                return null;
-        }
-
-        @Override
-        public void close() throws IOException {
-            reader.close();
-        }
-    }
-
-    private class CsvRowReader implements RowReader {
-        BufferedReader reader;
-
-        CsvRowReader(FileSystem fs, String path) throws IOException {
-            FSDataInputStream in = fs.open(new Path(path));
-            reader = new BufferedReader(new InputStreamReader(in, "UTF-8"));
-        }
-
-        @Override
-        public String nextLine() throws IOException {
-            return reader.readLine();
-        }
-
-        @Override
-        public void close() throws IOException {
-            reader.close();
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-dictionary/src/test/java/org/apache/kylin/dict/LookupTableTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/LookupTableTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/LookupTableTest.java
deleted file mode 100644
index 86fa635..0000000
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/LookupTableTest.java
+++ /dev/null
@@ -1,76 +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;
-
-import java.io.File;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.dict.lookup.FileTable;
-import org.apache.kylin.dict.lookup.LookupBytesTable;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.kylin.common.util.Array;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.TableDesc;
-
-/**
- * @author yangli9
- */
-public class LookupTableTest extends LocalFileMetadataTestCase {
-
-    @Before
-    public void setup() throws Exception {
-        createTestMetadata();
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-    }
-
-    @Test
-    public void testBasic() throws Exception {
-        TableDesc siteTable = MetadataManager.getInstance(getTestConfig()).getTableDesc("EDW.TEST_SITES");
-        TableDesc categoryTable = MetadataManager.getInstance(getTestConfig()).getTableDesc("DEFAULT.test_category_groupings");
-        LookupBytesTable lookup;
-
-        System.out.println("============================================================================");
-
-        File f = new File(LOCALMETA_TEST_DATA + "/data/EDW.TEST_SITES.csv");
-        lookup = new LookupBytesTable(siteTable, new String[] { "SITE_ID" }, new FileTable("file://" + f.getAbsolutePath(), 10));
-        lookup.dump();
-
-        System.out.println("============================================================================");
-
-        f = new File(LOCALMETA_TEST_DATA + "/data/DEFAULT.TEST_CATEGORY_GROUPINGS.csv");
-        lookup = new LookupBytesTable(categoryTable, new String[] { "leaf_categ_id", "site_id" }, new FileTable("file://" + f.getAbsolutePath(), 36));
-        lookup.dump();
-
-        System.out.println("============================================================================");
-
-        ByteArray k1 = new ByteArray(Bytes.toBytes("533"));
-        ByteArray k2 = new ByteArray(Bytes.toBytes("0"));
-        Array<ByteArray> key = new Array<ByteArray>(new ByteArray[] { k1, k2 });
-        System.out.println(lookup.getRow(key));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-dictionary/src/test/java/org/apache/kylin/dict/TableReaderTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TableReaderTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TableReaderTest.java
deleted file mode 100644
index cfecaee..0000000
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TableReaderTest.java
+++ /dev/null
@@ -1,48 +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;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.kylin.dict.lookup.FileTable;
-import org.apache.kylin.dict.lookup.FileTableReader;
-import org.junit.Test;
-
-/**
- * @author yangli9
- * 
- */
-public class TableReaderTest {
-
-    @Test
-    public void testBasicReader() throws IOException {
-        File f = new File("src/test/resources/dict/DW_SITES");
-        FileTableReader reader = new FileTableReader("file://" + f.getAbsolutePath(), FileTable.DELIM_AUTO, 10);
-        while (reader.next()) {
-            assertEquals("[-1, Korea Auction.co.kr, S, 48, 0, 111, 2009-02-11, , DW_OFFPLAT, ]", Arrays.toString(reader.getRow()));
-            break;
-        }
-        reader.close();
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-job/src/main/java/org/apache/kylin/job/lock/JobLock.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/lock/JobLock.java b/core-job/src/main/java/org/apache/kylin/job/lock/JobLock.java
new file mode 100644
index 0000000..490e3f7
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/lock/JobLock.java
@@ -0,0 +1,9 @@
+package org.apache.kylin.job.lock;
+
+/**
+ */
+public interface JobLock {
+    boolean lock();
+
+    void unlock();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-job/src/main/java/org/apache/kylin/job/lock/MockJobLock.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/lock/MockJobLock.java b/core-job/src/main/java/org/apache/kylin/job/lock/MockJobLock.java
new file mode 100644
index 0000000..f5cea4e
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/lock/MockJobLock.java
@@ -0,0 +1,15 @@
+package org.apache.kylin.job.lock;
+
+/**
+ */
+public class MockJobLock implements JobLock {
+    @Override
+    public boolean lock() {
+        return true;
+    }
+
+    @Override
+    public void unlock() {
+        return;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-metadata/pom.xml
----------------------------------------------------------------------
diff --git a/core-metadata/pom.xml b/core-metadata/pom.xml
index 3c4f180..71457bc 100644
--- a/core-metadata/pom.xml
+++ b/core-metadata/pom.xml
@@ -40,6 +40,8 @@
             <artifactId>kylin-core-common</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
+
+        <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-core-common</artifactId>
@@ -47,9 +49,6 @@
             <scope>test</scope>
             <version>${project.parent.version}</version>
         </dependency>
-
-        <!-- Env & Test -->
-
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/engine-mr/pom.xml
----------------------------------------------------------------------
diff --git a/engine-mr/pom.xml b/engine-mr/pom.xml
index 748a9df..a226e80 100644
--- a/engine-mr/pom.xml
+++ b/engine-mr/pom.xml
@@ -53,6 +53,46 @@
 
         <!-- Env & Test -->
         <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>provided</scope>
+            <!-- protobuf version conflict with hbase -->
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-app</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+        </dependency>
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/engine-mr/src/main/java/org/apache/kylin/engine/mr/DFSFileTable.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/DFSFileTable.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/DFSFileTable.java
new file mode 100644
index 0000000..152b4af
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/DFSFileTable.java
@@ -0,0 +1,99 @@
+/*
+ * 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.engine.mr;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.source.ReadableTable;
+
+/**
+ */
+public class DFSFileTable implements ReadableTable {
+
+    public static final String DELIM_AUTO = "auto";
+    public static final String DELIM_COMMA = ",";
+
+    String path;
+    String delim;
+    int nColumns;
+
+    public DFSFileTable(String path, int nColumns) {
+        this(path, DELIM_AUTO, nColumns);
+    }
+
+    public DFSFileTable(String path, String delim, int nColumns) {
+        this.path = path;
+        this.delim = delim;
+        this.nColumns = nColumns;
+    }
+
+    public String getColumnDelimeter() {
+        return delim;
+    }
+
+    @Override
+    public TableReader getReader() throws IOException {
+        return new DFSFileTableReader(path, delim, nColumns);
+    }
+
+    @Override
+    public TableSignature getSignature() throws IOException {
+        try {
+            Pair<Long, Long> sizeAndLastModified = getSizeAndLastModified(path);
+            return new TableSignature(path, sizeAndLastModified.getFirst(), sizeAndLastModified.getSecond());
+        } catch (FileNotFoundException ex) {
+            return null;
+        }
+    }
+
+    @Override
+    public String toString() {
+        return path;
+    }
+
+    public static Pair<Long, Long> getSizeAndLastModified(String path) throws IOException {
+        FileSystem fs = HadoopUtil.getFileSystem(path);
+
+        // get all contained files if path is directory
+        ArrayList<FileStatus> allFiles = new ArrayList<>();
+        FileStatus status = fs.getFileStatus(new Path(path));
+        if (status.isFile()) {
+            allFiles.add(status);
+        } else {
+            FileStatus[] listStatus = fs.listStatus(new Path(path));
+            allFiles.addAll(Arrays.asList(listStatus));
+        }
+
+        long size = 0;
+        long lastModified = 0;
+        for (FileStatus file : allFiles) {
+            size += file.getLen();
+            lastModified = Math.max(lastModified, file.getModificationTime());
+        }
+
+        return new Pair<Long, Long>(size, lastModified);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/engine-mr/src/main/java/org/apache/kylin/engine/mr/DFSFileTableReader.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/DFSFileTableReader.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/DFSFileTableReader.java
new file mode 100644
index 0000000..51fb734
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/DFSFileTableReader.java
@@ -0,0 +1,219 @@
+/*
+ * 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.engine.mr;
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.kylin.common.util.StringSplitter;
+import org.apache.kylin.source.ReadableTable.TableReader;
+
+/**
+ * Tables are typically CSV or SEQ file.
+ * 
+ * @author yangli9
+ */
+public class DFSFileTableReader implements TableReader {
+
+    private static final Logger logger = LoggerFactory.getLogger(DFSFileTableReader.class);
+    private static final char CSV_QUOTE = '"';
+    private static final String[] DETECT_DELIMS = new String[] { "\177", "|", "\t", "," };
+
+    private String filePath;
+    private String delim;
+    private RowReader reader;
+
+    private String curLine;
+    private String[] curColumns;
+    private int expectedColumnNumber = -1; // helps delimiter detection
+
+    public DFSFileTableReader(String filePath, int expectedColumnNumber) throws IOException {
+        this(filePath, DFSFileTable.DELIM_AUTO, expectedColumnNumber);
+    }
+    
+    public DFSFileTableReader(String filePath, String delim, int expectedColumnNumber) throws IOException {
+        filePath = HadoopUtil.fixWindowsPath(filePath);
+        this.filePath = filePath;
+        this.delim = delim;
+        this.expectedColumnNumber = expectedColumnNumber;
+
+        FileSystem fs = HadoopUtil.getFileSystem(filePath);
+
+        try {
+            this.reader = new SeqRowReader(HadoopUtil.getCurrentConfiguration(), fs, filePath);
+
+        } catch (IOException e) {
+            if (isExceptionSayingNotSeqFile(e) == false)
+                throw e;
+
+            this.reader = new CsvRowReader(fs, filePath);
+        }
+    }
+
+    private boolean isExceptionSayingNotSeqFile(IOException e) {
+        if (e.getMessage() != null && e.getMessage().contains("not a SequenceFile"))
+            return true;
+
+        if (e instanceof EOFException) // in case the file is very very small
+            return true;
+
+        return false;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+        curLine = reader.nextLine();
+        curColumns = null;
+        return curLine != null;
+    }
+
+    public String getLine() {
+        return curLine;
+    }
+
+    @Override
+    public String[] getRow() {
+        if (curColumns == null) {
+            if (DFSFileTable.DELIM_AUTO.equals(delim))
+                delim = autoDetectDelim(curLine);
+
+            if (delim == null)
+                curColumns = new String[] { curLine };
+            else
+                curColumns = split(curLine, delim);
+        }
+        return curColumns;
+    }
+
+    private String[] split(String line, String delim) {
+        // FIXME CVS line should be parsed considering escapes
+        String str[] = StringSplitter.split(line, delim);
+
+        // un-escape CSV
+        if (DFSFileTable.DELIM_COMMA.equals(delim)) {
+            for (int i = 0; i < str.length; i++) {
+                str[i] = unescapeCsv(str[i]);
+            }
+        }
+
+        return str;
+    }
+
+    private String unescapeCsv(String str) {
+        if (str == null || str.length() < 2)
+            return str;
+
+        str = StringEscapeUtils.unescapeCsv(str);
+
+        // unescapeCsv may not remove the outer most quotes
+        if (str.charAt(0) == CSV_QUOTE && str.charAt(str.length() - 1) == CSV_QUOTE)
+            str = str.substring(1, str.length() - 1);
+
+        return str;
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (reader != null)
+            reader.close();
+    }
+
+    private String autoDetectDelim(String line) {
+        if (expectedColumnNumber > 0) {
+            for (String delim : DETECT_DELIMS) {
+                if (StringSplitter.split(line, delim).length == expectedColumnNumber) {
+                    logger.info("Auto detect delim to be '" + delim + "', split line to " + expectedColumnNumber + " columns -- " + line);
+                    return delim;
+                }
+            }
+        }
+
+        logger.info("Auto detect delim to be null, will take THE-WHOLE-LINE as a single value, for " + filePath);
+        return null;
+    }
+
+    // ============================================================================
+
+    private interface RowReader extends Closeable {
+        String nextLine() throws IOException; // return null on EOF
+    }
+
+    private class SeqRowReader implements RowReader {
+        Reader reader;
+        Writable key;
+        Text value;
+
+        SeqRowReader(Configuration hconf, FileSystem fs, String path) throws IOException {
+            reader = new Reader(hconf, SequenceFile.Reader.file(new Path(path)));
+            key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), hconf);
+            value = new Text();
+        }
+
+        @Override
+        public String nextLine() throws IOException {
+            boolean hasNext = reader.next(key, value);
+            if (hasNext)
+                return Bytes.toString(value.getBytes(), 0, value.getLength());
+            else
+                return null;
+        }
+
+        @Override
+        public void close() throws IOException {
+            reader.close();
+        }
+    }
+
+    private class CsvRowReader implements RowReader {
+        BufferedReader reader;
+
+        CsvRowReader(FileSystem fs, String path) throws IOException {
+            FSDataInputStream in = fs.open(new Path(path));
+            reader = new BufferedReader(new InputStreamReader(in, "UTF-8"));
+        }
+
+        @Override
+        public String nextLine() throws IOException {
+            return reader.readLine();
+        }
+
+        @Override
+        public void close() throws IOException {
+            reader.close();
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/engine-mr/src/main/java/org/apache/kylin/engine/mr/HadoopUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/HadoopUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/HadoopUtil.java
new file mode 100644
index 0000000..1c00993
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/HadoopUtil.java
@@ -0,0 +1,109 @@
+/*
+ * 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.engine.mr;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.io.Writable;
+
+public class HadoopUtil {
+
+    private static ThreadLocal<Configuration> hadoopConfig = new ThreadLocal<>();
+
+    public static void setCurrentConfiguration(Configuration conf) {
+        hadoopConfig.set(conf);
+    }
+
+    public static Configuration getCurrentConfiguration() {
+        if (hadoopConfig.get() == null) {
+            hadoopConfig.set(new Configuration());
+        }
+        return hadoopConfig.get();
+    }
+
+    public static FileSystem getFileSystem(String path) throws IOException {
+        return FileSystem.get(makeURI(path), getCurrentConfiguration());
+    }
+
+    public static URI makeURI(String filePath) {
+        try {
+            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 Configuration newHadoopJobConfiguration() {
+        Configuration conf = new Configuration();
+        conf.set(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, "8");
+        return conf;
+    }
+
+    /**
+     * @param table the identifier of hive table, in format <db_name>.<table_name>
+     * @return a string array with 2 elements: {"db_name", "table_name"}
+     */
+    public static String[] parseHiveTableName(String table) {
+        int cut = table.indexOf('.');
+        String database = cut >= 0 ? table.substring(0, cut).trim() : "DEFAULT";
+        String tableName = cut >= 0 ? table.substring(cut + 1).trim() : table.trim();
+
+        return new String[] { database, tableName };
+    }
+
+    public static void deletePath(Configuration conf, Path path) throws IOException {
+        FileSystem fs = FileSystem.get(conf);
+        if (fs.exists(path)) {
+            fs.delete(path, true);
+        }
+    }
+
+    public static byte[] toBytes(Writable writable) {
+        try {
+            ByteArrayOutputStream bout = new ByteArrayOutputStream();
+            DataOutputStream out = new DataOutputStream(bout);
+            writable.write(out);
+            out.close();
+            bout.close();
+            return bout.toByteArray();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinMapper.java
new file mode 100644
index 0000000..a678e70
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinMapper.java
@@ -0,0 +1,32 @@
+/*
+ * 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.engine.mr;
+
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Mapper;
+
+/**
+ */
+public class KylinMapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> extends Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
+    
+    protected void bindCurrentConfiguration(Configuration conf) {
+        HadoopUtil.setCurrentConfiguration(conf);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
new file mode 100644
index 0000000..846c849
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
@@ -0,0 +1,30 @@
+/*
+ * 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.engine.mr;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Reducer;
+
+/**
+ */
+public class KylinReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> extends Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
+    protected void bindCurrentConfiguration(Configuration conf) {
+        HadoopUtil.setCurrentConfiguration(conf);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/engine-mr/src/test/java/org/apache/kylin/engine/mr/LookupTableTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/LookupTableTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/LookupTableTest.java
new file mode 100644
index 0000000..87f9133
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/LookupTableTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.engine.mr;
+
+import java.io.File;
+
+import org.apache.kylin.common.util.Array;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.dict.lookup.LookupBytesTable;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * @author yangli9
+ */
+public class LookupTableTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setup() throws Exception {
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void testBasic() throws Exception {
+        TableDesc siteTable = MetadataManager.getInstance(getTestConfig()).getTableDesc("EDW.TEST_SITES");
+        TableDesc categoryTable = MetadataManager.getInstance(getTestConfig()).getTableDesc("DEFAULT.test_category_groupings");
+        LookupBytesTable lookup;
+
+        System.out.println("============================================================================");
+
+        File f = new File(LOCALMETA_TEST_DATA + "/data/EDW.TEST_SITES.csv");
+        lookup = new LookupBytesTable(siteTable, new String[] { "SITE_ID" }, new DFSFileTable("file://" + f.getAbsolutePath(), 10));
+        lookup.dump();
+
+        System.out.println("============================================================================");
+
+        f = new File(LOCALMETA_TEST_DATA + "/data/DEFAULT.TEST_CATEGORY_GROUPINGS.csv");
+        lookup = new LookupBytesTable(categoryTable, new String[] { "leaf_categ_id", "site_id" }, new DFSFileTable("file://" + f.getAbsolutePath(), 36));
+        lookup.dump();
+
+        System.out.println("============================================================================");
+
+        ByteArray k1 = new ByteArray(Bytes.toBytes("533"));
+        ByteArray k2 = new ByteArray(Bytes.toBytes("0"));
+        Array<ByteArray> key = new Array<ByteArray>(new ByteArray[] { k1, k2 });
+        System.out.println(lookup.getRow(key));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/engine-mr/src/test/java/org/apache/kylin/engine/mr/TableReaderTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/TableReaderTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/TableReaderTest.java
new file mode 100644
index 0000000..2db139a
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/TableReaderTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.engine.mr;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.junit.Test;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class TableReaderTest {
+
+    @Test
+    public void testBasicReader() throws IOException {
+        File f = new File("src/test/resources/dict/DW_SITES");
+        DFSFileTableReader reader = new DFSFileTableReader("file://" + f.getAbsolutePath(), DFSFileTable.DELIM_AUTO, 10);
+        while (reader.next()) {
+            assertEquals("[-1, Korea Auction.co.kr, S, 48, 0, 111, 2009-02-11, , DW_OFFPLAT, ]", Arrays.toString(reader.getRow()));
+            break;
+        }
+        reader.close();
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/invertedindex/pom.xml
----------------------------------------------------------------------
diff --git a/invertedindex/pom.xml b/invertedindex/pom.xml
index 2fc0220..2eec2ce 100644
--- a/invertedindex/pom.xml
+++ b/invertedindex/pom.xml
@@ -30,18 +30,15 @@
 
     <dependencies>
 
+        <!--Kylin Jar -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-common</artifactId>
-            <type>test-jar</type>
-            <scope>test</scope>
+            <artifactId>kylin-core-metadata</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-
-        <!--Kylin Jar -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-metadata</artifactId>
+            <artifactId>kylin-core-dictionary</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
@@ -51,48 +48,18 @@
         </dependency>
 
         <dependency>
-            <groupId>commons-cli</groupId>
-            <artifactId>commons-cli</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-lang</groupId>
-            <artifactId>commons-lang</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-configuration</groupId>
-            <artifactId>commons-configuration</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-databind</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-httpclient</groupId>
-            <artifactId>commons-httpclient</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.ning</groupId>
-            <artifactId>compress-lzf</artifactId>
-        </dependency>
-        <dependency>
             <groupId>com.n3twork.druid</groupId>
             <artifactId>extendedset</artifactId>
         </dependency>
 
         <!-- Env & Test -->
-
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
index faf9079..a8a2244 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
@@ -18,25 +18,29 @@
 
 package org.apache.kylin.invertedindex;
 
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonManagedReference;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.model.*;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.LookupDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.metadata.realization.RealizationType;
 import org.apache.kylin.metadata.realization.SQLDigest;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonManagedReference;
+import com.fasterxml.jackson.annotation.JsonProperty;
 
 /**
  * @author honma

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIManager.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIManager.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIManager.java
index f478caf..547ff15 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIManager.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIManager.java
@@ -20,6 +20,7 @@ package org.apache.kylin.invertedindex;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
@@ -30,6 +31,7 @@ import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.DictionaryInfo;
 import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.dict.DistinctColumnValuesProvider;
 import org.apache.kylin.invertedindex.model.IIDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -130,7 +132,7 @@ public class IIManager implements IRealizationProvider {
         return result;
     }
 
-    public void buildInvertedIndexDictionary(IISegment iiSeg, String factColumnsPath) throws IOException {
+    public void buildInvertedIndexDictionary(IISegment iiSeg, DistinctColumnValuesProvider factTableValueProvider) throws IOException {
         logger.info("Start building ii dictionary");
         DictionaryManager dictMgr = getDictionaryManager();
         IIDesc iiDesc = iiSeg.getIIInstance().getDescriptor();
@@ -140,7 +142,7 @@ public class IIManager implements IRealizationProvider {
                 continue;
             }
 
-            DictionaryInfo dict = dictMgr.buildDictionary(iiDesc.getModel(), "true", column, factColumnsPath);
+            DictionaryInfo dict = dictMgr.buildDictionary(iiDesc.getModel(), "true", column, factTableValueProvider);
             iiSeg.putDictResPath(column, dict.getResourcePath());
         }
         updateII(iiSeg.getIIInstance());


[5/6] incubator-kylin git commit: KYLIN-875 Refactor core-common, remove dependency on hadoop/hbase

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java b/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java
index 3bbc93c..65fb656 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java
@@ -17,12 +17,7 @@
  */
 package org.apache.kylin.common.util;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.io.WritableUtils;
-import sun.misc.Unsafe;
+import static com.google.common.base.Preconditions.*;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -37,11 +32,13 @@ import java.security.AccessController;
 import java.security.PrivilegedAction;
 import java.security.SecureRandom;
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 
-import static com.google.common.base.Preconditions.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import sun.misc.Unsafe;
 
 /**
  * Utility class that handles byte arrays, conversions to/from other types,
@@ -108,7 +105,6 @@ public class Bytes {
      */
     public static final int SIZEOF_SHORT = Short.SIZE / Byte.SIZE;
 
-
     /**
      * Estimate of size cost to pay beyond payload in jvm for instance of byte [].
      * Estimate based on study of jhat and jprofiler numbers.
@@ -117,7 +113,6 @@ public class Bytes {
     // SizeOf which uses java.lang.instrument says 24 bytes. (3 longs?)
     public static final int ESTIMATED_HEAP_TAX = 16;
 
-
     /**
      * Returns length of the byte array, returning 0 if the array is null.
      * Useful for calculating sizes.
@@ -130,152 +125,6 @@ public class Bytes {
     }
 
     /**
-     * Byte array comparator class.
-     */
-    public static class ByteArrayComparator implements RawComparator<byte[]> {
-        /**
-         * Constructor
-         */
-        public ByteArrayComparator() {
-            super();
-        }
-
-        @Override
-        public int compare(byte[] left, byte[] right) {
-            return compareTo(left, right);
-        }
-
-        @Override
-        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-            return LexicographicalComparerHolder.BEST_COMPARER.
-                    compareTo(b1, s1, l1, b2, s2, l2);
-        }
-    }
-
-    /**
-     * A {@link ByteArrayComparator} that treats the empty array as the largest value.
-     * This is useful for comparing row end keys for regions.
-     */
-    // TODO: unfortunately, HBase uses byte[0] as both start and end keys for region
-    // boundaries. Thus semantically, we should treat empty byte array as the smallest value
-    // while comparing row keys, start keys etc; but as the largest value for comparing
-    // region boundaries for endKeys.
-    public static class RowEndKeyComparator extends ByteArrayComparator {
-        @Override
-        public int compare(byte[] left, byte[] right) {
-            return compare(left, 0, left.length, right, 0, right.length);
-        }
-
-        @Override
-        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-            if (b1 == b2 && s1 == s2 && l1 == l2) {
-                return 0;
-            }
-            if (l1 == 0) {
-                return l2; //0 or positive
-            }
-            if (l2 == 0) {
-                return -1;
-            }
-            return super.compare(b1, s1, l1, b2, s2, l2);
-        }
-    }
-
-    /**
-     * Pass this to TreeMaps where byte [] are keys.
-     */
-    public final static Comparator<byte[]> BYTES_COMPARATOR = new ByteArrayComparator();
-
-    /**
-     * Use comparing byte arrays, byte-by-byte
-     */
-    public final static RawComparator<byte[]> BYTES_RAWCOMPARATOR = new ByteArrayComparator();
-
-    /**
-     * Read byte-array written with a WritableableUtils.vint prefix.
-     *
-     * @param in Input to read from.
-     * @return byte array read off <code>in</code>
-     * @throws java.io.IOException e
-     */
-    public static byte[] readByteArray(final DataInput in)
-            throws IOException {
-        int len = WritableUtils.readVInt(in);
-        if (len < 0) {
-            throw new NegativeArraySizeException(Integer.toString(len));
-        }
-        byte[] result = new byte[len];
-        in.readFully(result, 0, len);
-        return result;
-    }
-
-    /**
-     * Read byte-array written with a WritableableUtils.vint prefix.
-     * IOException is converted to a RuntimeException.
-     *
-     * @param in Input to read from.
-     * @return byte array read off <code>in</code>
-     */
-    public static byte[] readByteArrayThrowsRuntime(final DataInput in) {
-        try {
-            return readByteArray(in);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Write byte-array with a WritableableUtils.vint prefix.
-     *
-     * @param out output stream to be written to
-     * @param b   array to write
-     * @throws IOException e
-     */
-    public static void writeByteArray(final DataOutput out, final byte[] b)
-            throws IOException {
-        if (b == null) {
-            WritableUtils.writeVInt(out, 0);
-        } else {
-            writeByteArray(out, b, 0, b.length);
-        }
-    }
-
-    /**
-     * Write byte-array to out with a vint length prefix.
-     *
-     * @param out    output stream
-     * @param b      array
-     * @param offset offset into array
-     * @param length length past offset
-     * @throws IOException e
-     */
-    public static void writeByteArray(final DataOutput out, final byte[] b,
-                                      final int offset, final int length)
-            throws IOException {
-        WritableUtils.writeVInt(out, length);
-        out.write(b, offset, length);
-    }
-
-    /**
-     * Write byte-array from src to tgt with a vint length prefix.
-     *
-     * @param tgt       target array
-     * @param tgtOffset offset into target array
-     * @param src       source array
-     * @param srcOffset source offset
-     * @param srcLength source length
-     * @return New offset in src array.
-     */
-    public static int writeByteArray(final byte[] tgt, final int tgtOffset,
-                                     final byte[] src, final int srcOffset, final int srcLength) {
-        byte[] vint = vintToBytes(srcLength);
-        System.arraycopy(vint, 0, tgt, tgtOffset, vint.length);
-        int offset = tgtOffset + vint.length;
-        System.arraycopy(src, srcOffset, tgt, offset, srcLength);
-        return offset + srcLength;
-    }
-
-    /**
      * Put bytes at the specified byte array position.
      *
      * @param tgtBytes  the byte array
@@ -285,8 +134,7 @@ public class Bytes {
      * @param srcLength source length
      * @return incremented offset
      */
-    public static int putBytes(byte[] tgtBytes, int tgtOffset, byte[] srcBytes,
-                               int srcOffset, int srcLength) {
+    public static int putBytes(byte[] tgtBytes, int tgtOffset, byte[] srcBytes, int srcOffset, int srcLength) {
         System.arraycopy(srcBytes, srcOffset, tgtBytes, tgtOffset, srcLength);
         return tgtOffset + srcLength;
     }
@@ -358,9 +206,7 @@ public class Bytes {
      * @param sep The separator to use.
      * @param b2  The second byte array.
      */
-    public static String toString(final byte[] b1,
-                                  String sep,
-                                  final byte[] b2) {
+    public static String toString(final byte[] b1, String sep, final byte[] b2) {
         return toString(b1, 0, b1.length) + sep + toString(b2, 0, b2.length);
     }
 
@@ -429,14 +275,13 @@ public class Bytes {
     public static String toStringBinary(final byte[] b, int off, int len) {
         StringBuilder result = new StringBuilder();
         // Just in case we are passed a 'len' that is > buffer length...
-        if (off >= b.length) return result.toString();
-        if (off + len > b.length) len = b.length - off;
+        if (off >= b.length)
+            return result.toString();
+        if (off + len > b.length)
+            len = b.length - off;
         for (int i = off; i < off + len; ++i) {
             int ch = b[i] & 0xFF;
-            if ((ch >= '0' && ch <= '9')
-                    || (ch >= 'A' && ch <= 'Z')
-                    || (ch >= 'a' && ch <= 'z')
-                    || " `~!@#$%^&*()-_=+[]{}|;:'\",.<>/?".indexOf(ch) >= 0) {
+            if ((ch >= '0' && ch <= '9') || (ch >= 'A' && ch <= 'Z') || (ch >= 'a' && ch <= 'z') || " `~!@#$%^&*()-_=+[]{}|;:'\",.<>/?".indexOf(ch) >= 0) {
                 result.append((char) ch);
             } else {
                 result.append(String.format("\\x%02X", ch));
@@ -446,9 +291,7 @@ public class Bytes {
     }
 
     private static boolean isHexDigit(char c) {
-        return
-                (c >= 'A' && c <= 'F') ||
-                        (c >= '0' && c <= '9');
+        return (c >= 'A' && c <= 'F') || (c >= '0' && c <= '9');
     }
 
     /**
@@ -477,8 +320,7 @@ public class Bytes {
                 char hd2 = in.charAt(i + 3);
 
                 // they need to be A-F0-9:
-                if (!isHexDigit(hd1) ||
-                        !isHexDigit(hd2)) {
+                if (!isHexDigit(hd1) || !isHexDigit(hd2)) {
                     // bogus escape code, ignore:
                     continue;
                 }
@@ -515,7 +357,7 @@ public class Bytes {
      * @return <code>b</code> encoded in a byte array.
      */
     public static byte[] toBytes(final boolean b) {
-        return new byte[]{b ? (byte) -1 : (byte) 0};
+        return new byte[] { b ? (byte) -1 : (byte) 0 };
     }
 
     /**
@@ -596,17 +438,12 @@ public class Bytes {
         }
     }
 
-    private static IllegalArgumentException
-    explainWrongLengthOrOffset(final byte[] bytes,
-                               final int offset,
-                               final int length,
-                               final int expectedLength) {
+    private static IllegalArgumentException explainWrongLengthOrOffset(final byte[] bytes, final int offset, final int length, final int expectedLength) {
         String reason;
         if (length != expectedLength) {
             reason = "Wrong length: " + length + ", expected " + expectedLength;
         } else {
-            reason = "offset (" + offset + ") + length (" + length + ") exceed the"
-                    + " capacity of the array: " + bytes.length;
+            reason = "offset (" + offset + ") + length (" + length + ") exceed the" + " capacity of the array: " + bytes.length;
         }
         return new IllegalArgumentException(reason);
     }
@@ -623,8 +460,7 @@ public class Bytes {
      */
     public static int putLong(byte[] bytes, int offset, long val) {
         if (bytes.length - offset < SIZEOF_LONG) {
-            throw new IllegalArgumentException("Not enough room to put a long at"
-                    + " offset " + offset + " in a " + bytes.length + " byte array");
+            throw new IllegalArgumentException("Not enough room to put a long at" + " offset " + offset + " in a " + bytes.length + " byte array");
         }
         if (org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.isAvailable()) {
             return putLongUnsafe(bytes, offset, val);
@@ -650,8 +486,7 @@ public class Bytes {
         if (org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.littleEndian) {
             val = Long.reverseBytes(val);
         }
-        org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.putLong(bytes, (long) offset +
-                org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET, val);
+        org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.putLong(bytes, (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET, val);
         return offset + SIZEOF_LONG;
     }
 
@@ -807,11 +642,9 @@ public class Bytes {
      */
     public static int toIntUnsafe(byte[] bytes, int offset) {
         if (org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.littleEndian) {
-            return Integer.reverseBytes(org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getInt(bytes,
-                    (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET));
+            return Integer.reverseBytes(org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getInt(bytes, (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET));
         } else {
-            return org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getInt(bytes,
-                    (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET);
+            return org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getInt(bytes, (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET);
         }
     }
 
@@ -824,11 +657,9 @@ public class Bytes {
      */
     public static short toShortUnsafe(byte[] bytes, int offset) {
         if (org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.littleEndian) {
-            return Short.reverseBytes(org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getShort(bytes,
-                    (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET));
+            return Short.reverseBytes(org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getShort(bytes, (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET));
         } else {
-            return org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getShort(bytes,
-                    (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET);
+            return org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getShort(bytes, (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET);
         }
     }
 
@@ -841,11 +672,9 @@ public class Bytes {
      */
     public static long toLongUnsafe(byte[] bytes, int offset) {
         if (org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.littleEndian) {
-            return Long.reverseBytes(org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getLong(bytes,
-                    (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET));
+            return Long.reverseBytes(org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getLong(bytes, (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET));
         } else {
-            return org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getLong(bytes,
-                    (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET);
+            return org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.getLong(bytes, (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET);
         }
     }
 
@@ -861,8 +690,7 @@ public class Bytes {
      */
     public static int readAsInt(byte[] bytes, int offset, final int length) {
         if (offset + length > bytes.length) {
-            throw new IllegalArgumentException("offset (" + offset + ") + length (" + length
-                    + ") exceed the" + " capacity of the array: " + bytes.length);
+            throw new IllegalArgumentException("offset (" + offset + ") + length (" + length + ") exceed the" + " capacity of the array: " + bytes.length);
         }
         int n = 0;
         for (int i = offset; i < (offset + length); i++) {
@@ -884,8 +712,7 @@ public class Bytes {
      */
     public static int putInt(byte[] bytes, int offset, int val) {
         if (bytes.length - offset < SIZEOF_INT) {
-            throw new IllegalArgumentException("Not enough room to put an int at"
-                    + " offset " + offset + " in a " + bytes.length + " byte array");
+            throw new IllegalArgumentException("Not enough room to put an int at" + " offset " + offset + " in a " + bytes.length + " byte array");
         }
         if (org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.isAvailable()) {
             return putIntUnsafe(bytes, offset, val);
@@ -911,8 +738,7 @@ public class Bytes {
         if (org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.littleEndian) {
             val = Integer.reverseBytes(val);
         }
-        org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.putInt(bytes, (long) offset +
-                org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET, val);
+        org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.putInt(bytes, (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET, val);
         return offset + SIZEOF_INT;
     }
 
@@ -1001,8 +827,7 @@ public class Bytes {
      */
     public static int putShort(byte[] bytes, int offset, short val) {
         if (bytes.length - offset < SIZEOF_SHORT) {
-            throw new IllegalArgumentException("Not enough room to put a short at"
-                    + " offset " + offset + " in a " + bytes.length + " byte array");
+            throw new IllegalArgumentException("Not enough room to put a short at" + " offset " + offset + " in a " + bytes.length + " byte array");
         }
         if (org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.isAvailable()) {
             return putShortUnsafe(bytes, offset, val);
@@ -1026,8 +851,7 @@ public class Bytes {
         if (org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.littleEndian) {
             val = Short.reverseBytes(val);
         }
-        org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.putShort(bytes, (long) offset +
-                org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET, val);
+        org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.theUnsafe.putShort(bytes, (long) offset + org.apache.kylin.common.util.Bytes.LexicographicalComparerHolder.UnsafeComparer.BYTE_ARRAY_BASE_OFFSET, val);
         return offset + SIZEOF_SHORT;
     }
 
@@ -1046,8 +870,7 @@ public class Bytes {
      */
     public static int putAsShort(byte[] bytes, int offset, int val) {
         if (bytes.length - offset < SIZEOF_SHORT) {
-            throw new IllegalArgumentException("Not enough room to put a short at"
-                    + " offset " + offset + " in a " + bytes.length + " byte array");
+            throw new IllegalArgumentException("Not enough room to put a short at" + " offset " + offset + " in a " + bytes.length + " byte array");
         }
         bytes[offset + 1] = (byte) val;
         val >>= 8;
@@ -1069,7 +892,6 @@ public class Bytes {
         return result;
     }
 
-
     /**
      * Converts a byte array to a BigDecimal
      *
@@ -1089,8 +911,7 @@ public class Bytes {
      * @return the char value
      */
     public static BigDecimal toBigDecimal(byte[] bytes, int offset, final int length) {
-        if (bytes == null || length < SIZEOF_INT + 1 ||
-                (offset + length > bytes.length)) {
+        if (bytes == null || length < SIZEOF_INT + 1 || (offset + length > bytes.length)) {
             return null;
         }
 
@@ -1120,95 +941,12 @@ public class Bytes {
     }
 
     /**
-     * @param vint Integer to make a vint of.
-     * @return Vint as bytes array.
-     */
-    public static byte[] vintToBytes(final long vint) {
-        long i = vint;
-        int size = WritableUtils.getVIntSize(i);
-        byte[] result = new byte[size];
-        int offset = 0;
-        if (i >= -112 && i <= 127) {
-            result[offset] = (byte) i;
-            return result;
-        }
-
-        int len = -112;
-        if (i < 0) {
-            i ^= -1L; // take one's complement'
-            len = -120;
-        }
-
-        long tmp = i;
-        while (tmp != 0) {
-            tmp = tmp >> 8;
-            len--;
-        }
-
-        result[offset++] = (byte) len;
-
-        len = (len < -120) ? -(len + 120) : -(len + 112);
-
-        for (int idx = len; idx != 0; idx--) {
-            int shiftbits = (idx - 1) * 8;
-            long mask = 0xFFL << shiftbits;
-            result[offset++] = (byte) ((i & mask) >> shiftbits);
-        }
-        return result;
-    }
-
-    /**
-     * @param buffer buffer to convert
-     * @return vint bytes as an integer.
-     */
-    public static long bytesToVint(final byte[] buffer) {
-        int offset = 0;
-        byte firstByte = buffer[offset++];
-        int len = WritableUtils.decodeVIntSize(firstByte);
-        if (len == 1) {
-            return firstByte;
-        }
-        long i = 0;
-        for (int idx = 0; idx < len - 1; idx++) {
-            byte b = buffer[offset++];
-            i = i << 8;
-            i = i | (b & 0xFF);
-        }
-        return (WritableUtils.isNegativeVInt(firstByte) ? ~i : i);
-    }
-
-    /**
-     * Reads a zero-compressed encoded long from input stream and returns it.
-     *
-     * @param buffer Binary array
-     * @param offset Offset into array at which vint begins.
-     * @return deserialized long from stream.
-     * @throws java.io.IOException e
-     */
-    public static long readVLong(final byte[] buffer, final int offset)
-            throws IOException {
-        byte firstByte = buffer[offset];
-        int len = WritableUtils.decodeVIntSize(firstByte);
-        if (len == 1) {
-            return firstByte;
-        }
-        long i = 0;
-        for (int idx = 0; idx < len - 1; idx++) {
-            byte b = buffer[offset + 1 + idx];
-            i = i << 8;
-            i = i | (b & 0xFF);
-        }
-        return (WritableUtils.isNegativeVInt(firstByte) ? ~i : i);
-    }
-
-    /**
      * @param left  left operand
      * @param right right operand
      * @return 0 if equal, < 0 if left is less than right, etc.
      */
     public static int compareTo(final byte[] left, final byte[] right) {
-        return LexicographicalComparerHolder.BEST_COMPARER.
-                compareTo(left, 0, left.length, right, 0, right.length);
+        return LexicographicalComparerHolder.BEST_COMPARER.compareTo(left, 0, left.length, right, 0, right.length);
     }
 
     /**
@@ -1222,16 +960,12 @@ public class Bytes {
      * @param length2 How much to compare from the right buffer
      * @return 0 if equal, < 0 if left is less than right, etc.
      */
-    public static int compareTo(byte[] buffer1, int offset1, int length1,
-                                byte[] buffer2, int offset2, int length2) {
-        return LexicographicalComparerHolder.BEST_COMPARER.
-                compareTo(buffer1, offset1, length1, buffer2, offset2, length2);
+    public static int compareTo(byte[] buffer1, int offset1, int length1, byte[] buffer2, int offset2, int length2) {
+        return LexicographicalComparerHolder.BEST_COMPARER.compareTo(buffer1, offset1, length1, buffer2, offset2, length2);
     }
 
     interface Comparer<T> {
-        int compareTo(
-                T buffer1, int offset1, int length1, T buffer2, int offset2, int length2
-        );
+        int compareTo(T buffer1, int offset1, int length1, T buffer2, int offset2, int length2);
     }
 
     static Comparer<byte[]> lexicographicalComparerJavaImpl() {
@@ -1246,8 +980,7 @@ public class Bytes {
      * {@code Unsafe} isn't available.
      */
     static class LexicographicalComparerHolder {
-        static final String UNSAFE_COMPARER_NAME =
-                LexicographicalComparerHolder.class.getName() + "$UnsafeComparer";
+        static final String UNSAFE_COMPARER_NAME = LexicographicalComparerHolder.class.getName() + "$UnsafeComparer";
 
         static final Comparer<byte[]> BEST_COMPARER = getBestComparer();
 
@@ -1261,8 +994,7 @@ public class Bytes {
 
                 // yes, UnsafeComparer does implement Comparer<byte[]>
                 @SuppressWarnings("unchecked")
-                Comparer<byte[]> comparer =
-                        (Comparer<byte[]>) theClass.getEnumConstants()[0];
+                Comparer<byte[]> comparer = (Comparer<byte[]>) theClass.getEnumConstants()[0];
                 return comparer;
             } catch (Throwable t) { // ensure we really catch *everything*
                 return lexicographicalComparerJavaImpl();
@@ -1273,12 +1005,9 @@ public class Bytes {
             INSTANCE;
 
             @Override
-            public int compareTo(byte[] buffer1, int offset1, int length1,
-                                 byte[] buffer2, int offset2, int length2) {
+            public int compareTo(byte[] buffer1, int offset1, int length1, byte[] buffer2, int offset2, int length2) {
                 // Short circuit equal case
-                if (buffer1 == buffer2 &&
-                        offset1 == offset2 &&
-                        length1 == length2) {
+                if (buffer1 == buffer2 && offset1 == offset2 && length1 == length2) {
                     return 0;
                 }
                 // Bring WritableComparator code local
@@ -1306,23 +1035,22 @@ public class Bytes {
             static final int BYTE_ARRAY_BASE_OFFSET;
 
             static {
-                theUnsafe = (Unsafe) AccessController.doPrivileged(
-                        new PrivilegedAction<Object>() {
-                            @Override
-                            public Object run() {
-                                try {
-                                    Field f = Unsafe.class.getDeclaredField("theUnsafe");
-                                    f.setAccessible(true);
-                                    return f.get(null);
-                                } catch (NoSuchFieldException e) {
-                                    // It doesn't matter what we throw;
-                                    // it's swallowed in getBestComparer().
-                                    throw new Error();
-                                } catch (IllegalAccessException e) {
-                                    throw new Error();
-                                }
-                            }
-                        });
+                theUnsafe = (Unsafe) AccessController.doPrivileged(new PrivilegedAction<Object>() {
+                    @Override
+                    public Object run() {
+                        try {
+                            Field f = Unsafe.class.getDeclaredField("theUnsafe");
+                            f.setAccessible(true);
+                            return f.get(null);
+                        } catch (NoSuchFieldException e) {
+                            // It doesn't matter what we throw;
+                            // it's swallowed in getBestComparer().
+                            throw new Error();
+                        } catch (IllegalAccessException e) {
+                            throw new Error();
+                        }
+                    }
+                });
 
                 BYTE_ARRAY_BASE_OFFSET = theUnsafe.arrayBaseOffset(byte[].class);
 
@@ -1332,8 +1060,7 @@ public class Bytes {
                 }
             }
 
-            static final boolean littleEndian =
-                    ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN);
+            static final boolean littleEndian = ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN);
 
             /**
              * Returns true if x1 is less than x2, when both values are treated as
@@ -1380,13 +1107,10 @@ public class Bytes {
              * @return 0 if equal, < 0 if left is less than right, etc.
              */
             @Override
-            public int compareTo(byte[] buffer1, int offset1, int length1,
-                                 byte[] buffer2, int offset2, int length2) {
+            public int compareTo(byte[] buffer1, int offset1, int length1, byte[] buffer2, int offset2, int length2) {
 
                 // Short circuit equal case
-                if (buffer1 == buffer2 &&
-                        offset1 == offset2 &&
-                        length1 == length2) {
+                if (buffer1 == buffer2 && offset1 == offset2 && length1 == length2) {
                     return 0;
                 }
                 final int minLength = Math.min(length1, length2);
@@ -1394,11 +1118,11 @@ public class Bytes {
                 final long offset1Adj = offset1 + BYTE_ARRAY_BASE_OFFSET;
                 final long offset2Adj = offset2 + BYTE_ARRAY_BASE_OFFSET;
 
-        /*
-         * Compare 8 bytes at a time. Benchmarking shows comparing 8 bytes at a
-         * time is no slower than comparing 4 bytes at a time even on 32-bit.
-         * On the other hand, it is substantially faster on 64-bit.
-         */
+                /*
+                 * Compare 8 bytes at a time. Benchmarking shows comparing 8 bytes at a
+                 * time is no slower than comparing 4 bytes at a time even on 32-bit.
+                 * On the other hand, it is substantially faster on 64-bit.
+                 */
                 for (int i = 0; i < minWords * SIZEOF_LONG; i += SIZEOF_LONG) {
                     long lw = theUnsafe.getLong(buffer1, offset1Adj + (long) i);
                     long rw = theUnsafe.getLong(buffer2, offset2Adj + (long) i);
@@ -1457,25 +1181,27 @@ public class Bytes {
     public static boolean equals(final byte[] left, final byte[] right) {
         // Could use Arrays.equals?
         //noinspection SimplifiableConditionalExpression
-        if (left == right) return true;
-        if (left == null || right == null) return false;
-        if (left.length != right.length) return false;
-        if (left.length == 0) return true;
+        if (left == right)
+            return true;
+        if (left == null || right == null)
+            return false;
+        if (left.length != right.length)
+            return false;
+        if (left.length == 0)
+            return true;
 
         // Since we're often comparing adjacent sorted data,
         // it's usual to have equal arrays except for the very last byte
         // so check that first
-        if (left[left.length - 1] != right[right.length - 1]) return false;
+        if (left[left.length - 1] != right[right.length - 1])
+            return false;
 
         return compareTo(left, right) == 0;
     }
 
-    public static boolean equals(final byte[] left, int leftOffset, int leftLen,
-                                 final byte[] right, int rightOffset, int rightLen) {
+    public static boolean equals(final byte[] left, int leftOffset, int leftLen, final byte[] right, int rightOffset, int rightLen) {
         // short circuit case
-        if (left == right &&
-                leftOffset == rightOffset &&
-                leftLen == rightLen) {
+        if (left == right && leftOffset == rightOffset && leftLen == rightLen) {
             return true;
         }
         // different lengths fast check
@@ -1489,22 +1215,24 @@ public class Bytes {
         // Since we're often comparing adjacent sorted data,
         // it's usual to have equal arrays except for the very last byte
         // so check that first
-        if (left[leftOffset + leftLen - 1] != right[rightOffset + rightLen - 1]) return false;
+        if (left[leftOffset + leftLen - 1] != right[rightOffset + rightLen - 1])
+            return false;
 
-        return LexicographicalComparerHolder.BEST_COMPARER.
-                compareTo(left, leftOffset, leftLen, right, rightOffset, rightLen) == 0;
+        return LexicographicalComparerHolder.BEST_COMPARER.compareTo(left, leftOffset, leftLen, right, rightOffset, rightLen) == 0;
     }
 
-
     /**
      * @param a   left operand
      * @param buf right operand
      * @return True if equal
      */
     public static boolean equals(byte[] a, ByteBuffer buf) {
-        if (a == null) return buf == null;
-        if (buf == null) return false;
-        if (a.length != buf.remaining()) return false;
+        if (a == null)
+            return buf == null;
+        if (buf == null)
+            return false;
+        if (a.length != buf.remaining())
+            return false;
 
         // Thou shalt not modify the original byte buffer in what should be read only operations.
         ByteBuffer b = buf.duplicate();
@@ -1516,16 +1244,12 @@ public class Bytes {
         return true;
     }
 
-
     /**
      * Return true if the byte array on the right is a prefix of the byte
      * array on the left.
      */
     public static boolean startsWith(byte[] bytes, byte[] prefix) {
-        return bytes != null && prefix != null &&
-                bytes.length >= prefix.length &&
-                LexicographicalComparerHolder.BEST_COMPARER.
-                        compareTo(bytes, 0, prefix.length, prefix, 0, prefix.length) == 0;
+        return bytes != null && prefix != null && bytes.length >= prefix.length && LexicographicalComparerHolder.BEST_COMPARER.compareTo(bytes, 0, prefix.length, prefix, 0, prefix.length) == 0;
     }
 
     /**
@@ -1546,7 +1270,15 @@ public class Bytes {
      * {@link org.apache.hadoop.hbase.io.ImmutableBytesWritable} use calculating hash code.
      */
     public static int hashCode(final byte[] b, final int length) {
-        return WritableComparator.hashBytes(b, length);
+        return hashBytes(b, 0, length);
+    }
+
+    /** Compute hash for binary data. */
+    public static int hashBytes(byte[] bytes, int offset, int length) {
+        int hash = 1;
+        for (int i = offset; i < offset + length; i++)
+            hash = (31 * hash) + (int) bytes[i];
+        return hash;
     }
 
     /**
@@ -1672,8 +1404,7 @@ public class Bytes {
      *                  the range in two; i.e. one split.
      * @return Array of dividing values
      */
-    public static byte[][] split(final byte[] a, final byte[] b,
-                                 boolean inclusive, final int num) {
+    public static byte[][] split(final byte[] a, final byte[] b, boolean inclusive, final int num) {
         byte[][] ret = new byte[num + 2][];
         int i = 0;
         Iterable<byte[]> iter = iterateOnSplits(a, b, inclusive, num);
@@ -1688,16 +1419,14 @@ public class Bytes {
     /**
      * Iterate over keys within the passed range, splitting at an [a,b) boundary.
      */
-    public static Iterable<byte[]> iterateOnSplits(final byte[] a,
-                                                   final byte[] b, final int num) {
+    public static Iterable<byte[]> iterateOnSplits(final byte[] a, final byte[] b, final int num) {
         return iterateOnSplits(a, b, false, num);
     }
 
     /**
      * Iterate over keys within the passed range.
      */
-    public static Iterable<byte[]> iterateOnSplits(
-            final byte[] a, final byte[] b, boolean inclusive, final int num) {
+    public static Iterable<byte[]> iterateOnSplits(final byte[] a, final byte[] b, boolean inclusive, final int num) {
         byte[] aPadded;
         byte[] bPadded;
         if (a.length < b.length) {
@@ -1716,7 +1445,7 @@ public class Bytes {
         if (num <= 0) {
             throw new IllegalArgumentException("num cannot be <= 0");
         }
-        byte[] prependHeader = {1, 0};
+        byte[] prependHeader = { 1, 0 };
         final BigInteger startBI = new BigInteger(add(prependHeader, aPadded));
         final BigInteger stopBI = new BigInteger(add(prependHeader, bPadded));
         BigInteger diffBI = stopBI.subtract(startBI);
@@ -1746,8 +1475,10 @@ public class Bytes {
             @Override
             public byte[] next() {
                 i++;
-                if (i == 0) return a;
-                if (i == num + 1) return b;
+                if (i == 0)
+                    return a;
+                if (i == num + 1)
+                    return b;
 
                 BigInteger curBI = startBI.add(intervalBI.multiply(BigInteger.valueOf(i)));
                 byte[] padded = curBI.toByteArray();
@@ -1818,45 +1549,6 @@ public class Bytes {
     }
 
     /**
-     * Binary search for keys in indexes.
-     *
-     * @param arr        array of byte arrays to search for
-     * @param key        the key you want to find
-     * @param offset     the offset in the key you want to find
-     * @param length     the length of the key
-     * @param comparator a comparator to compare.
-     * @return zero-based index of the key, if the key is present in the array.
-     * Otherwise, a value -(i + 1) such that the key is between arr[i -
-     * 1] and arr[i] non-inclusively, where i is in [0, i], if we define
-     * arr[-1] = -Inf and arr[N] = Inf for an N-element array. The above
-     * means that this function can return 2N + 1 different values
-     * ranging from -(N + 1) to N - 1.
-     */
-    public static int binarySearch(byte[][] arr, byte[] key, int offset,
-                                   int length, RawComparator<?> comparator) {
-        int low = 0;
-        int high = arr.length - 1;
-
-        while (low <= high) {
-            int mid = (low + high) >>> 1;
-            // we have to compare in this order, because the comparator order
-            // has special logic when the 'left side' is a special key.
-            int cmp = comparator.compare(key, offset, length,
-                    arr[mid], 0, arr[mid].length);
-            // key lives above the midpoint
-            if (cmp > 0)
-                low = mid + 1;
-                // key lives below the midpoint
-            else if (cmp < 0)
-                high = mid - 1;
-                // BAM. how often does this really happen?
-            else
-                return mid;
-        }
-        return -(low + 1);
-    }
-
-    /**
      * Bytewise binary increment/deincrement of long contained in byte array
      * on given amount.
      *
@@ -1870,18 +1562,17 @@ public class Bytes {
             // Hopefully this doesn't happen too often.
             byte[] newvalue;
             if (val[0] < 0) {
-                newvalue = new byte[]{-1, -1, -1, -1, -1, -1, -1, -1};
+                newvalue = new byte[] { -1, -1, -1, -1, -1, -1, -1, -1 };
             } else {
                 newvalue = new byte[SIZEOF_LONG];
             }
-            System.arraycopy(val, 0, newvalue, newvalue.length - val.length,
-                    val.length);
+            System.arraycopy(val, 0, newvalue, newvalue.length - val.length, val.length);
             val = newvalue;
         } else if (val.length > SIZEOF_LONG) {
-            throw new IllegalArgumentException("Increment Bytes - value too big: " +
-                    val.length);
+            throw new IllegalArgumentException("Increment Bytes - value too big: " + val.length);
         }
-        if (amount == 0) return val;
+        if (amount == 0)
+            return val;
         if (val[0] < 0) {
             return binaryIncrementNeg(val, amount);
         }
@@ -1908,7 +1599,8 @@ public class Bytes {
                 amo -= sign;
             }
             value[value.length - i - 1] = (byte) total;
-            if (amo == 0) return value;
+            if (amo == 0)
+                return value;
         }
         return value;
     }
@@ -1933,7 +1625,8 @@ public class Bytes {
                 total %= 256;
             }
             value[value.length - i - 1] = (byte) total;
-            if (amo == 0) return value;
+            if (amo == 0)
+                return value;
         }
         return value;
     }
@@ -1941,12 +1634,10 @@ public class Bytes {
     /**
      * Writes a string as a fixed-size field, padded with zeros.
      */
-    public static void writeStringFixedSize(final DataOutput out, String s,
-                                            int size) throws IOException {
+    public static void writeStringFixedSize(final DataOutput out, String s, int size) throws IOException {
         byte[] b = toBytes(s);
         if (b.length > size) {
-            throw new IOException("Trying to write " + b.length + " bytes (" +
-                    toStringBinary(b) + ") into a field of length " + size);
+            throw new IOException("Trying to write " + b.length + " bytes (" + toStringBinary(b) + ") into a field of length " + size);
         }
 
         out.writeBytes(s);
@@ -1957,8 +1648,7 @@ public class Bytes {
     /**
      * Reads a fixed-size field and interprets it as a string padded with zeros.
      */
-    public static String readStringFixedSize(final DataInput in, int size)
-            throws IOException {
+    public static String readStringFixedSize(final DataInput in, int size) throws IOException {
         byte[] b = new byte[size];
         in.readFully(b);
         int n = b.length;
@@ -1976,7 +1666,8 @@ public class Bytes {
      * @return a copy of the given byte array
      */
     public static byte[] copy(byte[] bytes) {
-        if (bytes == null) return null;
+        if (bytes == null)
+            return null;
         byte[] result = new byte[bytes.length];
         System.arraycopy(bytes, 0, result, 0, bytes.length);
         return result;
@@ -1992,7 +1683,8 @@ public class Bytes {
      * @return a copy of the given designated byte array
      */
     public static byte[] copy(byte[] bytes, final int offset, final int length) {
-        if (bytes == null) return null;
+        if (bytes == null)
+            return null;
         byte[] result = new byte[length];
         System.arraycopy(bytes, offset, result, 0, length);
         return result;
@@ -2077,7 +1769,6 @@ public class Bytes {
         return true;
     }
 
-
     /**
      * Returns the index of the first appearance of the value {@code target} in
      * {@code array}.
@@ -2114,8 +1805,7 @@ public class Bytes {
             return 0;
         }
 
-        outer:
-        for (int i = 0; i < array.length - target.length + 1; i++) {
+        outer: for (int i = 0; i < array.length - target.length + 1; i++) {
             for (int j = 0; j < target.length; j++) {
                 if (array[i + j] != target[j]) {
                     continue outer;
@@ -2221,8 +1911,7 @@ public class Bytes {
         }
         byte[] result = new byte[srcBytes.length * multiNum];
         for (int i = 0; i < multiNum; i++) {
-            System.arraycopy(srcBytes, 0, result, i * srcBytes.length,
-                    srcBytes.length);
+            System.arraycopy(srcBytes, 0, result, i * srcBytes.length, srcBytes.length);
         }
         return result;
     }
@@ -2250,11 +1939,9 @@ public class Bytes {
         hex = hex.toUpperCase();
         byte[] b = new byte[hex.length() / 2];
         for (int i = 0; i < b.length; i++) {
-            b[i] = (byte) ((toBinaryFromHex((byte) hex.charAt(2 * i)) << 4) +
-                    toBinaryFromHex((byte) hex.charAt((2 * i + 1))));
+            b[i] = (byte) ((toBinaryFromHex((byte) hex.charAt(2 * i)) << 4) + toBinaryFromHex((byte) hex.charAt((2 * i + 1))));
         }
         return b;
     }
 
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/util/BytesSplitter.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/BytesSplitter.java b/core-common/src/main/java/org/apache/kylin/common/util/BytesSplitter.java
index 7249dcf..357e3f6 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/BytesSplitter.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/BytesSplitter.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.kylin.common.util.Bytes;
-import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -120,13 +118,13 @@ public class BytesSplitter {
             throw new IOException("No delimiter found");
     }
 
-    public int detectDelim(Text value, int expectedParts) {
+    public int detectDelim(byte[] array, int arrayLen, int expectedParts) {
         for (int i = 0; i < COMMON_DELIMS.length; i++) {
-            int nParts = split(value.getBytes(), value.getLength(), (byte) COMMON_DELIMS[i]);
+            int nParts = split(array, arrayLen, (byte) COMMON_DELIMS[i]);
             if (nParts == expectedParts)
                 return COMMON_DELIMS[i];
         }
-        throw new RuntimeException("Cannot detect delimeter from first line -- " + value.toString() + " -- expect " + expectedParts + " columns");
+        throw new RuntimeException("Cannot detect delimeter from first line -- " + Bytes.toString(array, 0, arrayLen) + " -- expect " + expectedParts + " columns");
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
index c9f1e08..59f8164 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
@@ -18,12 +18,6 @@
 
 package org.apache.kylin.common.util;
 
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.io.Writable;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
 
@@ -374,19 +368,6 @@ public class BytesUtil {
         return array;
     }
 
-    public static byte[] toBytes(Writable writable) {
-        try {
-            ByteArrayOutputStream bout = new ByteArrayOutputStream();
-            DataOutputStream out = new DataOutputStream(bout);
-            writable.write(out);
-            out.close();
-            bout.close();
-            return bout.toByteArray();
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
     public static String toReadableText(byte[] array) {
         if (array == null)
             return null;
@@ -410,13 +391,10 @@ public class BytesUtil {
     }
 
     public static String toHex(byte[] array) {
-        return toHex(new ImmutableBytesWritable(array));
+        return toHex(array, 0, array.length);
     }
 
-    public static String toHex(ImmutableBytesWritable bytes) {
-        byte[] array = bytes.get();
-        int offset = bytes.getOffset();
-        int length = bytes.getLength();
+    public static String toHex(byte[] array, int offset, int length) {
         StringBuilder sb = new StringBuilder(length * 4);
         for (int i = 0; i < length; i++) {
             int b = array[offset + i];

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java b/core-common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
deleted file mode 100644
index 12b0c40..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
+++ /dev/null
@@ -1,128 +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.
-*/
-
-/** This class will come with HBase 2.0 in package org.apache.hadoop.hbase.util **/
-package org.apache.kylin.common.util;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HRegionInfo;
-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.kylin.common.util.Bytes;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class HBaseRegionSizeCalculator {
-
-    private static final Logger logger = LoggerFactory.getLogger(HBaseRegionSizeCalculator.class);
-
-    /**
-     * Maps each region to its size in bytes.
-     **/
-    private final Map<byte[], Long> sizeMap = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
-
-    static final String ENABLE_REGIONSIZECALCULATOR = "hbase.regionsizecalculator.enable";
-
-    /**
-     * 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 {
-
-        try {
-            if (!enabled(table.getConfiguration())) {
-                logger.info("Region size calculation disabled.");
-                return;
-            }
-
-            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
-
-            // Get regions for table.
-            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
-            Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
-
-            for (HRegionInfo regionInfo : tableRegionInfos) {
-                tableRegions.add(regionInfo.getRegionName());
-            }
-
-            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
-            Collection<ServerName> servers = clusterStatus.getServers();
-            final long megaByte = 1024L * 1024L;
-
-            // Iterate all cluster regions, filter regions from our table and
-            // compute their size.
-            for (ServerName serverName : servers) {
-                ServerLoad serverLoad = clusterStatus.getLoad(serverName);
-
-                for (RegionLoad regionLoad : serverLoad.getRegionsLoad().values()) {
-                    byte[] regionId = regionLoad.getName();
-
-                    if (tableRegions.contains(regionId)) {
-
-                        long regionSizeBytes = regionLoad.getStorefileSizeMB() * megaByte;
-                        sizeMap.put(regionId, regionSizeBytes);
-
-                        // logger.info("Region " + regionLoad.getNameAsString()
-                        // + " has size " + regionSizeBytes);
-                    }
-                }
-            }
-        } finally {
-            hBaseAdmin.close();
-        }
-
-    }
-
-    boolean enabled(Configuration configuration) {
-        return configuration.getBoolean(ENABLE_REGIONSIZECALCULATOR, true);
-    }
-
-    /**
-     * Returns size of given region in bytes. Returns 0 if region was not found.
-     **/
-    public long getRegionSize(byte[] regionId) {
-        Long size = sizeMap.get(regionId);
-        if (size == null) {
-            logger.info("Unknown region:" + Arrays.toString(regionId));
-            return 0;
-        } else {
-            return size;
-        }
-    }
-
-    public Map<byte[], Long> getRegionSizeMap() {
-        return Collections.unmodifiableMap(sizeMap);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
deleted file mode 100644
index fb6b9bb..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
+++ /dev/null
@@ -1,162 +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.common.util;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.UnknownHostException;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-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.HConstants;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class HadoopUtil {
-    private static final Logger logger = LoggerFactory.getLogger(HadoopUtil.class);
-
-    private static ThreadLocal<Configuration> hadoopConfig = new ThreadLocal<>();
-
-    public static void setCurrentConfiguration(Configuration conf) {
-        hadoopConfig.set(conf);
-    }
-
-    public static Configuration getCurrentConfiguration() {
-        if (hadoopConfig.get() == null) {
-            hadoopConfig.set(new Configuration());
-        }
-        return hadoopConfig.get();
-    }
-
-    public static FileSystem getFileSystem(String path) throws IOException {
-        return FileSystem.get(makeURI(path), getCurrentConfiguration());
-    }
-
-    public static URI makeURI(String filePath) {
-        try {
-            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 Configuration newHadoopJobConfiguration() {
-        Configuration conf = new Configuration();
-        conf.set(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, "8");
-        return conf;
-    }
-
-    /**
-     * e.g.
-     * 0. hbase (recommended way)
-     * 1. hbase:zk-1.hortonworks.com,zk-2.hortonworks.com,zk-3.hortonworks.com:2181:/hbase-unsecure
-     * 2. hbase:zk-1.hortonworks.com,zk-2.hortonworks.com,zk-3.hortonworks.com:2181
-     * 3. hbase:zk-1.hortonworks.com:2181:/hbase-unsecure
-     * 4. hbase:zk-1.hortonworks.com:2181
-     */
-    public static Configuration newHBaseConfiguration(String url) {
-        Configuration conf = HBaseConfiguration.create();
-        // reduce rpc retry
-        conf.set(HConstants.HBASE_CLIENT_PAUSE, "3000");
-        conf.set(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "5");
-        conf.set(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, "60000");
-        // conf.set(ScannerCallable.LOG_SCANNER_ACTIVITY, "true");
-        if (StringUtils.isEmpty(url)) {
-            return conf;
-        }
-
-        // chop off "hbase"
-        if (url.startsWith("hbase") == false) {
-            throw new IllegalArgumentException("hbase url must start with 'hbase' -- " + url);
-        }
-
-        url = StringUtils.substringAfter(url, "hbase");
-        if (StringUtils.isEmpty(url)) {
-            return conf;
-        }
-
-        // case of "hbase:domain.com:2181:/hbase-unsecure"
-        Pattern urlPattern = Pattern.compile("[:]((?:[\\w\\-.]+)(?:\\,[\\w\\-.]+)*)[:](\\d+)(?:[:](.+))");
-        Matcher m = urlPattern.matcher(url);
-        if (m.matches() == false)
-            throw new IllegalArgumentException("HBase URL '" + url + "' is invalid, expected url is like '" + "hbase:domain.com:2181:/hbase-unsecure" + "'");
-
-        logger.debug("Creating hbase conf by parsing -- " + url);
-
-        String quorums = m.group(1);
-        String quorum = null;
-        try {
-            String[] tokens = quorums.split(",");
-            for (String s : tokens) {
-                quorum = s;
-                InetAddress.getByName(quorum);
-            }
-        } catch (UnknownHostException e) {
-            throw new IllegalArgumentException("Zookeeper quorum is invalid: " + quorum + "; urlString=" + url, e);
-        }
-        conf.set(HConstants.ZOOKEEPER_QUORUM, quorums);
-
-        String port = m.group(2);
-        conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, port);
-
-        String znodePath = m.group(3);
-        conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, znodePath);
-
-        return conf;
-    }
-
-    /**
-     * @param table the identifier of hive table, in format <db_name>.<table_name>
-     * @return a string array with 2 elements: {"db_name", "table_name"}
-     */
-    public static String[] parseHiveTableName(String table) {
-        int cut = table.indexOf('.');
-        String database = cut >= 0 ? table.substring(0, cut).trim() : "DEFAULT";
-        String tableName = cut >= 0 ? table.substring(cut + 1).trim() : table.trim();
-
-        return new String[] { database, tableName };
-    }
-
-    public static void deletePath(Configuration conf, Path path) throws IOException {
-        FileSystem fs = FileSystem.get(conf);
-        if (fs.exists(path)) {
-            fs.delete(path, true);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/util/HiveClient.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/HiveClient.java b/core-common/src/main/java/org/apache/kylin/common/util/HiveClient.java
deleted file mode 100644
index a5be14e..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/util/HiveClient.java
+++ /dev/null
@@ -1,162 +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.common.util;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.common.StatsSetupConst;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
-import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.apache.hadoop.hive.ql.session.SessionState;
-
-/**
- * Hive meta API client for Kylin
- * @author shaoshi
- *
- */
-public class HiveClient {
-
-    protected HiveConf hiveConf = null;
-    protected Driver driver = null;
-    protected HiveMetaStoreClient metaStoreClient = null;
-
-    public HiveClient() {
-        hiveConf = new HiveConf(HiveClient.class);
-    }
-
-    public HiveClient(Map<String, String> configMap) {
-        this();
-        appendConfiguration(configMap);
-    }
-
-    public HiveConf getHiveConf() {
-        return hiveConf;
-    }
-
-    /**
-     * Get the hive ql driver to execute ddl or dml
-     * @return
-     */
-    private Driver getDriver() {
-        if (driver == null) {
-            driver = new Driver(hiveConf);
-            SessionState.start(new CliSessionState(hiveConf));
-        }
-
-        return driver;
-    }
-
-    /**
-     * Append or overwrite the default hive client configuration; You need call this before invoke #executeHQL;
-     * @param configMap
-     */
-    public void appendConfiguration(Map<String, String> configMap) {
-        if (configMap != null && configMap.size() > 0) {
-            for (Entry<String, String> e : configMap.entrySet()) {
-                hiveConf.set(e.getKey(), e.getValue());
-            }
-        }
-    }
-
-    /**
-     * 
-     * @param hql
-     * @throws CommandNeedRetryException
-     * @throws IOException
-     */
-    public void executeHQL(String hql) throws CommandNeedRetryException, IOException {
-        CommandProcessorResponse response = getDriver().run(hql);
-        int retCode = response.getResponseCode();
-        if (retCode != 0) {
-            String err = response.getErrorMessage();
-            throw new IOException("Failed to execute hql [" + hql + "], error message is: " + err);
-        }
-    }
-
-    public void executeHQL(String[] hqls) throws CommandNeedRetryException, IOException {
-        for (String sql : hqls)
-            executeHQL(sql);
-    }
-
-    private HiveMetaStoreClient getMetaStoreClient() throws Exception {
-        if (metaStoreClient == null) {
-            metaStoreClient = new HiveMetaStoreClient(hiveConf);
-        }
-        return metaStoreClient;
-    }
-
-    public Table getHiveTable(String database, String tableName) throws Exception {
-        return getMetaStoreClient().getTable(database, tableName);
-    }
-
-    public List<FieldSchema> getHiveTableFields(String database, String tableName) throws Exception {
-        return getMetaStoreClient().getFields(database, tableName);
-    }
-
-    public String getHiveTableLocation(String database, String tableName) throws Exception {
-        Table t = getHiveTable(database, tableName);
-        return t.getSd().getLocation();
-    }
-
-    public long getFileSizeForTable(Table table) {
-        return getBasicStatForTable(new org.apache.hadoop.hive.ql.metadata.Table(table), StatsSetupConst.TOTAL_SIZE);
-    }
-
-    public long getFileNumberForTable(Table table) {
-        return getBasicStatForTable(new org.apache.hadoop.hive.ql.metadata.Table(table), StatsSetupConst.NUM_FILES);
-    }
-
-    /**
-     * COPIED FROM org.apache.hadoop.hive.ql.stats.StatsUtil for backward compatibility
-     * 
-     * Get basic stats of table
-     * @param table
-     *          - table
-     * @param statType
-     *          - type of stats
-     * @return value of stats
-     */
-    public static long getBasicStatForTable(org.apache.hadoop.hive.ql.metadata.Table table, String statType) {
-        Map<String, String> params = table.getParameters();
-        long result = 0;
-
-        if (params != null) {
-            try {
-                result = Long.parseLong(params.get(statType));
-            } catch (NumberFormatException e) {
-                result = 0;
-            }
-        }
-        return result;
-    }
-
-    public boolean isNativeTable(String database, String tableName)  throws Exception{
-        return !MetaStoreUtils.isNonNativeTable(getMetaStoreClient().getTable(database, tableName));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/util/JsonUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/JsonUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/JsonUtil.java
index 5b3a22c..216abae 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/JsonUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/JsonUtil.java
@@ -18,23 +18,29 @@
 
 package org.apache.kylin.common.util;
 
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.util.HashMap;
+import java.util.Map;
+
 import com.fasterxml.jackson.core.JsonGenerationException;
 import com.fasterxml.jackson.core.JsonParseException;
 import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.DeserializationFeature;
 import com.fasterxml.jackson.databind.JsonMappingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializationFeature;
 
-import java.io.*;
-
 public class JsonUtil {
 
     // reuse the object mapper to save memory footprint
     private static final ObjectMapper mapper = new ObjectMapper();
     private static final ObjectMapper indentMapper = new ObjectMapper();
 
-
     static {
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
         indentMapper.configure(SerializationFeature.INDENT_OUTPUT, true);
@@ -60,6 +66,12 @@ public class JsonUtil {
         return mapper.readValue(src, valueType);
     }
 
+    public static Map<String, String> readValueAsMap(String content) throws IOException {
+        TypeReference<HashMap<String, String>> typeRef = new TypeReference<HashMap<String, String>>() {
+        };
+        return mapper.readValue(content, typeRef);
+    }
+
     public static void writeValueIndent(OutputStream out, Object value) throws IOException, JsonGenerationException, JsonMappingException {
         indentMapper.writeValue(out, value);
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/util/TarGZUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/TarGZUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/TarGZUtil.java
deleted file mode 100644
index 6f623d3..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/util/TarGZUtil.java
+++ /dev/null
@@ -1,69 +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.common.util;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-
-import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
-import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
-import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
-
-public class TarGZUtil {
-
-    public static void uncompressTarGZ(File tarFile, File dest) throws IOException {
-        dest.mkdir();
-        TarArchiveInputStream tarIn = null;
-
-        tarIn = new TarArchiveInputStream(new GzipCompressorInputStream(new BufferedInputStream(new FileInputStream(tarFile))));
-
-        TarArchiveEntry tarEntry = tarIn.getNextTarEntry();
-        // tarIn is a TarArchiveInputStream
-        while (tarEntry != null) {// create a file with the same name as the tarEntry
-            File destPath = new File(dest, tarEntry.getName());
-            System.out.println("working: " + destPath.getCanonicalPath());
-            if (tarEntry.isDirectory()) {
-                destPath.mkdirs();
-            } else {
-                destPath.createNewFile();
-                //byte [] btoRead = new byte[(int)tarEntry.getSize()];
-                byte[] btoRead = new byte[1024];
-                //FileInputStream fin 
-                //  = new FileInputStream(destPath.getCanonicalPath());
-                BufferedOutputStream bout = new BufferedOutputStream(new FileOutputStream(destPath));
-                int len = 0;
-
-                while ((len = tarIn.read(btoRead)) != -1) {
-                    bout.write(btoRead, 0, len);
-                }
-
-                bout.close();
-                btoRead = null;
-
-            }
-            tarEntry = tarIn.getNextTarEntry();
-        }
-        tarIn.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/test/java/org/apache/kylin/common/persistence/ITHBaseResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/persistence/ITHBaseResourceStoreTest.java b/core-common/src/test/java/org/apache/kylin/common/persistence/ITHBaseResourceStoreTest.java
deleted file mode 100644
index 6195423..0000000
--- a/core-common/src/test/java/org/apache/kylin/common/persistence/ITHBaseResourceStoreTest.java
+++ /dev/null
@@ -1,208 +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.common.persistence;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-
-import static org.junit.Assert.*;
-
-public class ITHBaseResourceStoreTest extends HBaseMetadataTestCase {
-
-    @Before
-    public void setup() throws Exception {
-        this.createTestMetadata();
-    }
-
-    @After
-    public void after() throws Exception {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void testHBaseStore() throws Exception {
-        testAStore(ResourceStore.getStore(KylinConfig.getInstanceFromEnv()));
-    }
-
-    @Test
-    public void testHBaseStoreWithLargeCell() throws Exception {
-        String path = "/cube/_test_large_cell.json";
-        String largeContent = "THIS_IS_A_LARGE_CELL";
-        StringEntity content = new StringEntity(largeContent);
-        KylinConfig config = KylinConfig.getInstanceFromEnv();
-        int origSize = config.getHBaseKeyValueSize();
-        ResourceStore store = ResourceStore.getStore(KylinConfig.getInstanceFromEnv());
-
-        try {
-            config.setProperty("kylin.hbase.client.keyvalue.maxsize", String.valueOf(largeContent.length() - 1));
-
-            store.deleteResource(path);
-
-            store.putResource(path, content, StringEntity.serializer);
-            assertTrue(store.exists(path));
-            StringEntity t = store.getResource(path, StringEntity.class, StringEntity.serializer);
-            assertEquals(content, t);
-
-            Path redirectPath = ((HBaseResourceStore) store).bigCellHDFSPath(path);
-            Configuration hconf = HadoopUtil.getCurrentConfiguration();
-            FileSystem fileSystem = FileSystem.get(hconf);
-            assertTrue(fileSystem.exists(redirectPath));
-
-            FSDataInputStream in = fileSystem.open(redirectPath);
-            assertEquals(largeContent, in.readUTF());
-            in.close();
-
-            store.deleteResource(path);
-        } finally {
-            config.setProperty("kylin.hbase.client.keyvalue.maxsize", "" + origSize);
-            store.deleteResource(path);
-        }
-    }
-
-    void testAStore(ResourceStore store) throws IOException {
-        String dir1 = "/cube";
-        String path1 = "/cube/_test.json";
-        StringEntity content1 = new StringEntity("anything");
-        String dir2 = "/table";
-        String path2 = "/table/_test.json";
-        StringEntity content2 = new StringEntity("something");
-
-        // cleanup legacy if any
-        store.deleteResource(path1);
-        store.deleteResource(path2);
-
-        StringEntity t;
-
-        // put/get
-        store.putResource(path1, content1, StringEntity.serializer);
-        assertTrue(store.exists(path1));
-        t = store.getResource(path1, StringEntity.class, StringEntity.serializer);
-        assertEquals(content1, t);
-
-        store.putResource(path2, content2, StringEntity.serializer);
-        assertTrue(store.exists(path2));
-        t = store.getResource(path2, StringEntity.class, StringEntity.serializer);
-        assertEquals(content2, t);
-
-        // overwrite
-        t.str = "new string";
-        store.putResource(path2, t, StringEntity.serializer);
-
-        // write conflict
-        try {
-            t.setLastModified(t.lastModified - 1);
-            store.putResource(path2, t, StringEntity.serializer);
-            fail("write conflict should trigger IllegalStateException");
-        } catch (IllegalStateException e) {
-            // expected
-        }
-
-        // list
-        ArrayList<String> list;
-
-        list = store.listResources(dir1);
-        assertTrue(list.contains(path1));
-        assertTrue(list.contains(path2) == false);
-
-        list = store.listResources(dir2);
-        assertTrue(list.contains(path2));
-        assertTrue(list.contains(path1) == false);
-
-        list = store.listResources("/");
-        assertTrue(list.contains(dir1));
-        assertTrue(list.contains(dir2));
-        assertTrue(list.contains(path1) == false);
-        assertTrue(list.contains(path2) == false);
-
-        list = store.listResources(path1);
-        assertNull(list);
-        list = store.listResources(path2);
-        assertNull(list);
-
-        // delete/exist
-        store.deleteResource(path1);
-        assertTrue(store.exists(path1) == false);
-        list = store.listResources(dir1);
-        assertTrue(list == null || list.contains(path1) == false);
-
-        store.deleteResource(path2);
-        assertTrue(store.exists(path2) == false);
-        list = store.listResources(dir2);
-        assertTrue(list == null || list.contains(path2) == false);
-    }
-
-    public static class StringEntity extends RootPersistentEntity {
-
-        static final Serializer<StringEntity> serializer = new Serializer<StringEntity>() {
-            @Override
-            public void serialize(StringEntity obj, DataOutputStream out) throws IOException {
-                out.writeUTF(obj.str);
-            }
-
-            @Override
-            public StringEntity deserialize(DataInputStream in) throws IOException {
-                String str = in.readUTF();
-                return new StringEntity(str);
-            }
-        };
-
-        String str;
-
-        public StringEntity(String str) {
-            this.str = str;
-        }
-
-        @Override
-        public int hashCode() {
-            final int prime = 31;
-            int result = super.hashCode();
-            result = prime * result + ((str == null) ? 0 : str.hashCode());
-            return result;
-        }
-
-        @Override
-        public boolean equals(Object obj) {
-            if (obj == this)
-                return true;
-            if (!(obj instanceof StringEntity))
-                return false;
-            return StringUtils.equals(this.str, ((StringEntity) obj).str);
-        }
-
-        @Override
-        public String toString() {
-            return str;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/test/java/org/apache/kylin/common/util/HBaseMetadataTestCase.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/HBaseMetadataTestCase.java b/core-common/src/test/java/org/apache/kylin/common/util/HBaseMetadataTestCase.java
deleted file mode 100644
index 6473b3d..0000000
--- a/core-common/src/test/java/org/apache/kylin/common/util/HBaseMetadataTestCase.java
+++ /dev/null
@@ -1,73 +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.common.util;
-
-import org.apache.kylin.common.KylinConfig;
-
-import java.io.File;
-
-/**
- * @author ysong1
- */
-public class HBaseMetadataTestCase extends AbstractKylinTestCase {
-
-    static {
-        if (useSandbox()) {
-            try {
-                ClassUtil.addClasspath(new File("../examples/test_case_data/sandbox/").getAbsolutePath());
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    @Override
-    public void createTestMetadata() throws Exception {
-        staticCreateTestMetadata();
-    }
-
-    @Override
-    public void cleanupTestMetadata() {
-        staticCleanupTestMetadata();
-    }
-
-    public static void staticCreateTestMetadata() throws Exception {
-        if (useSandbox()) {
-            staticCreateTestMetadata(SANDBOX_TEST_DATA);
-        } else {
-            staticCreateTestMetadata(MINICLUSTER_TEST_DATA);
-            HBaseMiniclusterHelper.startupMinicluster();
-        }
-
-    }
-    public static void staticCreateTestMetadata(String kylinConfigFolder) {
-
-        KylinConfig.destoryInstance();
-
-        if (System.getProperty(KylinConfig.KYLIN_CONF) == null && System.getenv(KylinConfig.KYLIN_CONF) == null)
-            System.setProperty(KylinConfig.KYLIN_CONF, kylinConfigFolder);
-
-    }
-
-    public static boolean useSandbox() {
-        String useSandbox = System.getProperty("useSandbox");
-        return Boolean.parseBoolean(useSandbox);
-    }
-    
-}


[6/6] incubator-kylin git commit: KYLIN-875 Refactor core-common, remove dependency on hadoop/hbase

Posted by li...@apache.org.
KYLIN-875 Refactor core-common, remove dependency on hadoop/hbase


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

Branch: refs/heads/0.8
Commit: d2456215cc881358025561d0375e787fb960daf1
Parents: dd5f2b2
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Jul 22 18:49:06 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Jul 22 18:49:46 2015 +0800

----------------------------------------------------------------------
 core-common/pom.xml                             |  64 +-
 .../org/apache/kylin/common/lock/JobLock.java   |   9 -
 .../apache/kylin/common/lock/MockJobLock.java   |  15 -
 .../kylin/common/lock/ZookeeperJobLock.java     |  83 ---
 .../org/apache/kylin/common/mr/KylinMapper.java |  33 --
 .../apache/kylin/common/mr/KylinReducer.java    |  31 -
 .../common/persistence/HBaseConnection.java     | 167 ------
 .../common/persistence/HBaseResourceStore.java  | 323 ----------
 .../kylin/common/persistence/ResourceStore.java |  13 +-
 .../kylin/common/restclient/RestClient.java     |  12 +-
 .../org/apache/kylin/common/util/Bytes.java     | 557 ++++-------------
 .../apache/kylin/common/util/BytesSplitter.java |   8 +-
 .../org/apache/kylin/common/util/BytesUtil.java |  26 +-
 .../common/util/HBaseRegionSizeCalculator.java  | 128 ----
 .../apache/kylin/common/util/HadoopUtil.java    | 162 -----
 .../apache/kylin/common/util/HiveClient.java    | 162 -----
 .../org/apache/kylin/common/util/JsonUtil.java  |  18 +-
 .../org/apache/kylin/common/util/TarGZUtil.java |  69 ---
 .../persistence/ITHBaseResourceStoreTest.java   | 208 -------
 .../common/util/HBaseMetadataTestCase.java      |  73 ---
 .../common/util/HBaseMiniclusterHelper.java     | 167 ------
 .../apache/kylin/common/util/HbaseImporter.java | 121 ----
 .../kylin/common/util/IdentityUtilTest.java     |   9 +-
 .../kylin/common/util/RandomSamplerTest.java    |  11 +-
 .../org/apache/kylin/common/util/RangeTest.java |   1 +
 .../apache/kylin/common/util/SSHClientTest.java |   6 +-
 core-cube/pom.xml                               |   4 +-
 .../java/org/apache/kylin/cube/CubeManager.java |   8 +-
 .../kylin/cube/cli/DictionaryGeneratorCLI.java  |  13 +-
 core-dictionary/pom.xml                         |   4 +-
 .../apache/kylin/dict/DictionaryManager.java    |  74 +--
 .../dict/DistinctColumnValuesProvider.java      |  17 +
 .../org/apache/kylin/dict/lookup/FileTable.java | 100 ----
 .../kylin/dict/lookup/FileTableReader.java      | 220 -------
 .../org/apache/kylin/dict/LookupTableTest.java  |  76 ---
 .../org/apache/kylin/dict/TableReaderTest.java  |  48 --
 .../java/org/apache/kylin/job/lock/JobLock.java |   9 +
 .../org/apache/kylin/job/lock/MockJobLock.java  |  15 +
 core-metadata/pom.xml                           |   5 +-
 engine-mr/pom.xml                               |  40 ++
 .../apache/kylin/engine/mr/DFSFileTable.java    |  99 ++++
 .../kylin/engine/mr/DFSFileTableReader.java     | 219 +++++++
 .../org/apache/kylin/engine/mr/HadoopUtil.java  | 109 ++++
 .../org/apache/kylin/engine/mr/KylinMapper.java |  32 +
 .../apache/kylin/engine/mr/KylinReducer.java    |  30 +
 .../apache/kylin/engine/mr/LookupTableTest.java |  74 +++
 .../apache/kylin/engine/mr/TableReaderTest.java |  46 ++
 invertedindex/pom.xml                           |  53 +-
 .../apache/kylin/invertedindex/IIInstance.java  |  24 +-
 .../apache/kylin/invertedindex/IIManager.java   |   6 +-
 .../apache/kylin/invertedindex/tools/IICLI.java | 108 ----
 job/dependency-reduced-pom.xml                  | 594 -------------------
 job/pom.xml                                     |  80 +--
 .../engine/mr/steps/InMemCuboidMapper.java      |   2 +-
 .../engine/mr/steps/InMemCuboidReducer.java     |   2 +-
 .../mr/steps/MergeCuboidFromStorageMapper.java  |   2 +-
 .../engine/mr/steps/MergeStatisticsStep.java    |   3 +-
 .../engine/mr/steps/SaveStatisticsStep.java     |   2 +-
 .../java/org/apache/kylin/job/Scheduler.java    |   2 +-
 .../apache/kylin/job/common/HqlExecutable.java  |   3 +-
 .../kylin/job/hadoop/AbstractHadoopJob.java     |   2 +-
 .../cardinality/ColumnCardinalityMapper.java    |   2 +-
 .../cardinality/ColumnCardinalityReducer.java   |   3 +-
 .../job/hadoop/cube/BaseCuboidMapperBase.java   |   3 +-
 .../kylin/job/hadoop/cube/CubeHFileMapper.java  |   2 +-
 .../kylin/job/hadoop/cube/CuboidReducer.java    |   3 +-
 .../cube/FactDistinctColumnsCombiner.java       |   3 +-
 .../cube/FactDistinctColumnsMapperBase.java     |   2 +-
 .../hadoop/cube/FactDistinctColumnsReducer.java |   3 +-
 .../job/hadoop/cube/MergeCuboidMapper.java      |   3 +-
 .../kylin/job/hadoop/cube/NDCuboidMapper.java   |   4 +-
 .../job/hadoop/cube/NewBaseCuboidMapper.java    |   2 +-
 .../hadoop/cube/RangeKeyDistributionMapper.java |   2 +-
 .../cube/RangeKeyDistributionReducer.java       |   3 +-
 .../cube/RowKeyDistributionCheckerMapper.java   |   3 +-
 .../cube/RowKeyDistributionCheckerReducer.java  |   3 +-
 .../job/hadoop/dict/CreateDictionaryJob.java    |  20 +-
 .../dict/CreateInvertedIndexDictionaryJob.java  |  19 +-
 .../kylin/job/hadoop/hbase/CreateHTableJob.java |   2 +-
 .../job/hadoop/invertedindex/IIBulkLoadJob.java |  16 +-
 .../hadoop/invertedindex/IICreateHFileJob.java  |   4 +-
 .../invertedindex/IICreateHFileMapper.java      |   2 +-
 .../hadoop/invertedindex/IICreateHTableJob.java |   4 +-
 .../IIDistinctColumnsCombiner.java              |   3 +-
 .../invertedindex/IIDistinctColumnsJob.java     |   2 +-
 .../invertedindex/IIDistinctColumnsMapper.java  |   3 +-
 .../invertedindex/IIDistinctColumnsReducer.java |   3 +-
 .../hadoop/invertedindex/InvertedIndexJob.java  |   2 +-
 .../invertedindex/InvertedIndexMapper.java      |   2 +-
 .../invertedindex/InvertedIndexReducer.java     |   2 +-
 .../RandomKeyDistributionMapper.java            |   3 +-
 .../RandomKeyDistributionReducer.java           |   3 +-
 .../job/impl/threadpool/DefaultScheduler.java   |   3 +-
 .../kylin/job/streaming/CubeStreamConsumer.java |   4 +-
 .../kylin/job/streaming/StreamingBootstrap.java |   5 +-
 .../kylin/job/tools/DeployCoprocessorCLI.java   |   5 +-
 .../job/tools/GridTableHBaseBenchmark.java      |   2 +-
 .../java/org/apache/kylin/job/tools/IICLI.java  | 108 ++++
 .../org/apache/kylin/job/tools/TarGZUtil.java   |  69 +++
 .../apache/kylin/source/hive/HiveMRInput.java   |   2 +-
 .../source/hive/HiveSourceTableLoader.java      |   4 +-
 .../org/apache/kylin/source/hive/HiveTable.java |   5 +-
 .../kylin/storage/hbase/HBaseMROutput2.java     |   2 +-
 .../kylin/job/BuildCubeWithEngineTest.java      |   4 +-
 .../kylin/job/BuildCubeWithStreamTest.java      |   2 +-
 .../apache/kylin/job/BuildIIWithEngineTest.java |   5 +-
 .../apache/kylin/job/BuildIIWithStreamTest.java |   2 +-
 .../kylin/job/DeployLocalMetaToRemoteTest.java  |   2 +-
 .../java/org/apache/kylin/job/DeployUtil.java   |   3 +-
 .../org/apache/kylin/job/ExportHBaseData.java   |   5 +-
 .../job/ITKafkaBasedIIStreamBuilderTest.java    |   2 +-
 .../org/apache/kylin/job/ImportHBaseData.java   |   9 +-
 .../cubev2/FactDistinctColumnsReducerTest.java  |   3 +-
 .../kylin/job/hadoop/hdfs/ITHdfsOpsTest.java    |   2 +-
 .../job/impl/threadpool/BaseSchedulerTest.java  |   4 +-
 .../job/inmemcubing/InMemCubeBuilderTest.java   |   6 +-
 .../job/streaming/CubeStreamConsumerTest.java   |   3 +-
 .../hive/ITHiveSourceTableLoaderTest.java       |   2 +-
 .../source/hive/ITHiveTableReaderTest.java      |   2 +-
 .../source/hive/ITSnapshotManagerTest.java      |   2 +-
 query/pom.xml                                   |  21 +-
 .../kylin/query/test/ITKylinQueryTest.java      |   2 +-
 server/pom.xml                                  |  25 +-
 .../kylin/rest/controller/JobController.java    |   2 +-
 .../rest/security/RealAclHBaseStorage.java      |   2 +-
 .../apache/kylin/rest/service/CubeService.java  |   8 +-
 .../apache/kylin/rest/service/QueryService.java |   2 +-
 .../org/apache/kylin/jdbc/ITJDBCDriverTest.java |   2 +-
 source-hive/pom.xml                             |  11 +
 .../apache/kylin/source/hive/HiveClient.java    | 162 +++++
 storage-hbase/pom.xml                           |  38 ++
 .../kylin/storage/hbase/HBaseConnection.java    | 234 ++++++++
 .../hbase/HBaseRegionSizeCalculator.java        | 128 ++++
 .../kylin/storage/hbase/HBaseResourceStore.java | 326 ++++++++++
 .../kylin/storage/hbase/ZookeeperJobLock.java   |  83 +++
 .../storage/hbase/HBaseMetadataTestCase.java    |  75 +++
 .../storage/hbase/HBaseMiniclusterHelper.java   | 167 ++++++
 .../kylin/storage/hbase/HbaseImporter.java      | 121 ++++
 .../storage/hbase/ITHBaseResourceStoreTest.java | 211 +++++++
 storage/pom.xml                                 |  30 +-
 .../storage/cube/CubeHBaseReadonlyStore.java    |   2 +-
 .../gridtable/diskstore/HadoopFileSystem.java   |   2 +-
 .../kylin/storage/hbase/CubeStorageQuery.java   |   2 +-
 .../hbase/InvertedIndexStorageQuery.java        |   2 +-
 .../kylin/storage/hbase/PingHBaseCLI.java       |   6 +-
 .../storage/hbase/ITInvertedIndexHBaseTest.java |  10 +-
 .../kylin/storage/test/ITStorageTest.java       |   2 +-
 streaming/pom.xml                               |  45 +-
 .../invertedindex/IIStreamConsumer.java         |   3 +-
 .../streaming/invertedindex/SliceBuilder.java   |  45 +-
 150 files changed, 2924 insertions(+), 3844 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/pom.xml
----------------------------------------------------------------------
diff --git a/core-common/pom.xml b/core-common/pom.xml
index c71a4ad..adb189e 100644
--- a/core-common/pom.xml
+++ b/core-common/pom.xml
@@ -60,6 +60,10 @@
             <artifactId>commons-email</artifactId>
         </dependency>
         <dependency>
+            <groupId>commons-httpclient</groupId>
+            <artifactId>commons-httpclient</artifactId>
+        </dependency>
+        <dependency>
             <groupId>com.google.guava</groupId>
             <artifactId>guava</artifactId>
         </dependency>
@@ -76,60 +80,30 @@
             <artifactId>compress-lzf</artifactId>
         </dependency>
 
-        <!-- Env & Test -->
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-common</artifactId>
-            <scope>provided</scope>
-        </dependency>
+		<!-- Logging -->
         <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-hdfs</artifactId>
-            <scope>provided</scope>
-            <!-- protobuf version conflict with hbase -->
-            <exclusions>
-                <exclusion>
-                    <groupId>com.google.protobuf</groupId>
-                    <artifactId>protobuf-java</artifactId>
-                </exclusion>
-            </exclusions>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-common</artifactId>
-            <scope>provided</scope>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-client</artifactId>
-            <scope>provided</scope>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-server</artifactId>
-            <scope>provided</scope>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
         </dependency>
+
+        <!-- Env & Test -->
         <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-testing-util</artifactId>
-            <version>${hbase-hadoop2.version}</version>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
             <scope>test</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>javax.servlet</groupId>
-                    <artifactId>servlet-api</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>javax.servlet.jsp</groupId>
-                    <artifactId>jsp-api</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-         <dependency>
-            <groupId>org.apache.hive.hcatalog</groupId>
-            <artifactId>hive-hcatalog-core</artifactId>
-            <version>${hive-hcatalog.version}</version>
-            <scope>provided</scope>
-          </dependency>
+        </dependency>
+        
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/lock/JobLock.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/lock/JobLock.java b/core-common/src/main/java/org/apache/kylin/common/lock/JobLock.java
deleted file mode 100644
index 7fdb64c..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/lock/JobLock.java
+++ /dev/null
@@ -1,9 +0,0 @@
-package org.apache.kylin.common.lock;
-
-/**
- */
-public interface JobLock {
-    boolean lock();
-
-    void unlock();
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/lock/MockJobLock.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/lock/MockJobLock.java b/core-common/src/main/java/org/apache/kylin/common/lock/MockJobLock.java
deleted file mode 100644
index 230d4d8..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/lock/MockJobLock.java
+++ /dev/null
@@ -1,15 +0,0 @@
-package org.apache.kylin.common.lock;
-
-/**
- */
-public class MockJobLock implements JobLock {
-    @Override
-    public boolean lock() {
-        return true;
-    }
-
-    @Override
-    public void unlock() {
-        return;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/lock/ZookeeperJobLock.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/lock/ZookeeperJobLock.java b/core-common/src/main/java/org/apache/kylin/common/lock/ZookeeperJobLock.java
deleted file mode 100644
index 603894c..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/lock/ZookeeperJobLock.java
+++ /dev/null
@@ -1,83 +0,0 @@
-package org.apache.kylin.common.lock;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.curator.RetryPolicy;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.framework.imps.CuratorFrameworkState;
-import org.apache.curator.framework.recipes.locks.InterProcessMutex;
-import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- */
-public class ZookeeperJobLock implements JobLock {
-    private Logger logger = LoggerFactory.getLogger(ZookeeperJobLock.class);
-
-    private static final String ZOOKEEPER_LOCK_PATH = "/kylin/job_engine/lock";
-
-    private String scheduleID;
-    private InterProcessMutex sharedLock;
-    private CuratorFramework zkClient;
-
-    @Override
-    public boolean lock() {
-        this.scheduleID = schedulerId();
-        String ZKConnectString = getZKConnectString();
-        if (StringUtils.isEmpty(ZKConnectString)) {
-            throw new IllegalArgumentException("ZOOKEEPER_QUORUM is empty!");
-        }
-
-        RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
-        this.zkClient = CuratorFrameworkFactory.newClient(ZKConnectString, retryPolicy);
-        this.zkClient.start();
-        this.sharedLock = new InterProcessMutex(zkClient, this.scheduleID);
-        boolean hasLock = false;
-        try {
-            hasLock = sharedLock.acquire(3, TimeUnit.SECONDS);
-        } catch (Exception e) {
-            logger.warn("error acquire lock", e);
-        }
-        if (!hasLock) {
-            logger.warn("fail to acquire lock, scheduler has not been started");
-            zkClient.close();
-            return false;
-        }
-        return true;
-    }
-
-    @Override
-    public void unlock() {
-        releaseLock();
-    }
-
-    private String getZKConnectString() {
-        Configuration conf = HadoopUtil.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl());
-        return conf.get(HConstants.ZOOKEEPER_QUORUM) + ":" + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT);
-    }
-
-    private void releaseLock() {
-        try {
-            if (zkClient.getState().equals(CuratorFrameworkState.STARTED)) {
-                // client.setData().forPath(ZOOKEEPER_LOCK_PATH, null);
-                if (zkClient.checkExists().forPath(scheduleID) != null) {
-                    zkClient.delete().guaranteed().deletingChildrenIfNeeded().forPath(scheduleID);
-                }
-            }
-        } catch (Exception e) {
-            logger.error("error release lock:" + scheduleID);
-            throw new RuntimeException(e);
-        }
-    }
-
-    private String schedulerId() {
-        return ZOOKEEPER_LOCK_PATH + "/" + KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/mr/KylinMapper.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/mr/KylinMapper.java b/core-common/src/main/java/org/apache/kylin/common/mr/KylinMapper.java
deleted file mode 100644
index fd0b337..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/mr/KylinMapper.java
+++ /dev/null
@@ -1,33 +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.common.mr;
-
-
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Mapper;
-
-/**
- */
-public class KylinMapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> extends Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
-    
-    protected void bindCurrentConfiguration(Configuration conf) {
-        HadoopUtil.setCurrentConfiguration(conf);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/mr/KylinReducer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/mr/KylinReducer.java b/core-common/src/main/java/org/apache/kylin/common/mr/KylinReducer.java
deleted file mode 100644
index bc63df7..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/mr/KylinReducer.java
+++ /dev/null
@@ -1,31 +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.common.mr;
-
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Reducer;
-
-/**
- */
-public class KylinReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> extends Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
-    protected void bindCurrentConfiguration(Configuration conf) {
-        HadoopUtil.setCurrentConfiguration(conf);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java b/core-common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
deleted file mode 100644
index a574d0b..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
+++ /dev/null
@@ -1,167 +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.common.persistence;
-
-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.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;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * @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();
-                    }
-                }
-            }
-        });
-    }
-
-    // returned HConnection can be shared by multiple threads and does not require close()
-    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 {
-            while (true) {
-                // I don't use DCL since recreate a connection is not a big issue.
-                if (connection == null || connection.isClosed()) {
-                    logger.info("connection is null or closed, creating a new one");
-                    connection = HConnectionManager.createConnection(conf);
-                    ConnPool.put(url, connection);
-                }
-
-                if (connection == null || connection.isClosed()) {
-                    Thread.sleep(10000);// wait a while and retry
-                } else {
-                    break;
-                }
-            }
-
-        } catch (Throwable t) {
-            logger.error("Error when open connection " + url, t);
-            throw new StorageException("Error when open connection " + url, t);
-        }
-
-        return connection;
-    }
-
-    public static boolean tableExists(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
-        try {
-            return hbase.tableExists(TableName.valueOf(tableName));
-        } finally {
-            hbase.close();
-        }
-    }
-
-    public static boolean tableExists(String hbaseUrl, String tableName) throws IOException {
-        return tableExists(HBaseConnection.get(hbaseUrl), tableName);
-    }
-
-    public static void createHTableIfNeeded(String hbaseUrl, String tableName, String... families) throws IOException {
-        createHTableIfNeeded(HBaseConnection.get(hbaseUrl), tableName, families);
-    }
-
-    public static void deleteTable(String hbaseUrl, String tableName) throws IOException {
-        deleteTable(HBaseConnection.get(hbaseUrl), tableName);
-    }
-
-    public static void createHTableIfNeeded(HConnection conn, String tableName, String... families) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
-
-        try {
-            if (tableExists(conn, tableName)) {
-                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();
-        }
-    }
-
-    public static void deleteTable(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
-
-        try {
-            if (!tableExists(conn, tableName)) {
-                logger.debug("HTable '" + tableName + "' does not exists");
-                return;
-            }
-
-            logger.debug("delete HTable '" + tableName + "'");
-
-            if (hbase.isTableEnabled(tableName)) {
-                hbase.disableTable(tableName);
-            }
-            hbase.deleteTable(tableName);
-
-            logger.debug("HTable '" + tableName + "' deleted");
-        } finally {
-            hbase.close();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java b/core-common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
deleted file mode 100644
index 6c5847e..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
+++ /dev/null
@@ -1,323 +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.common.persistence;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.io.IOUtils;
-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.client.*;
-import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.HadoopUtil;
-
-import java.io.*;
-import java.util.*;
-
-public class HBaseResourceStore extends ResourceStore {
-
-    private static final String DEFAULT_TABLE_NAME = "kylin_metadata";
-    private static final String FAMILY = "f";
-    private static final byte[] B_FAMILY = Bytes.toBytes(FAMILY);
-    private static final String COLUMN = "c";
-    private static final byte[] B_COLUMN = Bytes.toBytes(COLUMN);
-    private static final String COLUMN_TS = "t";
-    private static final byte[] B_COLUMN_TS = Bytes.toBytes(COLUMN_TS);
-
-    private static final Map<String, String> TABLE_SUFFIX_MAP = new LinkedHashMap<String, String>();
-
-    static {
-        TABLE_SUFFIX_MAP.put(CUBE_RESOURCE_ROOT + "/", "_cube");
-        TABLE_SUFFIX_MAP.put(DICT_RESOURCE_ROOT + "/", "_dict");
-        TABLE_SUFFIX_MAP.put("/invertedindex/", "_invertedindex");
-        TABLE_SUFFIX_MAP.put(JOB_PATH_ROOT + "/", "_job");
-        TABLE_SUFFIX_MAP.put(JOB_OUTPUT_PATH_ROOT + "/", "_job_output");
-        TABLE_SUFFIX_MAP.put(PROJECT_RESOURCE_ROOT + "/", "_proj");
-        TABLE_SUFFIX_MAP.put(SNAPSHOT_RESOURCE_ROOT + "/", "_table_snapshot");
-        TABLE_SUFFIX_MAP.put("", ""); // DEFAULT CASE
-    }
-
-    final String tableNameBase;
-    final String hbaseUrl;
-
-    //    final Map<String, String> tableNameMap; // path prefix ==> HBase table name
-
-    private HConnection getConnection() throws IOException {
-        return HBaseConnection.get(hbaseUrl);
-    }
-
-    public HBaseResourceStore(KylinConfig kylinConfig) throws IOException {
-        super(kylinConfig);
-
-        String metadataUrl = kylinConfig.getMetadataUrl();
-        // split TABLE@HBASE_URL
-        int cut = metadataUrl.indexOf('@');
-        tableNameBase = cut < 0 ? DEFAULT_TABLE_NAME : metadataUrl.substring(0, cut);
-        hbaseUrl = cut < 0 ? metadataUrl : metadataUrl.substring(cut + 1);
-
-        createHTableIfNeeded(getAllInOneTableName());
-
-        //        tableNameMap = new LinkedHashMap<String, String>();
-        //        for (Entry<String, String> entry : TABLE_SUFFIX_MAP.entrySet()) {
-        //            String pathPrefix = entry.getKey();
-        //            String tableName = tableNameBase + entry.getValue();
-        //            tableNameMap.put(pathPrefix, tableName);
-        //            createHTableIfNeeded(tableName);
-        //        }
-
-    }
-
-    private void createHTableIfNeeded(String tableName) throws IOException {
-        HBaseConnection.createHTableIfNeeded(getConnection(), tableName, FAMILY);
-    }
-
-    private String getAllInOneTableName() {
-        return tableNameBase;
-    }
-
-    @Override
-    protected ArrayList<String> listResourcesImpl(String resPath) throws IOException {
-        assert resPath.startsWith("/");
-        String lookForPrefix = resPath.endsWith("/") ? resPath : resPath + "/";
-        byte[] startRow = Bytes.toBytes(lookForPrefix);
-        byte[] endRow = Bytes.toBytes(lookForPrefix);
-        endRow[endRow.length - 1]++;
-
-        ArrayList<String> result = new ArrayList<String>();
-
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
-        Scan scan = new Scan(startRow, endRow);
-        scan.setFilter(new KeyOnlyFilter());
-        try {
-            ResultScanner scanner = table.getScanner(scan);
-            for (Result r : scanner) {
-                String path = Bytes.toString(r.getRow());
-                assert path.startsWith(lookForPrefix);
-                int cut = path.indexOf('/', lookForPrefix.length());
-                String child = cut < 0 ? path : path.substring(0, cut);
-                if (result.contains(child) == false)
-                    result.add(child);
-            }
-        } finally {
-            IOUtils.closeQuietly(table);
-        }
-        // return null to indicate not a folder
-        return result.isEmpty() ? null : result;
-    }
-
-    @Override
-    protected boolean existsImpl(String resPath) throws IOException {
-        Result r = getByScan(resPath, null, null);
-        return r != null;
-    }
-
-    @Override
-    protected List<RawResource> getAllResources(String rangeStart, String rangeEnd) throws IOException {
-        byte[] startRow = Bytes.toBytes(rangeStart);
-        byte[] endRow = plusZero(Bytes.toBytes(rangeEnd));
-
-        Scan scan = new Scan(startRow, endRow);
-        scan.addColumn(B_FAMILY, B_COLUMN_TS);
-        scan.addColumn(B_FAMILY, B_COLUMN);
-
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
-        List<RawResource> result = Lists.newArrayList();
-        try {
-            ResultScanner scanner = table.getScanner(scan);
-            for (Result r : scanner) {
-                result.add(new RawResource(getInputStream(Bytes.toString(r.getRow()), r), getTimestamp(r)));
-            }
-        } catch (IOException e) {
-            for (RawResource rawResource : result) {
-                IOUtils.closeQuietly(rawResource.resource);
-            }
-            throw e;
-        } finally {
-            IOUtils.closeQuietly(table);
-        }
-        return result;
-    }
-
-    private InputStream getInputStream(String resPath, Result r) throws IOException {
-        if (r == null) {
-            return null;
-        }
-        byte[] value = r.getValue(B_FAMILY, B_COLUMN);
-        if (value.length == 0) {
-            Path redirectPath = bigCellHDFSPath(resPath);
-            Configuration hconf = HadoopUtil.getCurrentConfiguration();
-            FileSystem fileSystem = FileSystem.get(hconf);
-
-            return fileSystem.open(redirectPath);
-        } else {
-            return new ByteArrayInputStream(value);
-        }
-    }
-
-    private long getTimestamp(Result r) {
-        if (r == null) {
-            return 0;
-        } else {
-            return Bytes.toLong(r.getValue(B_FAMILY, B_COLUMN_TS));
-        }
-    }
-
-    @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);
-    }
-
-    @Override
-    protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException {
-        ByteArrayOutputStream bout = new ByteArrayOutputStream();
-        IOUtils.copy(content, bout);
-        bout.close();
-
-        HTableInterface table = getConnection().getTable(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);
-        }
-    }
-
-    @Override
-    protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
-        try {
-            byte[] row = Bytes.toBytes(resPath);
-            byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
-            Put put = buildPut(resPath, newTS, row, content, table);
-
-            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 " + oldTS + ", but it is " + real);
-            }
-
-            table.flushCommits();
-
-            return newTS;
-        } finally {
-            IOUtils.closeQuietly(table);
-        }
-    }
-
-    @Override
-    protected void deleteResourceImpl(String resPath) throws IOException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
-        try {
-            Delete del = new Delete(Bytes.toBytes(resPath));
-            table.delete(del);
-            table.flushCommits();
-        } finally {
-            IOUtils.closeQuietly(table);
-        }
-    }
-
-    @Override
-    protected String getReadableResourcePathImpl(String resPath) {
-        return getAllInOneTableName() + "(key='" + resPath + "')@" + kylinConfig.getMetadataUrl();
-    }
-
-    private Result getByScan(String path, byte[] family, byte[] column) throws IOException {
-        byte[] startRow = Bytes.toBytes(path);
-        byte[] endRow = plusZero(startRow);
-
-        Scan scan = new Scan(startRow, endRow);
-        if (family == null || column == null) {
-            scan.setFilter(new KeyOnlyFilter());
-        } else {
-            scan.addColumn(family, column);
-        }
-
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
-        try {
-            ResultScanner scanner = table.getScanner(scan);
-            Result result = null;
-            for (Result r : scanner) {
-                result = r;
-            }
-            return result == null || result.isEmpty() ? null : result;
-        } finally {
-            IOUtils.closeQuietly(table);
-        }
-    }
-
-    private byte[] plusZero(byte[] startRow) {
-        byte[] endRow = Arrays.copyOf(startRow, startRow.length + 1);
-        endRow[endRow.length - 1] = 0;
-        return endRow;
-    }
-
-    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
-        Path redirectPath = bigCellHDFSPath(resPath);
-        Configuration hconf = HadoopUtil.getCurrentConfiguration();
-        FileSystem fileSystem = FileSystem.get(hconf);
-
-        if (fileSystem.exists(redirectPath)) {
-            fileSystem.delete(redirectPath, true);
-        }
-
-        FSDataOutputStream out = fileSystem.create(redirectPath);
-
-        try {
-            out.write(largeColumn);
-        } finally {
-            IOUtils.closeQuietly(out);
-        }
-
-        return redirectPath;
-    }
-
-    public Path bigCellHDFSPath(String resPath) {
-        String hdfsWorkingDirectory = this.kylinConfig.getHdfsWorkingDirectory();
-        Path redirectPath = new Path(hdfsWorkingDirectory, "resources" + resPath);
-        return redirectPath;
-    }
-
-    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
-        int kvSizeLimit = this.kylinConfig.getHBaseKeyValueSize();
-        if (content.length > kvSizeLimit) {
-            writeLargeCellToHdfs(resPath, content, table);
-            content = BytesUtil.EMPTY_BYTE_ARRAY;
-        }
-
-        Put put = new Put(row);
-        put.add(B_FAMILY, B_COLUMN, content);
-        put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
-
-        return put;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
index 926b729..ed3a6a5 100644
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
@@ -30,6 +30,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.ClassUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,8 +63,12 @@ abstract public class ResourceStore {
     public static final ArrayList<Class<? extends ResourceStore>> knownImpl = new ArrayList<Class<? extends ResourceStore>>();
 
     static {
-        knownImpl.add(HBaseResourceStore.class);
-        knownImpl.add(FileResourceStore.class);
+        try {
+            knownImpl.add(ClassUtil.forName("org.apache.kylin.storage.hbase.HBaseResourceStore", ResourceStore.class));
+            knownImpl.add(FileResourceStore.class);
+        } catch (ClassNotFoundException e) {
+            throw new RuntimeException(e);
+        }
     }
 
     public static ResourceStore getStore(KylinConfig kylinConfig) {
@@ -99,9 +104,9 @@ abstract public class ResourceStore {
 
     // ============================================================================
 
-    KylinConfig kylinConfig;
+    final protected KylinConfig kylinConfig;
 
-    ResourceStore(KylinConfig kylinConfig) {
+    public ResourceStore(KylinConfig kylinConfig) {
         this.kylinConfig = kylinConfig;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
index 743acfe..5e3a39f 100644
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
+++ b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
@@ -19,6 +19,7 @@
 package org.apache.kylin.common.restclient;
 
 import java.io.IOException;
+import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -31,8 +32,7 @@ import org.apache.commons.httpclient.auth.AuthScope;
 import org.apache.commons.httpclient.methods.GetMethod;
 import org.apache.commons.httpclient.methods.PutMethod;
 import org.apache.kylin.common.util.Bytes;
-import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONObject;
+import org.apache.kylin.common.util.JsonUtil;
 
 /**
  * @author yangli9
@@ -111,16 +111,14 @@ public class RestClient {
         try {
             int code = client.executeMethod(request);
             String msg = Bytes.toString(request.getResponseBody());
-            JSONObject obj = new JSONObject(msg);
-            msg = obj.getString("config");
+            Map<String, String> map = JsonUtil.readValueAsMap(msg);
+            msg = map.get("config");
 
             if (code != 200)
                 throw new IOException("Invalid response " + code + " with cache wipe url " + url + "\n" + msg);
 
             return msg;
-
-        } catch (JSONException e) {
-            throw new IOException("Error when parsing json response from REST");
+            
         } finally {
             request.releaseConnection();
         }



[2/6] incubator-kylin git commit: KYLIN-875 Refactor core-common, remove dependency on hadoop/hbase

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/tools/IICLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/IICLI.java b/job/src/main/java/org/apache/kylin/job/tools/IICLI.java
new file mode 100644
index 0000000..4e38309
--- /dev/null
+++ b/job/src/main/java/org/apache/kylin/job/tools/IICLI.java
@@ -0,0 +1,108 @@
+/*
+ * 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 org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IIManager;
+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.IIKeyValueCodec;
+import org.apache.kylin.invertedindex.model.IIRow;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+/**
+ * @author yangli9
+ */
+public class IICLI {
+
+	public static void main(String[] args) throws IOException {
+		Configuration hconf = HadoopUtil.getCurrentConfiguration();
+		IIManager mgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+		String iiName = args[0];
+		IIInstance ii = mgr.getII(iiName);
+
+		String path = args[1];
+		System.out.println("Reading from " + path + " ...");
+
+		TableRecordInfo info = new TableRecordInfo(ii.getFirstSegment());
+		IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest());
+		int count = 0;
+		for (Slice slice : codec.decodeKeyValue(readSequenceKVs(hconf, path))) {
+			for (RawTableRecord rec : slice) {
+				System.out.printf(new TableRecord(rec, info).toString());
+				count++;
+			}
+		}
+		System.out.println("Total " + count + " records");
+	}
+
+	public static Iterable<IIRow> readSequenceKVs(
+			Configuration hconf, String path) throws IOException {
+		final Reader reader = new Reader(hconf,
+				SequenceFile.Reader.file(new Path(path)));
+		return new Iterable<IIRow>() {
+			@Override
+			public Iterator<IIRow> iterator() {
+				return new Iterator<IIRow>() {
+					ImmutableBytesWritable k = new ImmutableBytesWritable();
+					ImmutableBytesWritable v = new ImmutableBytesWritable();
+                    IIRow pair = new IIRow(k, v, null);
+
+					@Override
+					public boolean hasNext() {
+						boolean hasNext = false;
+						try {
+							hasNext = reader.next(k, v);
+						} catch (IOException e) {
+							throw new RuntimeException(e);
+						} finally {
+							if (hasNext == false) {
+								IOUtils.closeQuietly(reader);
+							}
+						}
+						return hasNext;
+					}
+
+					@Override
+					public IIRow next() {
+						return pair;
+					}
+
+					@Override
+					public void remove() {
+						throw new UnsupportedOperationException();
+					}
+				};
+			}
+		};
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/job/tools/TarGZUtil.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/TarGZUtil.java b/job/src/main/java/org/apache/kylin/job/tools/TarGZUtil.java
new file mode 100644
index 0000000..2ce5889
--- /dev/null
+++ b/job/src/main/java/org/apache/kylin/job/tools/TarGZUtil.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job.tools;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+
+public class TarGZUtil {
+
+    public static void uncompressTarGZ(File tarFile, File dest) throws IOException {
+        dest.mkdir();
+        TarArchiveInputStream tarIn = null;
+
+        tarIn = new TarArchiveInputStream(new GzipCompressorInputStream(new BufferedInputStream(new FileInputStream(tarFile))));
+
+        TarArchiveEntry tarEntry = tarIn.getNextTarEntry();
+        // tarIn is a TarArchiveInputStream
+        while (tarEntry != null) {// create a file with the same name as the tarEntry
+            File destPath = new File(dest, tarEntry.getName());
+            System.out.println("working: " + destPath.getCanonicalPath());
+            if (tarEntry.isDirectory()) {
+                destPath.mkdirs();
+            } else {
+                destPath.createNewFile();
+                //byte [] btoRead = new byte[(int)tarEntry.getSize()];
+                byte[] btoRead = new byte[1024];
+                //FileInputStream fin 
+                //  = new FileInputStream(destPath.getCanonicalPath());
+                BufferedOutputStream bout = new BufferedOutputStream(new FileOutputStream(destPath));
+                int len = 0;
+
+                while ((len = tarIn.read(btoRead)) != -1) {
+                    bout.write(btoRead, 0, len);
+                }
+
+                bout.close();
+                btoRead = null;
+
+            }
+            tarEntry = tarIn.getNextTarEntry();
+        }
+        tarIn.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/job/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index f005bc9..c40924e 100644
--- a/job/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/job/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.engine.mr.IMRInput;
 import org.apache.kylin.engine.mr.JobBuilderSupport;
 import org.apache.kylin.job.JoinedFlatTable;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java b/job/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
index 76b9bab..31878a8 100644
--- a/job/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
+++ b/job/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
@@ -21,11 +21,11 @@ package org.apache.kylin.source.hive;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.common.util.HiveClient;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/source/hive/HiveTable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/source/hive/HiveTable.java b/job/src/main/java/org/apache/kylin/source/hive/HiveTable.java
index 02e7c45..c09b6fc 100644
--- a/job/src/main/java/org/apache/kylin/source/hive/HiveTable.java
+++ b/job/src/main/java/org/apache/kylin/source/hive/HiveTable.java
@@ -21,9 +21,8 @@ package org.apache.kylin.source.hive;
 import java.io.IOException;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HiveClient;
 import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.dict.lookup.FileTable;
+import org.apache.kylin.engine.mr.DFSFileTable;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.source.ReadableTable;
 import org.slf4j.Logger;
@@ -54,7 +53,7 @@ public class HiveTable implements ReadableTable {
     public TableSignature getSignature() throws IOException {
         try {
             String path = computeHDFSLocation();
-            Pair<Long, Long> sizeAndLastModified = FileTable.getSizeAndLastModified(path);
+            Pair<Long, Long> sizeAndLastModified = DFSFileTable.getSizeAndLastModified(path);
             long size = sizeAndLastModified.getFirst();
             long lastModified = sizeAndLastModified.getSecond();
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java b/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
index cd156b0..b5f6d65 100644
--- a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
+++ b/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
@@ -53,7 +53,6 @@ import org.apache.hadoop.mapreduce.Mapper.Context;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
-import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
@@ -62,6 +61,7 @@ import org.apache.kylin.cube.kv.RowValueDecoder;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
 import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
 import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.engine.mr.IMROutput2;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.metadata.measure.MeasureCodec;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java b/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
index 5423ddd..1b626d2 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
@@ -34,10 +34,8 @@ import java.util.concurrent.Future;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.lock.ZookeeperJobLock;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -50,6 +48,8 @@ import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.impl.threadpool.DefaultScheduler;
 import org.apache.kylin.job.manager.ExecutableManager;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
+import org.apache.kylin.storage.hbase.ZookeeperJobLock;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java b/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
index b1aad82..6a29bb8 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
@@ -38,9 +38,9 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.job.streaming.BootstrapConfig;
 import org.apache.kylin.job.streaming.StreamingBootstrap;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.apache.kylin.streaming.StreamingConfig;
 import org.apache.kylin.streaming.StreamingManager;
 import org.junit.After;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java b/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
index 2382540..41457ed 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
@@ -19,14 +19,13 @@
 package org.apache.kylin.job;
 
 import com.google.common.collect.Lists;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.lock.ZookeeperJobLock;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
-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;
@@ -39,6 +38,8 @@ import org.apache.kylin.job.invertedindex.IIJob;
 import org.apache.kylin.job.invertedindex.IIJobBuilder;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
+import org.apache.kylin.storage.hbase.ZookeeperJobLock;
 import org.junit.*;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java b/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
index 1f91b25..071ae3a 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
@@ -55,7 +55,6 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.DateFormat;
-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;
@@ -69,6 +68,7 @@ import org.apache.kylin.job.hadoop.invertedindex.IICreateHTableJob;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.source.hive.HiveTableReader;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.apache.kylin.streaming.StreamBuilder;
 import org.apache.kylin.streaming.StreamMessage;
 import org.apache.kylin.streaming.invertedindex.IIStreamConsumer;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/DeployLocalMetaToRemoteTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/DeployLocalMetaToRemoteTest.java b/job/src/test/java/org/apache/kylin/job/DeployLocalMetaToRemoteTest.java
index 276cf67..ff23040 100644
--- a/job/src/test/java/org/apache/kylin/job/DeployLocalMetaToRemoteTest.java
+++ b/job/src/test/java/org/apache/kylin/job/DeployLocalMetaToRemoteTest.java
@@ -23,7 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.junit.*;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 98927bb..7ebfac0 100644
--- a/job/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/job/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -19,6 +19,7 @@
 package org.apache.kylin.job;
 
 import com.google.common.collect.Lists;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
@@ -27,7 +28,6 @@ import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.ResourceTool;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.CliCommandExecutor;
-import org.apache.kylin.common.util.HiveClient;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.CubeInstance;
@@ -40,6 +40,7 @@ import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.source.hive.HiveClient;
 import org.apache.kylin.streaming.StreamMessage;
 import org.apache.kylin.streaming.StreamingConfig;
 import org.apache.kylin.streaming.TimedJsonStreamParser;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 00ecb23..e3120f3 100644
--- a/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
+++ b/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
@@ -25,14 +25,13 @@ 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.kylin.common.util.HBaseMiniclusterHelper;
 import org.codehaus.plexus.util.FileUtils;
-
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.CliCommandExecutor;
 import org.apache.kylin.common.util.SSHClient;
+import org.apache.kylin.storage.hbase.HBaseConnection;
+import org.apache.kylin.storage.hbase.HBaseMiniclusterHelper;
 
 public class ExportHBaseData {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/ITKafkaBasedIIStreamBuilderTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/ITKafkaBasedIIStreamBuilderTest.java b/job/src/test/java/org/apache/kylin/job/ITKafkaBasedIIStreamBuilderTest.java
index 71c9644..17d907c 100644
--- a/job/src/test/java/org/apache/kylin/job/ITKafkaBasedIIStreamBuilderTest.java
+++ b/job/src/test/java/org/apache/kylin/job/ITKafkaBasedIIStreamBuilderTest.java
@@ -37,8 +37,8 @@ package org.apache.kylin.job;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.job.streaming.StreamingBootstrap;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/ImportHBaseData.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/ImportHBaseData.java b/job/src/test/java/org/apache/kylin/job/ImportHBaseData.java
index 0de026a..8aea870 100644
--- a/job/src/test/java/org/apache/kylin/job/ImportHBaseData.java
+++ b/job/src/test/java/org/apache/kylin/job/ImportHBaseData.java
@@ -27,15 +27,14 @@ 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.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.HBaseConnection;
-import org.apache.kylin.common.persistence.HBaseResourceStore;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.CliCommandExecutor;
-import org.apache.kylin.common.util.HBaseMiniclusterHelper;
 import org.apache.kylin.common.util.SSHClient;
-import org.apache.kylin.common.util.TarGZUtil;
+import org.apache.kylin.job.tools.TarGZUtil;
+import org.apache.kylin.storage.hbase.HBaseConnection;
+import org.apache.kylin.storage.hbase.HBaseMiniclusterHelper;
+import org.apache.kylin.storage.hbase.HBaseResourceStore;
 
 public class ImportHBaseData {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/FactDistinctColumnsReducerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/FactDistinctColumnsReducerTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/FactDistinctColumnsReducerTest.java
index 2e2a945..b85afcc 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/FactDistinctColumnsReducerTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/FactDistinctColumnsReducerTest.java
@@ -1,11 +1,12 @@
 package org.apache.kylin.job.hadoop.cubev2;
 
 import com.google.common.collect.Maps;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.job.hadoop.cube.FactDistinctColumnsReducer;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/hadoop/hdfs/ITHdfsOpsTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/hdfs/ITHdfsOpsTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/hdfs/ITHdfsOpsTest.java
index 8710681..69519e0 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/hdfs/ITHdfsOpsTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/hdfs/ITHdfsOpsTest.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java b/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
index 29b8304..20208f2 100644
--- a/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
+++ b/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
@@ -19,14 +19,14 @@
 package org.apache.kylin.job.impl.threadpool;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.lock.ZookeeperJobLock;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.job.DeployUtil;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.manager.ExecutableManager;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
+import org.apache.kylin.storage.hbase.ZookeeperJobLock;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilderTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilderTest.java b/job/src/test/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilderTest.java
index 9600ef7..3caa1b0 100644
--- a/job/src/test/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilderTest.java
+++ b/job/src/test/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilderTest.java
@@ -39,7 +39,7 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.DictionaryGenerator;
-import org.apache.kylin.dict.lookup.FileTableReader;
+import org.apache.kylin.engine.mr.DFSFileTableReader;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.storage.gridtable.GTRecord;
 import org.junit.AfterClass;
@@ -133,7 +133,7 @@ public class InMemCubeBuilderTest extends LocalFileMetadataTestCase {
             distinctSets[i] = new TreeSet<String>();
 
         // get distinct values on each column
-        FileTableReader reader = new FileTableReader(flatTable, nColumns);
+        DFSFileTableReader reader = new DFSFileTableReader(flatTable, nColumns);
         while (count > 0 && reader.next()) {
             String[] row = reader.getRow();
             for (int i = 0; i < nColumns; i++)
@@ -183,7 +183,7 @@ public class InMemCubeBuilderTest extends LocalFileMetadataTestCase {
 
     private static List<byte[]> readValueList(String flatTable, int nColumns, int c) throws IOException {
         List<byte[]> result = Lists.newArrayList();
-        FileTableReader reader = new FileTableReader(flatTable, nColumns);
+        DFSFileTableReader reader = new DFSFileTableReader(flatTable, nColumns);
         while (reader.next()) {
             String[] row = reader.getRow();
             if (row[c] != null) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/job/streaming/CubeStreamConsumerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/streaming/CubeStreamConsumerTest.java b/job/src/test/java/org/apache/kylin/job/streaming/CubeStreamConsumerTest.java
index b246a73..912d218 100644
--- a/job/src/test/java/org/apache/kylin/job/streaming/CubeStreamConsumerTest.java
+++ b/job/src/test/java/org/apache/kylin/job/streaming/CubeStreamConsumerTest.java
@@ -1,16 +1,17 @@
 package org.apache.kylin.job.streaming;
 
 import com.google.common.collect.Lists;
+
 import org.apache.hadoop.util.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.job.DeployUtil;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.apache.kylin.streaming.StreamBuilder;
 import org.apache.kylin.streaming.StreamMessage;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/source/hive/ITHiveSourceTableLoaderTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/source/hive/ITHiveSourceTableLoaderTest.java b/job/src/test/java/org/apache/kylin/source/hive/ITHiveSourceTableLoaderTest.java
index de6df2c..019dc56 100644
--- a/job/src/test/java/org/apache/kylin/source/hive/ITHiveSourceTableLoaderTest.java
+++ b/job/src/test/java/org/apache/kylin/source/hive/ITHiveSourceTableLoaderTest.java
@@ -24,8 +24,8 @@ import java.io.IOException;
 import java.util.Set;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.source.hive.HiveSourceTableLoader;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/source/hive/ITHiveTableReaderTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/source/hive/ITHiveTableReaderTest.java b/job/src/test/java/org/apache/kylin/source/hive/ITHiveTableReaderTest.java
index 624f158..ac2275a 100644
--- a/job/src/test/java/org/apache/kylin/source/hive/ITHiveTableReaderTest.java
+++ b/job/src/test/java/org/apache/kylin/source/hive/ITHiveTableReaderTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.kylin.source.hive;
 
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.source.hive.HiveTableReader;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java b/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java
index 975d69f..2b44f9d 100644
--- a/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java
+++ b/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java
@@ -20,7 +20,6 @@ package org.apache.kylin.source.hive;
 
 import static org.junit.Assert.*;
 
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.dict.lookup.SnapshotManager;
 import org.apache.kylin.dict.lookup.SnapshotTable;
 import org.apache.kylin.metadata.MetadataManager;
@@ -28,6 +27,7 @@ import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.source.ReadableTable;
 import org.apache.kylin.source.ReadableTable.TableReader;
 import org.apache.kylin.source.TableSourceFactory;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/query/pom.xml
----------------------------------------------------------------------
diff --git a/query/pom.xml b/query/pom.xml
index 604f03c..054124e 100644
--- a/query/pom.xml
+++ b/query/pom.xml
@@ -38,13 +38,6 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-common</artifactId>
-            <type>test-jar</type>
-            <scope>test</scope>
-            <version>${project.parent.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
             <artifactId>atopcalcite</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
@@ -77,6 +70,20 @@
         <!-- Env & Test -->
 
         <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-storage-hbase</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
             <groupId>xerces</groupId>
             <artifactId>xercesImpl</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java b/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java
index a595dc9..48578af 100644
--- a/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java
+++ b/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java
@@ -20,11 +20,11 @@ package org.apache.kylin.query.test;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.query.enumerator.OLAPQuery;
 import org.apache.kylin.query.relnode.OLAPContext;
 import org.apache.kylin.query.schema.OLAPSchemaFactory;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.apache.kylin.storage.hbase.coprocessor.observer.ObserverEnabler;
 import org.dbunit.database.DatabaseConnection;
 import org.dbunit.database.IDatabaseConnection;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index 0b74879..a757522 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -38,14 +38,6 @@
 
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-common</artifactId>
-            <type>test-jar</type>
-            <scope>test</scope>
-            <version>${project.parent.version}</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-query</artifactId>
             <version>${project.parent.version}</version>
             <exclusions>
@@ -70,6 +62,22 @@
                 </exclusion>
             </exclusions>
         </dependency>
+
+        <!-- Test & Env -->
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-storage-hbase</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-job</artifactId>
@@ -77,6 +85,7 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+        
         <!-- depends on kylin-jdbc just for running jdbc test cases in server module -->
         <dependency>
             <groupId>org.apache.kylin</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/server/src/main/java/org/apache/kylin/rest/controller/JobController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/controller/JobController.java b/server/src/main/java/org/apache/kylin/rest/controller/JobController.java
index 6427a98..5e40e12 100644
--- a/server/src/main/java/org/apache/kylin/rest/controller/JobController.java
+++ b/server/src/main/java/org/apache/kylin/rest/controller/JobController.java
@@ -19,11 +19,11 @@
 package org.apache.kylin.rest.controller;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.lock.JobLock;
 import org.apache.kylin.job.JobInstance;
 import org.apache.kylin.job.constant.JobStatusEnum;
 import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.impl.threadpool.DefaultScheduler;
+import org.apache.kylin.job.lock.JobLock;
 import org.apache.kylin.rest.constant.Constant;
 import org.apache.kylin.rest.exception.InternalErrorException;
 import org.apache.kylin.rest.request.JobListRequest;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/server/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java b/server/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
index 553e157..08b027b 100644
--- a/server/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
+++ b/server/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
@@ -2,9 +2,9 @@ package org.apache.kylin.rest.security;
 
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.rest.service.AclService;
 import org.apache.kylin.rest.service.UserService;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.h2.util.StringUtils;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 f3f81a4..1e55b2c 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,11 +29,10 @@ import java.util.Set;
 import java.util.WeakHashMap;
 
 import com.google.common.collect.Lists;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HBaseRegionSizeCalculator;
-import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -42,6 +41,7 @@ import org.apache.kylin.cube.cuboid.CuboidCLI;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.engine.BuildEngineFactory;
 import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.job.common.HadoopShellExecutable;
 import org.apache.kylin.job.exception.JobException;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
@@ -64,6 +64,8 @@ import org.apache.kylin.rest.response.HBaseResponse;
 import org.apache.kylin.rest.response.MetricsResponse;
 import org.apache.kylin.rest.security.AclPermission;
 import org.apache.kylin.source.hive.HiveSourceTableLoader;
+import org.apache.kylin.storage.hbase.HBaseConnection;
+import org.apache.kylin.storage.hbase.HBaseRegionSizeCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -444,7 +446,7 @@ public class CubeService extends BasicService {
 
         // Get HBase storage conf.
         String hbaseUrl = KylinConfig.getInstanceFromEnv().getStorageUrl();
-        Configuration hconf = HadoopUtil.newHBaseConfiguration(hbaseUrl);
+        Configuration hconf = HBaseConnection.newHBaseConfiguration(hbaseUrl);
 
         HTable table = null;
         HBaseResponse hr = null;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/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 2a78e21..3e62483 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
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -55,6 +54,7 @@ import org.apache.kylin.rest.request.SQLRequest;
 import org.apache.kylin.rest.response.SQLResponse;
 import org.apache.kylin.rest.util.QueryUtil;
 import org.apache.kylin.rest.util.Serializer;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.h2.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/server/src/test/java/org/apache/kylin/jdbc/ITJDBCDriverTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/jdbc/ITJDBCDriverTest.java b/server/src/test/java/org/apache/kylin/jdbc/ITJDBCDriverTest.java
index 08f5217..67cc573 100644
--- a/server/src/test/java/org/apache/kylin/jdbc/ITJDBCDriverTest.java
+++ b/server/src/test/java/org/apache/kylin/jdbc/ITJDBCDriverTest.java
@@ -22,7 +22,7 @@ package org.apache.kylin.jdbc;
 import com.google.common.collect.Lists;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
+import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.webapp.WebAppContext;
 import org.junit.*;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/source-hive/pom.xml
----------------------------------------------------------------------
diff --git a/source-hive/pom.xml b/source-hive/pom.xml
index fcfd953..17a918e 100644
--- a/source-hive/pom.xml
+++ b/source-hive/pom.xml
@@ -43,6 +43,17 @@
 
         <!-- Env & Test -->
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hive.hcatalog</groupId>
+            <artifactId>hive-hcatalog-core</artifactId>
+            <version>${hive-hcatalog.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/source-hive/src/main/java/org/apache/kylin/source/hive/HiveClient.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveClient.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveClient.java
new file mode 100644
index 0000000..c5a9bca
--- /dev/null
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveClient.java
@@ -0,0 +1,162 @@
+/*
+ * 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.source.hive;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hive.cli.CliSessionState;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ * Hive meta API client for Kylin
+ * @author shaoshi
+ *
+ */
+public class HiveClient {
+
+    protected HiveConf hiveConf = null;
+    protected Driver driver = null;
+    protected HiveMetaStoreClient metaStoreClient = null;
+
+    public HiveClient() {
+        hiveConf = new HiveConf(HiveClient.class);
+    }
+
+    public HiveClient(Map<String, String> configMap) {
+        this();
+        appendConfiguration(configMap);
+    }
+
+    public HiveConf getHiveConf() {
+        return hiveConf;
+    }
+
+    /**
+     * Get the hive ql driver to execute ddl or dml
+     * @return
+     */
+    private Driver getDriver() {
+        if (driver == null) {
+            driver = new Driver(hiveConf);
+            SessionState.start(new CliSessionState(hiveConf));
+        }
+
+        return driver;
+    }
+
+    /**
+     * Append or overwrite the default hive client configuration; You need call this before invoke #executeHQL;
+     * @param configMap
+     */
+    public void appendConfiguration(Map<String, String> configMap) {
+        if (configMap != null && configMap.size() > 0) {
+            for (Entry<String, String> e : configMap.entrySet()) {
+                hiveConf.set(e.getKey(), e.getValue());
+            }
+        }
+    }
+
+    /**
+     * 
+     * @param hql
+     * @throws CommandNeedRetryException
+     * @throws IOException
+     */
+    public void executeHQL(String hql) throws CommandNeedRetryException, IOException {
+        CommandProcessorResponse response = getDriver().run(hql);
+        int retCode = response.getResponseCode();
+        if (retCode != 0) {
+            String err = response.getErrorMessage();
+            throw new IOException("Failed to execute hql [" + hql + "], error message is: " + err);
+        }
+    }
+
+    public void executeHQL(String[] hqls) throws CommandNeedRetryException, IOException {
+        for (String sql : hqls)
+            executeHQL(sql);
+    }
+
+    private HiveMetaStoreClient getMetaStoreClient() throws Exception {
+        if (metaStoreClient == null) {
+            metaStoreClient = new HiveMetaStoreClient(hiveConf);
+        }
+        return metaStoreClient;
+    }
+
+    public Table getHiveTable(String database, String tableName) throws Exception {
+        return getMetaStoreClient().getTable(database, tableName);
+    }
+
+    public List<FieldSchema> getHiveTableFields(String database, String tableName) throws Exception {
+        return getMetaStoreClient().getFields(database, tableName);
+    }
+
+    public String getHiveTableLocation(String database, String tableName) throws Exception {
+        Table t = getHiveTable(database, tableName);
+        return t.getSd().getLocation();
+    }
+
+    public long getFileSizeForTable(Table table) {
+        return getBasicStatForTable(new org.apache.hadoop.hive.ql.metadata.Table(table), StatsSetupConst.TOTAL_SIZE);
+    }
+
+    public long getFileNumberForTable(Table table) {
+        return getBasicStatForTable(new org.apache.hadoop.hive.ql.metadata.Table(table), StatsSetupConst.NUM_FILES);
+    }
+
+    /**
+     * COPIED FROM org.apache.hadoop.hive.ql.stats.StatsUtil for backward compatibility
+     * 
+     * Get basic stats of table
+     * @param table
+     *          - table
+     * @param statType
+     *          - type of stats
+     * @return value of stats
+     */
+    public static long getBasicStatForTable(org.apache.hadoop.hive.ql.metadata.Table table, String statType) {
+        Map<String, String> params = table.getParameters();
+        long result = 0;
+
+        if (params != null) {
+            try {
+                result = Long.parseLong(params.get(statType));
+            } catch (NumberFormatException e) {
+                result = 0;
+            }
+        }
+        return result;
+    }
+
+    public boolean isNativeTable(String database, String tableName)  throws Exception{
+        return !MetaStoreUtils.isNonNativeTable(getMetaStoreClient().getTable(database, tableName));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/storage-hbase/pom.xml b/storage-hbase/pom.xml
index 951602e..013b009 100644
--- a/storage-hbase/pom.xml
+++ b/storage-hbase/pom.xml
@@ -43,6 +43,44 @@
 
         <!-- Env & Test -->
         <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-server</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-testing-util</artifactId>
+            <version>${hbase-hadoop2.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
new file mode 100644
index 0000000..d1bb216
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
@@ -0,0 +1,234 @@
+/*
+ * 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 java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+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.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.kylin.common.persistence.StorageException;
+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();
+                    }
+                }
+            }
+        });
+    }
+
+    /**
+     * e.g.
+     * 0. hbase (recommended way)
+     * 1. hbase:zk-1.hortonworks.com,zk-2.hortonworks.com,zk-3.hortonworks.com:2181:/hbase-unsecure
+     * 2. hbase:zk-1.hortonworks.com,zk-2.hortonworks.com,zk-3.hortonworks.com:2181
+     * 3. hbase:zk-1.hortonworks.com:2181:/hbase-unsecure
+     * 4. hbase:zk-1.hortonworks.com:2181
+     */
+    public static Configuration newHBaseConfiguration(String url) {
+        Configuration conf = HBaseConfiguration.create();
+        // reduce rpc retry
+        conf.set(HConstants.HBASE_CLIENT_PAUSE, "3000");
+        conf.set(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "5");
+        conf.set(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, "60000");
+        // conf.set(ScannerCallable.LOG_SCANNER_ACTIVITY, "true");
+        if (StringUtils.isEmpty(url)) {
+            return conf;
+        }
+
+        // chop off "hbase"
+        if (url.startsWith("hbase") == false) {
+            throw new IllegalArgumentException("hbase url must start with 'hbase' -- " + url);
+        }
+
+        url = StringUtils.substringAfter(url, "hbase");
+        if (StringUtils.isEmpty(url)) {
+            return conf;
+        }
+
+        // case of "hbase:domain.com:2181:/hbase-unsecure"
+        Pattern urlPattern = Pattern.compile("[:]((?:[\\w\\-.]+)(?:\\,[\\w\\-.]+)*)[:](\\d+)(?:[:](.+))");
+        Matcher m = urlPattern.matcher(url);
+        if (m.matches() == false)
+            throw new IllegalArgumentException("HBase URL '" + url + "' is invalid, expected url is like '" + "hbase:domain.com:2181:/hbase-unsecure" + "'");
+
+        logger.debug("Creating hbase conf by parsing -- " + url);
+
+        String quorums = m.group(1);
+        String quorum = null;
+        try {
+            String[] tokens = quorums.split(",");
+            for (String s : tokens) {
+                quorum = s;
+                InetAddress.getByName(quorum);
+            }
+        } catch (UnknownHostException e) {
+            throw new IllegalArgumentException("Zookeeper quorum is invalid: " + quorum + "; urlString=" + url, e);
+        }
+        conf.set(HConstants.ZOOKEEPER_QUORUM, quorums);
+
+        String port = m.group(2);
+        conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, port);
+
+        String znodePath = m.group(3);
+        conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, znodePath);
+
+        return conf;
+    }
+
+    // returned HConnection can be shared by multiple threads and does not require close()
+    @SuppressWarnings("resource")
+    public static HConnection get(String url) {
+        // find configuration
+        Configuration conf = ConfigCache.get(url);
+        if (conf == null) {
+            conf = newHBaseConfiguration(url);
+            ConfigCache.put(url, conf);
+        }
+
+        HConnection connection = ConnPool.get(url);
+        try {
+            while (true) {
+                // I don't use DCL since recreate a connection is not a big issue.
+                if (connection == null || connection.isClosed()) {
+                    logger.info("connection is null or closed, creating a new one");
+                    connection = HConnectionManager.createConnection(conf);
+                    ConnPool.put(url, connection);
+                }
+
+                if (connection == null || connection.isClosed()) {
+                    Thread.sleep(10000);// wait a while and retry
+                } else {
+                    break;
+                }
+            }
+
+        } catch (Throwable t) {
+            logger.error("Error when open connection " + url, t);
+            throw new StorageException("Error when open connection " + url, t);
+        }
+
+        return connection;
+    }
+
+    public static boolean tableExists(HConnection conn, String tableName) throws IOException {
+        HBaseAdmin hbase = new HBaseAdmin(conn);
+        try {
+            return hbase.tableExists(TableName.valueOf(tableName));
+        } finally {
+            hbase.close();
+        }
+    }
+
+    public static boolean tableExists(String hbaseUrl, String tableName) throws IOException {
+        return tableExists(HBaseConnection.get(hbaseUrl), tableName);
+    }
+
+    public static void createHTableIfNeeded(String hbaseUrl, String tableName, String... families) throws IOException {
+        createHTableIfNeeded(HBaseConnection.get(hbaseUrl), tableName, families);
+    }
+
+    public static void deleteTable(String hbaseUrl, String tableName) throws IOException {
+        deleteTable(HBaseConnection.get(hbaseUrl), tableName);
+    }
+
+    public static void createHTableIfNeeded(HConnection conn, String tableName, String... families) throws IOException {
+        HBaseAdmin hbase = new HBaseAdmin(conn);
+
+        try {
+            if (tableExists(conn, tableName)) {
+                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();
+        }
+    }
+
+    public static void deleteTable(HConnection conn, String tableName) throws IOException {
+        HBaseAdmin hbase = new HBaseAdmin(conn);
+
+        try {
+            if (!tableExists(conn, tableName)) {
+                logger.debug("HTable '" + tableName + "' does not exists");
+                return;
+            }
+
+            logger.debug("delete HTable '" + tableName + "'");
+
+            if (hbase.isTableEnabled(tableName)) {
+                hbase.disableTable(tableName);
+            }
+            hbase.deleteTable(tableName);
+
+            logger.debug("HTable '" + tableName + "' deleted");
+        } finally {
+            hbase.close();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseRegionSizeCalculator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseRegionSizeCalculator.java
new file mode 100644
index 0000000..bd41a99
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseRegionSizeCalculator.java
@@ -0,0 +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.
+*/
+
+/** This class will come with HBase 2.0 in package org.apache.hadoop.hbase.util **/
+package org.apache.kylin.storage.hbase;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HRegionInfo;
+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.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HBaseRegionSizeCalculator {
+
+    private static final Logger logger = LoggerFactory.getLogger(HBaseRegionSizeCalculator.class);
+
+    /**
+     * Maps each region to its size in bytes.
+     **/
+    private final Map<byte[], Long> sizeMap = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
+
+    static final String ENABLE_REGIONSIZECALCULATOR = "hbase.regionsizecalculator.enable";
+
+    /**
+     * 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 {
+
+        try {
+            if (!enabled(table.getConfiguration())) {
+                logger.info("Region size calculation disabled.");
+                return;
+            }
+
+            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
+
+            // Get regions for table.
+            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
+            Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+
+            for (HRegionInfo regionInfo : tableRegionInfos) {
+                tableRegions.add(regionInfo.getRegionName());
+            }
+
+            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
+            Collection<ServerName> servers = clusterStatus.getServers();
+            final long megaByte = 1024L * 1024L;
+
+            // Iterate all cluster regions, filter regions from our table and
+            // compute their size.
+            for (ServerName serverName : servers) {
+                ServerLoad serverLoad = clusterStatus.getLoad(serverName);
+
+                for (RegionLoad regionLoad : serverLoad.getRegionsLoad().values()) {
+                    byte[] regionId = regionLoad.getName();
+
+                    if (tableRegions.contains(regionId)) {
+
+                        long regionSizeBytes = regionLoad.getStorefileSizeMB() * megaByte;
+                        sizeMap.put(regionId, regionSizeBytes);
+
+                        // logger.info("Region " + regionLoad.getNameAsString()
+                        // + " has size " + regionSizeBytes);
+                    }
+                }
+            }
+        } finally {
+            hBaseAdmin.close();
+        }
+
+    }
+
+    boolean enabled(Configuration configuration) {
+        return configuration.getBoolean(ENABLE_REGIONSIZECALCULATOR, true);
+    }
+
+    /**
+     * Returns size of given region in bytes. Returns 0 if region was not found.
+     **/
+    public long getRegionSize(byte[] regionId) {
+        Long size = sizeMap.get(regionId);
+        if (size == null) {
+            logger.info("Unknown region:" + Arrays.toString(regionId));
+            return 0;
+        } else {
+            return size;
+        }
+    }
+
+    public Map<byte[], Long> getRegionSizeMap() {
+        return Collections.unmodifiableMap(sizeMap);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
new file mode 100644
index 0000000..0e897c8
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
@@ -0,0 +1,326 @@
+/*
+ * 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 com.google.common.collect.Lists;
+
+import org.apache.commons.io.IOUtils;
+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.client.*;
+import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
+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.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.engine.mr.HadoopUtil;
+
+import java.io.*;
+import java.util.*;
+
+public class HBaseResourceStore extends ResourceStore {
+
+    private static final String DEFAULT_TABLE_NAME = "kylin_metadata";
+    private static final String FAMILY = "f";
+    private static final byte[] B_FAMILY = Bytes.toBytes(FAMILY);
+    private static final String COLUMN = "c";
+    private static final byte[] B_COLUMN = Bytes.toBytes(COLUMN);
+    private static final String COLUMN_TS = "t";
+    private static final byte[] B_COLUMN_TS = Bytes.toBytes(COLUMN_TS);
+
+    private static final Map<String, String> TABLE_SUFFIX_MAP = new LinkedHashMap<String, String>();
+
+    static {
+        TABLE_SUFFIX_MAP.put(CUBE_RESOURCE_ROOT + "/", "_cube");
+        TABLE_SUFFIX_MAP.put(DICT_RESOURCE_ROOT + "/", "_dict");
+        TABLE_SUFFIX_MAP.put("/invertedindex/", "_invertedindex");
+        TABLE_SUFFIX_MAP.put(JOB_PATH_ROOT + "/", "_job");
+        TABLE_SUFFIX_MAP.put(JOB_OUTPUT_PATH_ROOT + "/", "_job_output");
+        TABLE_SUFFIX_MAP.put(PROJECT_RESOURCE_ROOT + "/", "_proj");
+        TABLE_SUFFIX_MAP.put(SNAPSHOT_RESOURCE_ROOT + "/", "_table_snapshot");
+        TABLE_SUFFIX_MAP.put("", ""); // DEFAULT CASE
+    }
+
+    final String tableNameBase;
+    final String hbaseUrl;
+
+    //    final Map<String, String> tableNameMap; // path prefix ==> HBase table name
+
+    private HConnection getConnection() throws IOException {
+        return HBaseConnection.get(hbaseUrl);
+    }
+
+    public HBaseResourceStore(KylinConfig kylinConfig) throws IOException {
+        super(kylinConfig);
+
+        String metadataUrl = kylinConfig.getMetadataUrl();
+        // split TABLE@HBASE_URL
+        int cut = metadataUrl.indexOf('@');
+        tableNameBase = cut < 0 ? DEFAULT_TABLE_NAME : metadataUrl.substring(0, cut);
+        hbaseUrl = cut < 0 ? metadataUrl : metadataUrl.substring(cut + 1);
+
+        createHTableIfNeeded(getAllInOneTableName());
+
+        //        tableNameMap = new LinkedHashMap<String, String>();
+        //        for (Entry<String, String> entry : TABLE_SUFFIX_MAP.entrySet()) {
+        //            String pathPrefix = entry.getKey();
+        //            String tableName = tableNameBase + entry.getValue();
+        //            tableNameMap.put(pathPrefix, tableName);
+        //            createHTableIfNeeded(tableName);
+        //        }
+
+    }
+
+    private void createHTableIfNeeded(String tableName) throws IOException {
+        HBaseConnection.createHTableIfNeeded(getConnection(), tableName, FAMILY);
+    }
+
+    private String getAllInOneTableName() {
+        return tableNameBase;
+    }
+
+    @Override
+    protected ArrayList<String> listResourcesImpl(String resPath) throws IOException {
+        assert resPath.startsWith("/");
+        String lookForPrefix = resPath.endsWith("/") ? resPath : resPath + "/";
+        byte[] startRow = Bytes.toBytes(lookForPrefix);
+        byte[] endRow = Bytes.toBytes(lookForPrefix);
+        endRow[endRow.length - 1]++;
+
+        ArrayList<String> result = new ArrayList<String>();
+
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Scan scan = new Scan(startRow, endRow);
+        scan.setFilter(new KeyOnlyFilter());
+        try {
+            ResultScanner scanner = table.getScanner(scan);
+            for (Result r : scanner) {
+                String path = Bytes.toString(r.getRow());
+                assert path.startsWith(lookForPrefix);
+                int cut = path.indexOf('/', lookForPrefix.length());
+                String child = cut < 0 ? path : path.substring(0, cut);
+                if (result.contains(child) == false)
+                    result.add(child);
+            }
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+        // return null to indicate not a folder
+        return result.isEmpty() ? null : result;
+    }
+
+    @Override
+    protected boolean existsImpl(String resPath) throws IOException {
+        Result r = getByScan(resPath, null, null);
+        return r != null;
+    }
+
+    @Override
+    protected List<RawResource> getAllResources(String rangeStart, String rangeEnd) throws IOException {
+        byte[] startRow = Bytes.toBytes(rangeStart);
+        byte[] endRow = plusZero(Bytes.toBytes(rangeEnd));
+
+        Scan scan = new Scan(startRow, endRow);
+        scan.addColumn(B_FAMILY, B_COLUMN_TS);
+        scan.addColumn(B_FAMILY, B_COLUMN);
+
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        List<RawResource> result = Lists.newArrayList();
+        try {
+            ResultScanner scanner = table.getScanner(scan);
+            for (Result r : scanner) {
+                result.add(new RawResource(getInputStream(Bytes.toString(r.getRow()), r), getTimestamp(r)));
+            }
+        } catch (IOException e) {
+            for (RawResource rawResource : result) {
+                IOUtils.closeQuietly(rawResource.resource);
+            }
+            throw e;
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+        return result;
+    }
+
+    private InputStream getInputStream(String resPath, Result r) throws IOException {
+        if (r == null) {
+            return null;
+        }
+        byte[] value = r.getValue(B_FAMILY, B_COLUMN);
+        if (value.length == 0) {
+            Path redirectPath = bigCellHDFSPath(resPath);
+            Configuration hconf = HadoopUtil.getCurrentConfiguration();
+            FileSystem fileSystem = FileSystem.get(hconf);
+
+            return fileSystem.open(redirectPath);
+        } else {
+            return new ByteArrayInputStream(value);
+        }
+    }
+
+    private long getTimestamp(Result r) {
+        if (r == null) {
+            return 0;
+        } else {
+            return Bytes.toLong(r.getValue(B_FAMILY, B_COLUMN_TS));
+        }
+    }
+
+    @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);
+    }
+
+    @Override
+    protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException {
+        ByteArrayOutputStream bout = new ByteArrayOutputStream();
+        IOUtils.copy(content, bout);
+        bout.close();
+
+        HTableInterface table = getConnection().getTable(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);
+        }
+    }
+
+    @Override
+    protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        try {
+            byte[] row = Bytes.toBytes(resPath);
+            byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
+            Put put = buildPut(resPath, newTS, row, content, table);
+
+            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 " + oldTS + ", but it is " + real);
+            }
+
+            table.flushCommits();
+
+            return newTS;
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+    }
+
+    @Override
+    protected void deleteResourceImpl(String resPath) throws IOException {
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        try {
+            Delete del = new Delete(Bytes.toBytes(resPath));
+            table.delete(del);
+            table.flushCommits();
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+    }
+
+    @Override
+    protected String getReadableResourcePathImpl(String resPath) {
+        return getAllInOneTableName() + "(key='" + resPath + "')@" + kylinConfig.getMetadataUrl();
+    }
+
+    private Result getByScan(String path, byte[] family, byte[] column) throws IOException {
+        byte[] startRow = Bytes.toBytes(path);
+        byte[] endRow = plusZero(startRow);
+
+        Scan scan = new Scan(startRow, endRow);
+        if (family == null || column == null) {
+            scan.setFilter(new KeyOnlyFilter());
+        } else {
+            scan.addColumn(family, column);
+        }
+
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        try {
+            ResultScanner scanner = table.getScanner(scan);
+            Result result = null;
+            for (Result r : scanner) {
+                result = r;
+            }
+            return result == null || result.isEmpty() ? null : result;
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+    }
+
+    private byte[] plusZero(byte[] startRow) {
+        byte[] endRow = Arrays.copyOf(startRow, startRow.length + 1);
+        endRow[endRow.length - 1] = 0;
+        return endRow;
+    }
+
+    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
+        Path redirectPath = bigCellHDFSPath(resPath);
+        Configuration hconf = HadoopUtil.getCurrentConfiguration();
+        FileSystem fileSystem = FileSystem.get(hconf);
+
+        if (fileSystem.exists(redirectPath)) {
+            fileSystem.delete(redirectPath, true);
+        }
+
+        FSDataOutputStream out = fileSystem.create(redirectPath);
+
+        try {
+            out.write(largeColumn);
+        } finally {
+            IOUtils.closeQuietly(out);
+        }
+
+        return redirectPath;
+    }
+
+    public Path bigCellHDFSPath(String resPath) {
+        String hdfsWorkingDirectory = this.kylinConfig.getHdfsWorkingDirectory();
+        Path redirectPath = new Path(hdfsWorkingDirectory, "resources" + resPath);
+        return redirectPath;
+    }
+
+    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
+        int kvSizeLimit = this.kylinConfig.getHBaseKeyValueSize();
+        if (content.length > kvSizeLimit) {
+            writeLargeCellToHdfs(resPath, content, table);
+            content = BytesUtil.EMPTY_BYTE_ARRAY;
+        }
+
+        Put put = new Put(row);
+        put.add(B_FAMILY, B_COLUMN, content);
+        put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
+
+        return put;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ZookeeperJobLock.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ZookeeperJobLock.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ZookeeperJobLock.java
new file mode 100644
index 0000000..5b13eb9
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ZookeeperJobLock.java
@@ -0,0 +1,83 @@
+package org.apache.kylin.storage.hbase;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.CuratorFrameworkState;
+import org.apache.curator.framework.recipes.locks.InterProcessMutex;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.job.lock.JobLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class ZookeeperJobLock implements JobLock {
+    private Logger logger = LoggerFactory.getLogger(ZookeeperJobLock.class);
+
+    private static final String ZOOKEEPER_LOCK_PATH = "/kylin/job_engine/lock";
+
+    private String scheduleID;
+    private InterProcessMutex sharedLock;
+    private CuratorFramework zkClient;
+
+    @Override
+    public boolean lock() {
+        this.scheduleID = schedulerId();
+        String ZKConnectString = getZKConnectString();
+        if (StringUtils.isEmpty(ZKConnectString)) {
+            throw new IllegalArgumentException("ZOOKEEPER_QUORUM is empty!");
+        }
+
+        RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+        this.zkClient = CuratorFrameworkFactory.newClient(ZKConnectString, retryPolicy);
+        this.zkClient.start();
+        this.sharedLock = new InterProcessMutex(zkClient, this.scheduleID);
+        boolean hasLock = false;
+        try {
+            hasLock = sharedLock.acquire(3, TimeUnit.SECONDS);
+        } catch (Exception e) {
+            logger.warn("error acquire lock", e);
+        }
+        if (!hasLock) {
+            logger.warn("fail to acquire lock, scheduler has not been started");
+            zkClient.close();
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public void unlock() {
+        releaseLock();
+    }
+
+    private String getZKConnectString() {
+        Configuration conf = HBaseConnection.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        return conf.get(HConstants.ZOOKEEPER_QUORUM) + ":" + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    }
+
+    private void releaseLock() {
+        try {
+            if (zkClient.getState().equals(CuratorFrameworkState.STARTED)) {
+                // client.setData().forPath(ZOOKEEPER_LOCK_PATH, null);
+                if (zkClient.checkExists().forPath(scheduleID) != null) {
+                    zkClient.delete().guaranteed().deletingChildrenIfNeeded().forPath(scheduleID);
+                }
+            }
+        } catch (Exception e) {
+            logger.error("error release lock:" + scheduleID);
+            throw new RuntimeException(e);
+        }
+    }
+
+    private String schedulerId() {
+        return ZOOKEEPER_LOCK_PATH + "/" + KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2456215/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HBaseMetadataTestCase.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HBaseMetadataTestCase.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HBaseMetadataTestCase.java
new file mode 100644
index 0000000..0926604
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/HBaseMetadataTestCase.java
@@ -0,0 +1,75 @@
+/*
+ * 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 org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.AbstractKylinTestCase;
+import org.apache.kylin.common.util.ClassUtil;
+
+import java.io.File;
+
+/**
+ * @author ysong1
+ */
+public class HBaseMetadataTestCase extends AbstractKylinTestCase {
+
+    static {
+        if (useSandbox()) {
+            try {
+                ClassUtil.addClasspath(new File("../examples/test_case_data/sandbox/").getAbsolutePath());
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    @Override
+    public void createTestMetadata() throws Exception {
+        staticCreateTestMetadata();
+    }
+
+    @Override
+    public void cleanupTestMetadata() {
+        staticCleanupTestMetadata();
+    }
+
+    public static void staticCreateTestMetadata() throws Exception {
+        if (useSandbox()) {
+            staticCreateTestMetadata(SANDBOX_TEST_DATA);
+        } else {
+            staticCreateTestMetadata(MINICLUSTER_TEST_DATA);
+            HBaseMiniclusterHelper.startupMinicluster();
+        }
+
+    }
+    public static void staticCreateTestMetadata(String kylinConfigFolder) {
+
+        KylinConfig.destoryInstance();
+
+        if (System.getProperty(KylinConfig.KYLIN_CONF) == null && System.getenv(KylinConfig.KYLIN_CONF) == null)
+            System.setProperty(KylinConfig.KYLIN_CONF, kylinConfigFolder);
+
+    }
+
+    public static boolean useSandbox() {
+        String useSandbox = System.getProperty("useSandbox");
+        return Boolean.parseBoolean(useSandbox);
+    }
+    
+}