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

[01/55] [abbrv] kylin git commit: KYLIN-1858 remove all ii related code [Forced Update!]

Repository: kylin
Updated Branches:
  refs/heads/1.5.x-CDH5.7 159d9fd8b -> b46029fba (forced update)


http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/protobuf/II.proto
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/protobuf/II.proto b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/protobuf/II.proto
deleted file mode 100644
index d2c34d4..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/protobuf/II.proto
+++ /dev/null
@@ -1,65 +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.
-//
-
-// usage:
-// protoc  --java_out=./storage-hbase/src/main/java  ./storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/protobuf/II.proto
-
-option java_package = "org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated";
-
-option java_outer_classname = "IIProtos";
-
-option java_generic_services = true;
-
-option java_generate_equals_and_hash = true;
-
-option optimize_for = SPEED;
-
-message IIRequest {
-    required bytes type = 1;
-    required bytes filter = 2;
-    required bytes projector = 3;
-    required bytes aggregator = 4;
-    optional bytes tsRange = 5;
-}
-
-message IIResponseInternal {
-    message IIRow {
-        required bytes columns = 1;
-        optional bytes measures = 2;
-    }
-    //all entries in this struct be optional to conveniently add more entries in the future
-    message Stats {
-        optional int32 myShard = 1;
-        optional int64 latestDataTime = 2;
-        optional int64 serviceStartTime = 3;
-        optional int64 serviceEndTime = 4;
-        optional int32 scannedSlices = 5;
-    }
-    repeated IIRow rows = 1;
-    required Stats stats = 2;
-}
-
-message IIResponse {
-    required bytes blob = 1;
-}
-
-
-
-service RowsService {
-    rpc getRows (IIRequest) returns (IIResponse);
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput.java
index 11c1711..c634a1d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput.java
@@ -20,7 +20,6 @@ package org.apache.kylin.storage.hbase.steps;
 
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.IMROutput;
-import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 
 public class HBaseMROutput implements IMROutput {
@@ -43,23 +42,6 @@ public class HBaseMROutput implements IMROutput {
     }
 
     @Override
-    public IMRBatchInvertedIndexingOutputSide getBatchInvertedIndexingOutputSide(final IISegment seg) {
-        return new IMRBatchInvertedIndexingOutputSide() {
-            HBaseMRSteps steps = new HBaseMRSteps(seg);
-
-            @Override
-            public void addStepPhase3_BuildII(DefaultChainedExecutable jobFlow, String rootPath) {
-                steps.addSaveIIToHTableSteps(jobFlow, rootPath);
-            }
-
-            @Override
-            public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
-                steps.addInvertedIndexGarbageCollectionSteps(jobFlow);
-            }
-        };
-    }
-
-    @Override
     public IMRBatchMergeOutputSide getBatchMergeOutputSide(final CubeSegment seg) {
         return new IMRBatchMergeOutputSide() {
             HBaseMRSteps steps = new HBaseMRSteps(seg);

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
index dcff5e9..9102fbc 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
@@ -32,9 +32,6 @@ import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.metadata.realization.IRealizationSegment;
 import org.apache.kylin.storage.hbase.HBaseConnection;
-import org.apache.kylin.storage.hbase.ii.IIBulkLoadJob;
-import org.apache.kylin.storage.hbase.ii.IICreateHFileJob;
-import org.apache.kylin.storage.hbase.ii.IICreateHTableJob;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -201,79 +198,4 @@ public class HBaseMRSteps extends JobBuilderSupport {
         jobFlow.addTask(step);
     }
 
-    public void addSaveIIToHTableSteps(DefaultChainedExecutable jobFlow, String rootPath) {
-        // create htable if it doesn't exist
-        jobFlow.addTask(createCreateIIHTableStep(seg));
-
-        final String iiPath = rootPath + "*";
-
-        // generate hfiles step
-        jobFlow.addTask(createConvertIIToHfileStep(seg, iiPath, jobFlow.getId()));
-
-        // bulk load step
-        jobFlow.addTask(createIIBulkLoadStep(seg, jobFlow.getId()));
-
-    }
-
-    public void addInvertedIndexGarbageCollectionSteps(DefaultChainedExecutable jobFlow) {
-        String jobId = jobFlow.getId();
-
-        List<String> toDeletePaths = new ArrayList<>();
-        toDeletePaths.add(getJobWorkingDir(jobId));
-
-        HDFSPathGarbageCollectionStep step = new HDFSPathGarbageCollectionStep();
-        step.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
-        step.setDeletePaths(toDeletePaths);
-        step.setJobId(jobId);
-
-        jobFlow.addTask(step);
-    }
-
-    private HadoopShellExecutable createCreateIIHTableStep(IRealizationSegment seg) {
-        HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
-        createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, BatchConstants.ARG_II_NAME, seg.getRealization().getName());
-        appendExecCmdParameters(cmd, BatchConstants.ARG_HTABLE_NAME, seg.getStorageLocationIdentifier());
-
-        createHtableStep.setJobParams(cmd.toString());
-        createHtableStep.setJobClass(IICreateHTableJob.class);
-
-        return createHtableStep;
-    }
-
-    private MapReduceExecutable createConvertIIToHfileStep(IRealizationSegment seg, String inputPath, String jobId) {
-        MapReduceExecutable createHFilesStep = new MapReduceExecutable();
-        createHFilesStep.setName(ExecutableConstants.STEP_NAME_CONVERT_II_TO_HFILE);
-        StringBuilder cmd = new StringBuilder();
-
-        appendMapReduceParameters(cmd);
-        appendExecCmdParameters(cmd, BatchConstants.ARG_II_NAME, seg.getRealization().getName());
-        appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, inputPath);
-        appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, getHFilePath(jobId));
-        appendExecCmdParameters(cmd, BatchConstants.ARG_HTABLE_NAME, seg.getStorageLocationIdentifier());
-        appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_HFile_Generator_" + seg.getRealization().getName() + "_Step");
-
-        createHFilesStep.setMapReduceParams(cmd.toString());
-        createHFilesStep.setMapReduceJobClass(IICreateHFileJob.class);
-
-        return createHFilesStep;
-    }
-
-    private HadoopShellExecutable createIIBulkLoadStep(IRealizationSegment seg, String jobId) {
-        HadoopShellExecutable bulkLoadStep = new HadoopShellExecutable();
-        bulkLoadStep.setName(ExecutableConstants.STEP_NAME_BULK_LOAD_HFILE);
-
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, getHFilePath(jobId));
-        appendExecCmdParameters(cmd, BatchConstants.ARG_HTABLE_NAME, seg.getStorageLocationIdentifier());
-        appendExecCmdParameters(cmd, BatchConstants.ARG_II_NAME, seg.getRealization().getName());
-
-        bulkLoadStep.setJobParams(cmd.toString());
-        bulkLoadStep.setJobClass(IIBulkLoadJob.class);
-
-        return bulkLoadStep;
-
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
index 270da13..63e8ac1 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
@@ -18,17 +18,7 @@
 
 package org.apache.kylin.storage.hbase.util;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Matcher;
-
+import com.google.common.collect.Lists;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -48,17 +38,17 @@ import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
-import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.*;
+import java.util.regex.Matcher;
 
 /**
  */
@@ -184,7 +174,6 @@ public class DeployCoprocessorCLI {
 
     public static void addCoprocessorOnHTable(HTableDescriptor desc, Path hdfsCoprocessorJar) throws IOException {
         logger.info("Add coprocessor on " + desc.getNameAsString());
-        desc.addCoprocessor(IIEndpointClass, hdfsCoprocessorJar, 1000, null);
         desc.addCoprocessor(CubeEndpointClass, hdfsCoprocessorJar, 1001, null);
         desc.addCoprocessor(CubeObserverClass, hdfsCoprocessorJar, 1002, null);
     }
@@ -384,18 +373,6 @@ public class DeployCoprocessorCLI {
             }
         }
 
-        for (IIInstance ii : IIManager.getInstance(config).listAllIIs()) {
-            if (ii.getStatus() == RealizationStatusEnum.READY) {
-                for (IISegment seg : ii.getSegments()) {//streaming segment is never "READY"
-                    String tableName = seg.getStorageLocationIdentifier();
-                    if (StringUtils.isBlank(tableName) == false) {
-                        result.add(tableName);
-                        System.out.println("added new table: " + tableName);
-                    }
-                }
-            }
-        }
-
         return result;
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/IIDeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/IIDeployCoprocessorCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/IIDeployCoprocessorCLI.java
deleted file mode 100644
index 93a9e67..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/IIDeployCoprocessorCLI.java
+++ /dev/null
@@ -1,157 +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.storage.hbase.util;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.engine.mr.HadoopUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * THIS IS A TAILORED DUPLICATE OF org.apache.kylin.storage.hbase.util.DeployCoprocessorCLI TO AVOID CYCLIC
- * DEPENDENCY. INVERTED-INDEX CODE NOW SPLITTED ACROSS kylin-invertedindex AND kylin-storage-hbase.
- * DEFENITELY NEED FURTHER REFACTOR.
- */
-public class IIDeployCoprocessorCLI {
-
-    private static final Logger logger = LoggerFactory.getLogger(IIDeployCoprocessorCLI.class);
-
-    public static final String CubeObserverClass = "org.apache.kylin.storage.hbase.cube.v1.coprocessor.observer.AggregateRegionObserver";
-    public static final String CubeEndpointClass = "org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.CubeVisitService";
-    public static final String IIEndpointClass = "org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.IIEndpoint";
-
-    public static void deployCoprocessor(HTableDescriptor tableDesc) {
-        try {
-            initHTableCoprocessor(tableDesc);
-            logger.info("hbase table " + tableDesc.getName() + " deployed with coprocessor.");
-
-        } catch (Exception ex) {
-            logger.error("Error deploying coprocessor on " + tableDesc.getName(), ex);
-            logger.error("Will try creating the table without coprocessor.");
-        }
-    }
-
-    private static void initHTableCoprocessor(HTableDescriptor desc) throws IOException {
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        Configuration hconf = HadoopUtil.getCurrentConfiguration();
-        FileSystem fileSystem = FileSystem.get(hconf);
-
-        String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();
-        Path hdfsCoprocessorJar = uploadCoprocessorJar(localCoprocessorJar, fileSystem, null);
-
-        addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
-    }
-
-    private static void addCoprocessorOnHTable(HTableDescriptor desc, Path hdfsCoprocessorJar) throws IOException {
-        logger.info("Add coprocessor on " + desc.getNameAsString());
-        desc.addCoprocessor(IIEndpointClass, hdfsCoprocessorJar, 1000, null);
-        desc.addCoprocessor(CubeEndpointClass, hdfsCoprocessorJar, 1001, null);
-        desc.addCoprocessor(CubeObserverClass, hdfsCoprocessorJar, 1002, null);
-    }
-
-    private static Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem, Set<String> oldJarPaths) throws IOException {
-        Path uploadPath = null;
-        File localCoprocessorFile = new File(localCoprocessorJar);
-
-        // check existing jars
-        if (oldJarPaths == null) {
-            oldJarPaths = new HashSet<String>();
-        }
-        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, KylinConfig.getInstanceFromEnv());
-        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
-            if (isSame(localCoprocessorFile, fileStatus)) {
-                uploadPath = fileStatus.getPath();
-                break;
-            }
-            String filename = fileStatus.getPath().toString();
-            if (filename.endsWith(".jar")) {
-                oldJarPaths.add(filename);
-            }
-        }
-
-        // upload if not existing
-        if (uploadPath == null) {
-            // figure out a unique new jar file name
-            Set<String> oldJarNames = new HashSet<String>();
-            for (String path : oldJarPaths) {
-                oldJarNames.add(new Path(path).getName());
-            }
-            String baseName = getBaseFileName(localCoprocessorJar);
-            String newName = null;
-            int i = 0;
-            while (newName == null) {
-                newName = baseName + "-" + (i++) + ".jar";
-                if (oldJarNames.contains(newName))
-                    newName = null;
-            }
-
-            // upload
-            uploadPath = new Path(coprocessorDir, newName);
-            FileInputStream in = null;
-            FSDataOutputStream out = null;
-            try {
-                in = new FileInputStream(localCoprocessorFile);
-                out = fileSystem.create(uploadPath);
-                IOUtils.copy(in, out);
-            } finally {
-                IOUtils.closeQuietly(in);
-                IOUtils.closeQuietly(out);
-            }
-
-            fileSystem.setTimes(uploadPath, localCoprocessorFile.lastModified(), -1);
-
-        }
-
-        uploadPath = uploadPath.makeQualified(fileSystem.getUri(), null);
-        return uploadPath;
-    }
-
-    private static boolean isSame(File localCoprocessorFile, FileStatus fileStatus) {
-        return fileStatus.getLen() == localCoprocessorFile.length() && fileStatus.getModificationTime() == localCoprocessorFile.lastModified();
-    }
-
-    private static String getBaseFileName(String localCoprocessorJar) {
-        File localJar = new File(localCoprocessorJar);
-        String baseName = localJar.getName();
-        if (baseName.endsWith(".jar"))
-            baseName = baseName.substring(0, baseName.length() - ".jar".length());
-        return baseName;
-    }
-
-    private static Path getCoprocessorHDFSDir(FileSystem fileSystem, KylinConfig config) throws IOException {
-        String hdfsWorkingDirectory = config.getHdfsWorkingDirectory();
-        Path coprocessorDir = new Path(hdfsWorkingDirectory, "coprocessor");
-        fileSystem.mkdirs(coprocessorDir);
-        return coprocessorDir;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
index af64df7..3f4a6d5 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
@@ -50,9 +50,6 @@ import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.job.JobInstance;
 import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.execution.ExecutableState;
@@ -107,7 +104,6 @@ public class StorageCleanupJob extends AbstractHadoopJob {
 
     private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
         CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
-        IIManager iiManager = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
         long TIME_THREADSHOLD = KylinConfig.getInstanceFromEnv().getStorageCleanupTimeThreshold();
         // get all kylin hbase tables
         HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
@@ -138,18 +134,6 @@ public class StorageCleanupJob extends AbstractHadoopJob {
             }
         }
 
-        // remove every ii segment htable from drop list
-        for (IIInstance ii : iiManager.listAllIIs()) {
-            for (IISegment seg : ii.getSegments()) {
-                String tablename = seg.getStorageLocationIdentifier();
-
-                if (allTablesNeedToBeDropped.contains(tablename)) {
-                    allTablesNeedToBeDropped.remove(tablename);
-                    logger.info("Exclude table " + tablename + " from drop list, as the table belongs to ii " + ii.getName() + " with status " + ii.getStatus());
-                }
-            }
-        }
-
         if (delete == true) {
             // drop tables
             ExecutorService executorService = Executors.newSingleThreadExecutor();

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
index 26ee055..b1f275f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
@@ -33,9 +33,6 @@ 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.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.IRealizationConstants;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
@@ -129,18 +126,6 @@ public class UpdateHTableHostCLI {
             }
         }
 
-        for (IIInstance ii : IIManager.getInstance(config).listAllIIs()) {
-            if (ii.getStatus() == RealizationStatusEnum.READY) {
-                for (IISegment seg : ii.getSegments()) {//streaming segment is never "READY"
-                    String tableName = seg.getStorageLocationIdentifier();
-                    if (!StringUtils.isBlank(tableName)) {
-                        result.add(tableName);
-                        System.out.println("added new table: " + tableName);
-                    }
-                }
-            }
-        }
-
         return result;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/common/TsConditionEraserTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/common/TsConditionEraserTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/common/TsConditionEraserTest.java
deleted file mode 100644
index 90f3370..0000000
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/common/TsConditionEraserTest.java
+++ /dev/null
@@ -1,160 +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.storage.hbase.common;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.StringCodeSystem;
-import org.apache.kylin.metadata.filter.TsConditionEraser;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilterSerializer;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorFilter;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-public class TsConditionEraserTest extends LocalFileMetadataTestCase {
-    IIInstance ii;
-    TableRecordInfo tableRecordInfo;
-    CoprocessorFilter filter;
-    TableDesc factTableDesc;
-
-    TblColRef caldt;
-    TblColRef siteId;
-
-    @Before
-    public void setup() throws IOException {
-        this.createTestMetadata();
-        IIManager iiManager = IIManager.getInstance(getTestConfig());
-        this.ii = iiManager.getII("test_kylin_ii_left_join");
-        IISegment segment = iiManager.buildSegment(ii, 0, System.currentTimeMillis());
-        ii.getSegments().add(segment);
-        this.tableRecordInfo = new TableRecordInfo(ii.getFirstSegment());
-        this.factTableDesc = MetadataManager.getInstance(getTestConfig()).getTableDesc("DEFAULT.TEST_KYLIN_FACT");
-        this.caldt = this.ii.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "CAL_DT");
-        this.siteId = this.ii.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "LSTG_SITE_ID");
-    }
-
-    @After
-    public void cleanUp() {
-        cleanupTestMetadata();
-    }
-
-    private TupleFilter mockFilter1(int year) {
-        CompareTupleFilter aFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GT);
-        aFilter.addChild(new ColumnTupleFilter(caldt));
-        aFilter.addChild(new ConstantTupleFilter(year + "-01-01"));
-
-        CompareTupleFilter bFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        bFilter.addChild(new ColumnTupleFilter(caldt));
-        bFilter.addChild(new ConstantTupleFilter(year + "-01-04"));
-
-        CompareTupleFilter cFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        cFilter.addChild(new ColumnTupleFilter(caldt));
-        cFilter.addChild(new ConstantTupleFilter(year + "-01-03"));
-
-        CompareTupleFilter dFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.EQ);
-        dFilter.addChild(new ColumnTupleFilter(siteId));
-        dFilter.addChild(new ConstantTupleFilter("0"));
-
-        LogicalTupleFilter subRoot = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
-        subRoot.addChildren(Lists.newArrayList(aFilter, bFilter, cFilter, dFilter));
-
-        CompareTupleFilter outFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        outFilter.addChild(new ColumnTupleFilter(caldt));
-        outFilter.addChild(new ConstantTupleFilter(year + "-01-02"));
-
-        LogicalTupleFilter root = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
-        root.addChildren(Lists.newArrayList(subRoot, outFilter));
-        return root;
-    }
-
-    private TupleFilter mockFilter2(int year) {
-        CompareTupleFilter aFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GT);
-        aFilter.addChild(new ColumnTupleFilter(caldt));
-        aFilter.addChild(new ConstantTupleFilter(year + "-01-01"));
-
-        CompareTupleFilter bFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        bFilter.addChild(new ColumnTupleFilter(caldt));
-        bFilter.addChild(new ConstantTupleFilter(year + "-01-04"));
-
-        CompareTupleFilter cFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        cFilter.addChild(new ColumnTupleFilter(caldt));
-        cFilter.addChild(new ConstantTupleFilter(year + "-01-03"));
-
-        CompareTupleFilter dFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.EQ);
-        dFilter.addChild(new ColumnTupleFilter(siteId));
-        dFilter.addChild(new ConstantTupleFilter("0"));
-
-        LogicalTupleFilter subRoot = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.OR);
-        subRoot.addChildren(Lists.newArrayList(aFilter, bFilter, cFilter, dFilter));
-
-        CompareTupleFilter outFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        outFilter.addChild(new ColumnTupleFilter(caldt));
-        outFilter.addChild(new ConstantTupleFilter(year + "-01-02"));
-
-        LogicalTupleFilter root = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
-        root.addChildren(Lists.newArrayList(subRoot, outFilter));
-        return root;
-    }
-
-    @Test
-    public void positiveTest() {
-
-        TupleFilter a = mockFilter1(2000);
-        TupleFilter b = mockFilter1(2001);
-
-        TsConditionEraser decoratorA = new TsConditionEraser(caldt, a);
-        byte[] aBytes = TupleFilterSerializer.serialize(a, decoratorA, StringCodeSystem.INSTANCE);
-        TsConditionEraser decoratorB = new TsConditionEraser(caldt, b);
-        byte[] bBytes = TupleFilterSerializer.serialize(b, decoratorB, StringCodeSystem.INSTANCE);
-        Assert.assertArrayEquals(aBytes, bBytes);
-
-    }
-
-    @Test
-    public void negativeTest() {
-        TupleFilter a = mockFilter2(2000);
-        TupleFilter b = mockFilter2(2001);
-
-        TsConditionEraser decoratorA = new TsConditionEraser(caldt, a);
-        byte[] aBytes = TupleFilterSerializer.serialize(a, decoratorA, StringCodeSystem.INSTANCE);
-        TsConditionEraser decoratorB = new TsConditionEraser(caldt, b);
-        byte[] bBytes = TupleFilterSerializer.serialize(b, decoratorB, StringCodeSystem.INSTANCE);
-        Assert.assertFalse(Arrays.equals(aBytes, bBytes));
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluatorTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluatorTest.java
deleted file mode 100644
index e2eeddb..0000000
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluatorTest.java
+++ /dev/null
@@ -1,228 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
-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.storage.hbase.ii.coprocessor.endpoint.BitMapFilterEvaluator.BitMapProvider;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-public class BitMapFilterEvaluatorTest {
-
-    static TblColRef colA;
-    static TblColRef colB;
-
-    static {
-        TableDesc table = TableDesc.mockup("DEFAULT.TABLE");
-        colA = ColumnDesc.mockup(table, 1, "colA", "string").getRef();
-        colB = ColumnDesc.mockup(table, 1, "colB", "string").getRef();
-    }
-
-    static class MockBitMapProivder implements BitMapProvider {
-
-        private static final int MAX_ID = 8;
-        private static final int REC_COUNT = 10;
-
-        @Override
-        public ConciseSet getBitMap(TblColRef col, Integer startId, Integer endId) {
-            if (!col.equals(colA))
-                return null;
-
-            // i-th record has value ID i, and last record has value null
-            if (startId == null && endId == null) {
-                //entry for getting null value
-                ConciseSet s = new ConciseSet();
-                s.add(getRecordCount() - 1);
-                return s;
-            }
-
-            int start = 0;
-            int end = MAX_ID;
-            if (startId != null) {
-                start = startId;
-            }
-            if (endId != null) {
-                end = endId;
-            }
-
-            ConciseSet ret = new ConciseSet();
-            for (int i = start; i <= end; ++i) {
-                ConciseSet temp = getBitMap(col, i);
-                ret.addAll(temp);
-            }
-            return ret;
-        }
-
-        public ConciseSet getBitMap(TblColRef col, int valueId) {
-            if (!col.equals(colA))
-                return null;
-
-            // i-th record has value ID i, and last record has value null
-            ConciseSet bitMap = new ConciseSet();
-            if (valueId < 0 || valueId > getMaxValueId(col)) // null
-                bitMap.add(getRecordCount() - 1);
-            else
-                bitMap.add(valueId);
-
-            return bitMap;
-        }
-
-        @Override
-        public int getRecordCount() {
-            return REC_COUNT;
-        }
-
-        @Override
-        public int getMaxValueId(TblColRef col) {
-            return MAX_ID;
-        }
-    }
-
-    BitMapFilterEvaluator eval = new BitMapFilterEvaluator(new MockBitMapProivder());
-    ArrayList<CompareTupleFilter> basicFilters = Lists.newArrayList();
-    ArrayList<ConciseSet> basicResults = Lists.newArrayList();
-
-    public BitMapFilterEvaluatorTest() {
-        basicFilters.add(compare(colA, FilterOperatorEnum.ISNULL));
-        basicResults.add(set(9));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.ISNOTNULL));
-        basicResults.add(set(0, 1, 2, 3, 4, 5, 6, 7, 8));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.EQ, 0));
-        basicResults.add(set(0));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.NEQ, 0));
-        basicResults.add(set(1, 2, 3, 4, 5, 6, 7, 8));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.IN, 0, 5));
-        basicResults.add(set(0, 5));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.NOTIN, 0, 5));
-        basicResults.add(set(1, 2, 3, 4, 6, 7, 8));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.LT, 3));
-        basicResults.add(set(0, 1, 2));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.LTE, 3));
-        basicResults.add(set(0, 1, 2, 3));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.GT, 3));
-        basicResults.add(set(4, 5, 6, 7, 8));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.GTE, 3));
-        basicResults.add(set(3, 4, 5, 6, 7, 8));
-    }
-
-    @Test
-    public void testBasics() {
-        for (int i = 0; i < basicFilters.size(); i++) {
-            assertEquals(basicResults.get(i), eval.evaluate(basicFilters.get(i)));
-        }
-    }
-
-    @Test
-    public void testLogicalAnd() {
-        for (int i = 0; i < basicFilters.size(); i++) {
-            for (int j = 0; j < basicFilters.size(); j++) {
-                LogicalTupleFilter f = logical(FilterOperatorEnum.AND, basicFilters.get(i), basicFilters.get(j));
-                ConciseSet r = basicResults.get(i).clone();
-                r.retainAll(basicResults.get(j));
-                assertEquals(r, eval.evaluate(f));
-            }
-        }
-    }
-
-    @Test
-    public void testLogicalOr() {
-        for (int i = 0; i < basicFilters.size(); i++) {
-            for (int j = 0; j < basicFilters.size(); j++) {
-                LogicalTupleFilter f = logical(FilterOperatorEnum.OR, basicFilters.get(i), basicFilters.get(j));
-                ConciseSet r = basicResults.get(i).clone();
-                r.addAll(basicResults.get(j));
-                assertEquals(r, eval.evaluate(f));
-            }
-        }
-    }
-
-    @Test
-    public void testNotEvaluable() {
-        CompareTupleFilter notEvaluable = compare(colB, FilterOperatorEnum.EQ, 0);
-        assertEquals(null, eval.evaluate(notEvaluable));
-
-        LogicalTupleFilter or = logical(FilterOperatorEnum.OR, basicFilters.get(1), notEvaluable);
-        assertEquals(null, eval.evaluate(or));
-
-        LogicalTupleFilter and = logical(FilterOperatorEnum.AND, basicFilters.get(1), notEvaluable);
-        assertEquals(basicResults.get(1), eval.evaluate(and));
-    }
-
-    public static CompareTupleFilter compare(TblColRef col, TupleFilter.FilterOperatorEnum op, int... ids) {
-        CompareTupleFilter filter = new CompareTupleFilter(op);
-        filter.addChild(columnFilter(col));
-        for (int i : ids) {
-            filter.addChild(constFilter(i));
-        }
-        return filter;
-    }
-
-    public static LogicalTupleFilter logical(TupleFilter.FilterOperatorEnum op, TupleFilter... filters) {
-        LogicalTupleFilter filter = new LogicalTupleFilter(op);
-        for (TupleFilter f : filters)
-            filter.addChild(f);
-        return filter;
-    }
-
-    public static ColumnTupleFilter columnFilter(TblColRef col) {
-        return new ColumnTupleFilter(col);
-    }
-
-    public static ConstantTupleFilter constFilter(int id) {
-        return new ConstantTupleFilter(idToStr(id));
-    }
-
-    public static ConciseSet set(int... ints) {
-        ConciseSet set = new ConciseSet();
-        for (int i : ints)
-            set.add(i);
-        return set;
-    }
-
-    public static String idToStr(int id) {
-        byte[] bytes = new byte[] { (byte) id };
-        return Dictionary.dictIdToString(bytes, 0, bytes.length);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
deleted file mode 100644
index ac9e995..0000000
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
+++ /dev/null
@@ -1,177 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
-import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.metadata.datatype.LongMutable;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorProjector;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-/**
- *
- * ii test
- */
-public class EndpointAggregationTest extends LocalFileMetadataTestCase {
-
-    @Before
-    public void setup() throws IOException {
-        this.createTestMetadata();
-    }
-
-    @After
-    public void cleanUp() {
-        cleanupTestMetadata();
-    }
-
-    private List<FunctionDesc> buildAggregations() {
-        List<FunctionDesc> functions = new ArrayList<FunctionDesc>();
-
-        FunctionDesc f1 = new FunctionDesc();
-        f1.setExpression("SUM");
-        ParameterDesc p1 = new ParameterDesc();
-        p1.setType("column");
-        p1.setValue("PRICE");
-        f1.setParameter(p1);
-        f1.setReturnType("decimal");
-        functions.add(f1);
-
-        FunctionDesc f2 = new FunctionDesc();
-        f2.setExpression("MIN");
-        ParameterDesc p2 = new ParameterDesc();
-        p2.setType("column");
-        p2.setValue("PRICE");
-        f2.setParameter(p2);
-        f2.setReturnType("decimal");
-        functions.add(f2);
-
-        return functions;
-    }
-
-    @Test
-    public void testSerializeAggregator() {
-        final IIInstance ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii_left_join");
-        if (ii.getFirstSegment() == null) {
-            IISegment segment = IIManager.getInstance(getTestConfig()).buildSegment(ii, 0, System.currentTimeMillis());
-            ii.getSegments().add(segment);
-        }
-        final TableRecordInfo tableRecordInfo = new TableRecordInfo(ii.getFirstSegment());
-        final EndpointAggregators endpointAggregators = EndpointAggregators.fromFunctions(tableRecordInfo, buildAggregations());
-        byte[] x = EndpointAggregators.serialize(endpointAggregators);
-        final EndpointAggregators result = EndpointAggregators.deserialize(x);
-        assertArrayEquals(endpointAggregators.dataTypes, result.dataTypes);
-        assertArrayEquals(endpointAggregators.funcNames, result.funcNames);
-        assertArrayEquals(endpointAggregators.metricValues, result.metricValues);
-        assertEquals(endpointAggregators.rawTableRecord.getBytes().length, result.rawTableRecord.getBytes().length);
-    }
-
-    private byte[] randomBytes(final int length) {
-        byte[] result = new byte[length];
-        Random random = new Random();
-        for (int i = 0; i < length; i++) {
-            random.nextBytes(result);
-        }
-        return result;
-    }
-
-    private List<byte[]> mockData(TableRecordInfo tableRecordInfo) {
-        ArrayList<byte[]> result = Lists.newArrayList();
-        final int priceColumnIndex = 23;
-        final int groupByColumnIndex = 0;
-        TblColRef column = tableRecordInfo.getDescriptor().listAllColumns().get(priceColumnIndex);
-        FixedLenMeasureCodec codec = FixedLenMeasureCodec.get(column.getType());
-
-        byte[] data = randomBytes(tableRecordInfo.getDigest().getByteFormLen());
-        byte[] groupOne = randomBytes(tableRecordInfo.getDigest().length(groupByColumnIndex));
-        codec.write(codec.valueOf("199.99"), data, tableRecordInfo.getDigest().offset(priceColumnIndex));
-        System.arraycopy(groupOne, 0, data, tableRecordInfo.getDigest().offset(groupByColumnIndex), groupOne.length);
-        result.add(data);
-
-        data = randomBytes(tableRecordInfo.getDigest().getByteFormLen());
-        codec.write(codec.valueOf("2.09"), data, tableRecordInfo.getDigest().offset(priceColumnIndex));
-        System.arraycopy(groupOne, 0, data, tableRecordInfo.getDigest().offset(groupByColumnIndex), groupOne.length);
-        result.add(data);
-
-        byte[] groupTwo = randomBytes(tableRecordInfo.getDigest().length(groupByColumnIndex));
-        data = randomBytes(tableRecordInfo.getDigest().getByteFormLen());
-        System.arraycopy(groupTwo, 0, data, tableRecordInfo.getDigest().offset(groupByColumnIndex), groupTwo.length);
-        codec.write(codec.valueOf("100"), data, tableRecordInfo.getDigest().offset(priceColumnIndex));
-        result.add(data);
-
-        return result;
-    }
-
-    @Test
-    public void basicTest() {
-        final IIInstance ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii_left_join");
-        if (ii.getFirstSegment() == null) {
-            IISegment segment = IIManager.getInstance(getTestConfig()).buildSegment(ii, 0, System.currentTimeMillis());
-            ii.getSegments().add(segment);
-        }
-        final TableRecordInfo tableRecordInfo = new TableRecordInfo(ii.getFirstSegment());
-        final EndpointAggregators aggregators = EndpointAggregators.fromFunctions(tableRecordInfo, buildAggregations());
-        final EndpointAggregationCache aggCache = new EndpointAggregationCache(aggregators);
-        final Collection<TblColRef> dims = new HashSet<>();
-        final TblColRef groupByColumn = ii.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "LSTG_FORMAT_NAME");
-        dims.add(groupByColumn);
-        CoprocessorProjector projector = CoprocessorProjector.makeForEndpoint(tableRecordInfo, dims);
-        List<byte[]> rawData = mockData(tableRecordInfo);
-        for (int i = 0; i < rawData.size(); ++i) {
-            byte[] data = rawData.get(i);
-            AggrKey aggKey = projector.getAggrKey(data);
-            MeasureAggregator[] bufs = aggCache.getBuffer(aggKey);
-            aggregators.aggregate(bufs, data);
-            aggCache.checkMemoryUsage();
-        }
-        long sumTotal = 0;
-        long minTotal = 0;
-        for (Map.Entry<AggrKey, MeasureAggregator[]> entry : aggCache.getAllEntries()) {
-            sumTotal += ((LongMutable) entry.getValue()[0].getState()).get();
-            minTotal += ((LongMutable) entry.getValue()[1].getState()).get();
-
-        }
-        assertEquals(3020800, sumTotal);
-        assertEquals(1020900, minTotal);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/TableRecordInfoTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/TableRecordInfoTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/TableRecordInfoTest.java
deleted file mode 100644
index 3e34495..0000000
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/TableRecordInfoTest.java
+++ /dev/null
@@ -1,63 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- */
-public class TableRecordInfoTest extends LocalFileMetadataTestCase {
-    IIInstance ii;
-    TableRecordInfo tableRecordInfo;
-
-    @Before
-    public void setup() throws IOException {
-        this.createTestMetadata();
-        this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii_left_join");
-        if (ii.getFirstSegment() == null) {
-            IISegment segment = IIManager.getInstance(getTestConfig()).buildSegment(ii, 0, System.currentTimeMillis());
-            ii.getSegments().add(segment);
-        }
-        this.tableRecordInfo = new TableRecordInfo(ii.getFirstSegment());
-    }
-
-    @Test
-    public void testSerialize() {
-        byte[] x = TableRecordInfoDigest.serialize(this.tableRecordInfo.getDigest());
-        TableRecordInfoDigest d = TableRecordInfoDigest.deserialize(x);
-        assertEquals(d.getColumnCount(), 25);
-    }
-
-    @After
-    public void cleanUp() {
-        cleanupTestMetadata();
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/TsConditionExtractorTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/TsConditionExtractorTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/TsConditionExtractorTest.java
deleted file mode 100644
index 4e5a9d9..0000000
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/TsConditionExtractorTest.java
+++ /dev/null
@@ -1,214 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import java.io.IOException;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.cache.TsConditionExtractor;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.collect.BoundType;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Range;
-
-/**
- *
- * ii test
- */
-public class TsConditionExtractorTest extends LocalFileMetadataTestCase {
-    IIInstance ii;
-    TableRecordInfo tableRecordInfo;
-    TableDesc factTableDesc;
-
-    TblColRef calDt;
-    TblColRef siteId;
-
-    @Before
-    public void setup() throws IOException {
-        this.createTestMetadata();
-        this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii_left_join");
-        if (ii.getFirstSegment() == null) {
-            IISegment segment = IIManager.getInstance(getTestConfig()).buildSegment(ii, 0, System.currentTimeMillis());
-            ii.getSegments().add(segment);
-        }
-        this.tableRecordInfo = new TableRecordInfo(ii.getFirstSegment());
-        this.factTableDesc = MetadataManager.getInstance(getTestConfig()).getTableDesc("DEFAULT.TEST_KYLIN_FACT");
-        this.calDt = this.ii.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "CAL_DT");
-        this.siteId = this.ii.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "LSTG_SITE_ID");
-    }
-
-    @After
-    public void cleanUp() {
-        cleanupTestMetadata();
-    }
-
-    @Test
-    public void testSimpleFilter() {
-        CompareTupleFilter aFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GT);
-        aFilter.addChild(new ColumnTupleFilter(calDt));
-        aFilter.addChild(new ConstantTupleFilter("2000-01-01"));
-
-        Range<Long> range = TsConditionExtractor.extractTsCondition(ii.getAllColumns().get(tableRecordInfo.getTimestampColumn()), aFilter);
-        Assert.assertEquals(946684800000L, range.lowerEndpoint().longValue());
-        Assert.assertEquals(BoundType.OPEN, range.lowerBoundType());
-        Assert.assertTrue(!range.hasUpperBound());
-    }
-
-    @Test
-    public void testComplexFilter() {
-        CompareTupleFilter aFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GT);
-        aFilter.addChild(new ColumnTupleFilter(calDt));
-        aFilter.addChild(new ConstantTupleFilter("2000-01-01"));
-
-        CompareTupleFilter bFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        bFilter.addChild(new ColumnTupleFilter(calDt));
-        bFilter.addChild(new ConstantTupleFilter("2000-01-03"));
-
-        CompareTupleFilter cFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        cFilter.addChild(new ColumnTupleFilter(calDt));
-        cFilter.addChild(new ConstantTupleFilter("2000-01-02"));
-
-        CompareTupleFilter dFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.EQ);
-        dFilter.addChild(new ColumnTupleFilter(siteId));
-        dFilter.addChild(new ConstantTupleFilter("0"));
-
-        LogicalTupleFilter rootFilter = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
-        rootFilter.addChildren(Lists.newArrayList(aFilter, bFilter, cFilter, dFilter));
-
-        Range<Long> range = TsConditionExtractor.extractTsCondition(ii.getAllColumns().get(tableRecordInfo.getTimestampColumn()), rootFilter);
-
-        Assert.assertEquals(946684800000L, range.lowerEndpoint().longValue());
-        Assert.assertEquals(946771200000L, range.upperEndpoint().longValue());
-        Assert.assertEquals(BoundType.OPEN, range.lowerBoundType());
-        Assert.assertEquals(BoundType.CLOSED, range.upperBoundType());
-    }
-
-    @Test
-    public void testMoreComplexFilter() {
-        CompareTupleFilter aFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GT);
-        aFilter.addChild(new ColumnTupleFilter(calDt));
-        aFilter.addChild(new ConstantTupleFilter("2000-01-01"));
-
-        CompareTupleFilter bFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        bFilter.addChild(new ColumnTupleFilter(calDt));
-        bFilter.addChild(new ConstantTupleFilter("2000-01-04"));
-
-        CompareTupleFilter cFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        cFilter.addChild(new ColumnTupleFilter(calDt));
-        cFilter.addChild(new ConstantTupleFilter("2000-01-03"));
-
-        CompareTupleFilter dFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.EQ);
-        dFilter.addChild(new ColumnTupleFilter(siteId));
-        dFilter.addChild(new ConstantTupleFilter("0"));
-
-        LogicalTupleFilter subRoot = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
-        subRoot.addChildren(Lists.newArrayList(aFilter, bFilter, cFilter, dFilter));
-
-        CompareTupleFilter outFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        outFilter.addChild(new ColumnTupleFilter(calDt));
-        outFilter.addChild(new ConstantTupleFilter("2000-01-02"));
-
-        LogicalTupleFilter root = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
-        root.addChildren(Lists.newArrayList(subRoot, outFilter));
-
-        Range<Long> range = TsConditionExtractor.extractTsCondition(ii.getAllColumns().get(tableRecordInfo.getTimestampColumn()), root);
-
-        Assert.assertEquals(946684800000L, range.lowerEndpoint().longValue());
-        Assert.assertEquals(946771200000L, range.upperEndpoint().longValue());
-        Assert.assertEquals(BoundType.OPEN, range.lowerBoundType());
-        Assert.assertEquals(BoundType.CLOSED, range.upperBoundType());
-    }
-
-    @Test
-    public void testComplexConflictFilter() {
-        CompareTupleFilter aFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GT);
-        aFilter.addChild(new ColumnTupleFilter(calDt));
-        aFilter.addChild(new ConstantTupleFilter("2000-01-01"));
-
-        CompareTupleFilter bFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        bFilter.addChild(new ColumnTupleFilter(calDt));
-        bFilter.addChild(new ConstantTupleFilter("1999-01-03"));
-
-        CompareTupleFilter cFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        cFilter.addChild(new ColumnTupleFilter(calDt));
-        cFilter.addChild(new ConstantTupleFilter("2000-01-02"));
-
-        CompareTupleFilter dFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.EQ);
-        dFilter.addChild(new ColumnTupleFilter(siteId));
-        dFilter.addChild(new ConstantTupleFilter("0"));
-
-        LogicalTupleFilter rootFilter = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
-        rootFilter.addChildren(Lists.newArrayList(aFilter, bFilter, cFilter, dFilter));
-
-        Range<Long> range = TsConditionExtractor.extractTsCondition(ii.getAllColumns().get(tableRecordInfo.getTimestampColumn()), rootFilter);
-
-        Assert.assertTrue(range == null);
-
-    }
-
-    @Test
-    public void testMoreComplexConflictFilter() {
-        CompareTupleFilter aFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GT);
-        aFilter.addChild(new ColumnTupleFilter(calDt));
-        aFilter.addChild(new ConstantTupleFilter("2000-01-01"));
-
-        CompareTupleFilter bFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        bFilter.addChild(new ColumnTupleFilter(calDt));
-        bFilter.addChild(new ConstantTupleFilter("2000-01-04"));
-
-        CompareTupleFilter cFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        cFilter.addChild(new ColumnTupleFilter(calDt));
-        cFilter.addChild(new ConstantTupleFilter("2000-01-03"));
-
-        CompareTupleFilter dFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.EQ);
-        dFilter.addChild(new ColumnTupleFilter(siteId));
-        dFilter.addChild(new ConstantTupleFilter("0"));
-
-        LogicalTupleFilter subRoot = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
-        subRoot.addChildren(Lists.newArrayList(aFilter, bFilter, cFilter, dFilter));
-
-        CompareTupleFilter outFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.LTE);
-        outFilter.addChild(new ColumnTupleFilter(calDt));
-        outFilter.addChild(new ConstantTupleFilter("1999-01-02"));
-
-        LogicalTupleFilter root = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
-        root.addChildren(Lists.newArrayList(subRoot, outFilter));
-
-        Range<Long> range = TsConditionExtractor.extractTsCondition(ii.getAllColumns().get(tableRecordInfo.getTimestampColumn()), root);
-
-        Assert.assertTrue(range == null);
-
-    }
-}


[18/55] [abbrv] kylin git commit: KYLIN-1741 Diagnosis logs to kylin.log

Posted by sh...@apache.org.
KYLIN-1741 Diagnosis logs to kylin.log


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 8a34d3c31808a6070a24a465ddd306879c8e70fb
Parents: 3ad49dd
Author: lidongsjtu <li...@apache.org>
Authored: Tue Jul 12 15:03:41 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Tue Jul 12 18:22:08 2016 +0800

----------------------------------------------------------------------
 build/bin/diag.sh                               |  23 ++-
 .../kylin/tool/AbstractInfoExtractor.java       |   2 +
 .../org/apache/kylin/tool/DiagnosisInfoCLI.java |  14 +-
 .../apache/kylin/tool/JobInstanceExtractor.java | 182 +++++++++++++++++++
 4 files changed, 214 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8a34d3c3/build/bin/diag.sh
----------------------------------------------------------------------
diff --git a/build/bin/diag.sh b/build/bin/diag.sh
index 2ddfa80..22805f9 100644
--- a/build/bin/diag.sh
+++ b/build/bin/diag.sh
@@ -24,6 +24,9 @@ dir=$(dirname ${0})
 export KYLIN_HOME=${dir}/../
 source ${dir}/check-env.sh
 
+tomcat_root=${dir}/../tomcat
+export tomcat_root
+
 if [ $# -eq 1 ] || [ $# -eq 2 ]
 then
     patient="$1"
@@ -52,9 +55,9 @@ then
     diagJar=`ls ${KYLIN_HOME}/tool/kylin-tool-*.jar`
     if [ -f "${diagJar}" ]; then
         if [ -f "${KYLIN_HOME}/commit_SHA1" ]; then
-            export HBASE_CLASSPATH=${HBASE_CLASSPATH}:${diagJar}:${KYLIN_HOME}/lib/*
+            export HBASE_CLASSPATH=${HBASE_CLASSPATH}:${diagJar}:${KYLIN_HOME}/lib/*:${KYLIN_HOME}/conf
         else
-            export HBASE_CLASSPATH=${HBASE_CLASSPATH}:${KYLIN_HOME}/lib/*:${diagJar}
+            export HBASE_CLASSPATH=${HBASE_CLASSPATH}:${KYLIN_HOME}/lib/*:${diagJar}:${KYLIN_HOME}/conf
         fi
     else
         echo "missing diagnosis jar file."
@@ -62,13 +65,23 @@ then
     fi
 
     if [ ${#patient} -eq 36 ]; then
-        exec hbase ${KYLIN_EXTRA_START_OPTS} -Dlog4j.configuration=kylin-log4j.properties org.apache.kylin.tool.JobDiagnosisInfoCLI -jobId $patient -destDir $destDir
+        hbase ${KYLIN_EXTRA_START_OPTS} \
+        -Dlog4j.configuration=kylin-server-log4j.properties \
+        -Dcatalina.home=${tomcat_root} \
+        org.apache.kylin.tool.JobDiagnosisInfoCLI \
+        -jobId $patient \
+        -destDir $destDir
     else
-        exec hbase ${KYLIN_EXTRA_START_OPTS} -Dlog4j.configuration=kylin-log4j.properties org.apache.kylin.tool.DiagnosisInfoCLI -project -all -destDir $destDir
+        hbase ${KYLIN_EXTRA_START_OPTS} \
+        -Dlog4j.configuration=kylin-server-log4j.properties \
+        -Dcatalina.home=${tomcat_root} \
+        org.apache.kylin.tool.DiagnosisInfoCLI \
+        -project -all \
+        -destDir $destDir
     fi
 
     exit 0
 else
     echo "usage: diag.sh Project|JobId [target_path]"
     exit 1
-fi
+fi
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/8a34d3c3/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java b/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java
index a3bf24a..f2d6102 100644
--- a/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java
+++ b/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java
@@ -90,7 +90,9 @@ public abstract class AbstractInfoExtractor extends AbstractApplication {
         if (!isSubmodule && new File(exportDest).exists()) {
             exportDest = exportDest + packageName + "/";
         }
+
         exportDir = new File(exportDest);
+        FileUtils.forceMkdir(exportDir);
 
         if (!isSubmodule) {
             dumpBasicDiagInfo();

http://git-wip-us.apache.org/repos/asf/kylin/blob/8a34d3c3/tool/src/main/java/org/apache/kylin/tool/DiagnosisInfoCLI.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/DiagnosisInfoCLI.java b/tool/src/main/java/org/apache/kylin/tool/DiagnosisInfoCLI.java
index 495cc5b..cf563a5 100644
--- a/tool/src/main/java/org/apache/kylin/tool/DiagnosisInfoCLI.java
+++ b/tool/src/main/java/org/apache/kylin/tool/DiagnosisInfoCLI.java
@@ -53,7 +53,7 @@ public class DiagnosisInfoCLI extends AbstractInfoExtractor {
     private static final Option OPTION_INCLUDE_CLIENT = OptionBuilder.withArgName("includeClient").hasArg().isRequired(false).withDescription("Specify whether to include client info to extract. Default true.").create("includeClient");
 
     @SuppressWarnings("static-access")
-    private static final Option OPTION_INCLUDE_JOB = OptionBuilder.withArgName("includeJobs").hasArg().isRequired(false).withDescription("Specify whether to include job output to extract. Default true.").create("includeJobs");
+    private static final Option OPTION_INCLUDE_JOB = OptionBuilder.withArgName("includeJobs").hasArg().isRequired(false).withDescription("Specify whether to include job info to extract. Default true.").create("includeJobs");
 
     public DiagnosisInfoCLI() {
         super();
@@ -82,6 +82,9 @@ public class DiagnosisInfoCLI extends AbstractInfoExtractor {
         } else {
             result.add(projectSeed);
         }
+        if (result.isEmpty()) {
+            throw new RuntimeException("No project to extract.");
+        }
         return result;
     }
 
@@ -95,11 +98,18 @@ public class DiagnosisInfoCLI extends AbstractInfoExtractor {
         String projectNames = StringUtils.join(getProjects(projectInput), ",");
 
         // export cube metadata
-        String[] cubeMetaArgs = { "-destDir", new File(exportDir, "metadata").getAbsolutePath(), "-project", projectNames, "-compress", "false", "-includeJobs", Boolean.toString(includeJob), "-submodule", "true" };
+        String[] cubeMetaArgs = { "-destDir", new File(exportDir, "metadata").getAbsolutePath(), "-project", projectNames, "-compress", "false", "-includeJobs", "false", "-submodule", "true" };
         CubeMetaExtractor cubeMetaExtractor = new CubeMetaExtractor();
         logger.info("CubeMetaExtractor args: " + Arrays.toString(cubeMetaArgs));
         cubeMetaExtractor.execute(cubeMetaArgs);
 
+        // extract all job instances
+        if (includeJob) {
+            String[] jobArgs = { "-destDir", new File(exportDir, "jobs").getAbsolutePath(), "-compress", "false", "-submodule", "true" };
+            JobInstanceExtractor jobInstanceExtractor = new JobInstanceExtractor();
+            jobInstanceExtractor.execute(jobArgs);
+        }
+
         // export HBase
         if (includeHBase) {
             String[] hbaseArgs = { "-destDir", new File(exportDir, "hbase").getAbsolutePath(), "-project", projectNames, "-compress", "false", "-submodule", "true" };

http://git-wip-us.apache.org/repos/asf/kylin/blob/8a34d3c3/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java b/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
new file mode 100644
index 0000000..5ad4953
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
@@ -0,0 +1,182 @@
+package org.apache.kylin.tool;
+
+import java.io.File;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.OptionsHelper;
+import org.apache.kylin.cube.model.CubeBuildTypeEnum;
+import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.job.JobInstance;
+import org.apache.kylin.job.common.ShellExecutable;
+import org.apache.kylin.job.constant.JobStatusEnum;
+import org.apache.kylin.job.constant.JobStepStatusEnum;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableState;
+import org.apache.kylin.job.execution.Output;
+import org.apache.kylin.job.manager.ExecutableManager;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.metadata.project.ProjectManager;
+import org.apache.kylin.metadata.realization.RealizationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class JobInstanceExtractor extends AbstractInfoExtractor {
+    private static final Logger logger = LoggerFactory.getLogger(JobInstanceExtractor.class);
+
+    @SuppressWarnings("static-access")
+    private static final Option OPTION_PROJECT = OptionBuilder.withArgName("project").hasArg().isRequired(false).withDescription("Specify jobs in which project to extract").create("project");
+    @SuppressWarnings("static-access")
+    private static final Option OPTION_CUBE = OptionBuilder.withArgName("cube").hasArg().isRequired(false).withDescription("Specify jobs related to which cube to extract").create("cube");
+
+    KylinConfig config;
+    ProjectManager projectManager;
+    ExecutableManager executableManager;
+
+    public JobInstanceExtractor() {
+        config = KylinConfig.getInstanceFromEnv();
+        executableManager = ExecutableManager.getInstance(config);
+        projectManager = ProjectManager.getInstance(config);
+
+        packageType = "jobInstances";
+
+        options.addOption(OPTION_PROJECT);
+        options.addOption(OPTION_CUBE);
+    }
+
+    @Override
+    protected void executeExtract(OptionsHelper optionsHelper, File exportDir) throws Exception {
+        String cube = optionsHelper.hasOption(OPTION_CUBE) ? optionsHelper.getOptionValue(OPTION_CUBE) : null;
+        String project = optionsHelper.hasOption(OPTION_PROJECT) ? optionsHelper.getOptionValue(OPTION_PROJECT) : null;
+
+        long endTime = System.currentTimeMillis();
+        long startTime = endTime - 3 * 24 * 3600 * 1000;
+        List<JobInstance> jobInstances = listJobInstances(cube, project, startTime, endTime);
+        logger.info("There are {} jobInstances to extract.", jobInstances.size());
+
+        ObjectMapper mapper = new ObjectMapper();
+        for (JobInstance jobInstance : jobInstances) {
+            mapper.writeValue(new File(exportDir, jobInstance.getUuid() + ".json"), jobInstance);
+        }
+    }
+
+    private List<JobInstance> listJobInstances(String project, String cube, long startTime, long endTime) {
+        final List<JobInstance> result = Lists.newArrayList();
+        final List<AbstractExecutable> executables = executableManager.getAllExecutables(startTime, endTime);
+        final Map<String, Output> allOutputs = executableManager.getAllOutputs();
+        for (AbstractExecutable executable : executables) {
+            if (executable instanceof CubingJob) {
+                String cubeName = CubingExecutableUtil.getCubeName(executable.getParams());
+                boolean shouldExtract = false;
+                if (cube == null || cube.equalsIgnoreCase(cubeName)) {
+                    if (project == null) {
+                        shouldExtract = true;
+                    } else {
+                        ProjectInstance projectInstance = projectManager.getProject(project);
+                        if (projectInstance != null && projectInstance.containsRealization(RealizationType.CUBE, cubeName)) {
+                            shouldExtract = true;
+                        }
+                    }
+                }
+
+                if (shouldExtract) {
+                    result.add(parseToJobInstance((CubingJob) executable, allOutputs));
+                }
+            }
+        }
+        return result;
+    }
+
+    private JobInstance parseToJobInstance(CubingJob cubeJob, Map<String, Output> outputs) {
+        Output output = outputs.get(cubeJob.getId());
+        final JobInstance result = new JobInstance();
+        result.setName(cubeJob.getName());
+        result.setRelatedCube(CubingExecutableUtil.getCubeName(cubeJob.getParams()));
+        result.setRelatedSegment(CubingExecutableUtil.getSegmentId(cubeJob.getParams()));
+        result.setLastModified(output.getLastModified());
+        result.setSubmitter(cubeJob.getSubmitter());
+        result.setUuid(cubeJob.getId());
+        result.setType(CubeBuildTypeEnum.BUILD);
+        result.setStatus(parseToJobStatus(output.getState()));
+        result.setMrWaiting(AbstractExecutable.getExtraInfoAsLong(output, CubingJob.MAP_REDUCE_WAIT_TIME, 0L) / 1000);
+        result.setDuration(AbstractExecutable.getDuration(AbstractExecutable.getStartTime(output), AbstractExecutable.getEndTime(output)) / 1000);
+        for (int i = 0; i < cubeJob.getTasks().size(); ++i) {
+            AbstractExecutable task = cubeJob.getTasks().get(i);
+            result.addStep(parseToJobStep(task, i, outputs.get(task.getId())));
+        }
+        return result;
+    }
+
+    private JobStatusEnum parseToJobStatus(ExecutableState state) {
+        switch (state) {
+        case READY:
+            return JobStatusEnum.PENDING;
+        case RUNNING:
+            return JobStatusEnum.RUNNING;
+        case ERROR:
+            return JobStatusEnum.ERROR;
+        case DISCARDED:
+            return JobStatusEnum.DISCARDED;
+        case SUCCEED:
+            return JobStatusEnum.FINISHED;
+        case STOPPED:
+        default:
+            throw new RuntimeException("invalid state:" + state);
+        }
+    }
+
+    private JobInstance.JobStep parseToJobStep(AbstractExecutable task, int i, Output stepOutput) {
+        Preconditions.checkNotNull(stepOutput);
+        JobInstance.JobStep result = new JobInstance.JobStep();
+        result.setId(task.getId());
+        result.setName(task.getName());
+        result.setSequenceID(i);
+        result.setStatus(parseToJobStepStatus(stepOutput.getState()));
+        for (Map.Entry<String, String> entry : stepOutput.getExtra().entrySet()) {
+            if (entry.getKey() != null && entry.getValue() != null) {
+                result.putInfo(entry.getKey(), entry.getValue());
+            }
+        }
+        result.setExecStartTime(AbstractExecutable.getStartTime(stepOutput));
+        result.setExecEndTime(AbstractExecutable.getEndTime(stepOutput));
+        if (task instanceof ShellExecutable) {
+            result.setExecCmd(((ShellExecutable) task).getCmd());
+        }
+        if (task instanceof MapReduceExecutable) {
+            result.setExecCmd(((MapReduceExecutable) task).getMapReduceParams());
+            result.setExecWaitTime(AbstractExecutable.getExtraInfoAsLong(stepOutput, MapReduceExecutable.MAP_REDUCE_WAIT_TIME, 0L) / 1000);
+        }
+        if (task instanceof HadoopShellExecutable) {
+            result.setExecCmd(((HadoopShellExecutable) task).getJobParams());
+        }
+        return result;
+    }
+
+    private JobStepStatusEnum parseToJobStepStatus(ExecutableState state) {
+        switch (state) {
+        case READY:
+            return JobStepStatusEnum.PENDING;
+        case RUNNING:
+            return JobStepStatusEnum.RUNNING;
+        case ERROR:
+            return JobStepStatusEnum.ERROR;
+        case DISCARDED:
+            return JobStepStatusEnum.DISCARDED;
+        case SUCCEED:
+            return JobStepStatusEnum.FINISHED;
+        case STOPPED:
+        default:
+            throw new RuntimeException("invalid state:" + state);
+        }
+    }
+}


[42/55] [abbrv] kylin git commit: KYLIN-1918 UnsupportedOperationException when unload hive table

Posted by sh...@apache.org.
KYLIN-1918 UnsupportedOperationException when unload hive table

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 543e9f90018fdca1c03720a0cc863ab2d15727aa
Parents: 3e553c0
Author: shaofengshi <sh...@apache.org>
Authored: Sat Jul 23 19:03:04 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sat Jul 23 19:03:04 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/rest/controller/TableController.java   | 4 ++--
 .../main/java/org/apache/kylin/rest/service/CacheService.java    | 3 ++-
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/543e9f90/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
index 1c3377b..95df27d 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -169,8 +169,8 @@ public class TableController extends BasicController {
                 unLoadFail.add(tableName);
             }
         }
-        result.put("result.unload.success", (String[]) unLoadSuccess.toArray(new String[unLoadSuccess.size()]));
-        result.put("result.unload.fail", (String[]) unLoadFail.toArray(new String[unLoadFail.size()]));
+        result.put("result.unload.success", unLoadSuccess.toArray(new String[unLoadSuccess.size()]));
+        result.put("result.unload.fail", unLoadFail.toArray(new String[unLoadFail.size()]));
         return result;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/543e9f90/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
index 02efe70..7b1e2b7 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
@@ -244,7 +244,8 @@ public class CacheService extends BasicService {
                 ProjectManager.clearCache();
                 break;
             case TABLE:
-                throw new UnsupportedOperationException(log);
+                MetadataManager.clearCache();
+                break;
             case EXTERNAL_FILTER:
                 throw new UnsupportedOperationException(log);
             case DATA_MODEL:


[14/55] [abbrv] kylin git commit: KYLIN-1741 Enhance diagnosis tool

Posted by sh...@apache.org.
KYLIN-1741 Enhance diagnosis tool


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 8735c28f455a074f996d70abadd52c343dc3ae61
Parents: e0ffd22
Author: lidongsjtu <li...@apache.org>
Authored: Sun Jul 10 16:25:23 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Sun Jul 10 16:32:12 2016 +0800

----------------------------------------------------------------------
 .../kylin/tool/AbstractInfoExtractor.java       |  1 +
 .../org/apache/kylin/tool/util/ToolUtil.java    | 25 ++++++++++++++++++++
 2 files changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8735c28f/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java b/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java
index cb2ebd0..a3bf24a 100644
--- a/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java
+++ b/tool/src/main/java/org/apache/kylin/tool/AbstractInfoExtractor.java
@@ -136,6 +136,7 @@ public abstract class AbstractInfoExtractor extends AbstractApplication {
         StringBuilder basicSb = new StringBuilder();
         basicSb.append("MetaStoreID: ").append(ToolUtil.getHBaseMetaStoreId()).append("\n");
         basicSb.append("PackageType: ").append(packageType.toUpperCase()).append("\n");
+        basicSb.append("Host: ").append(ToolUtil.getHostName()).append("\n");
         FileUtils.writeStringToFile(new File(exportDir, "info"), basicSb.toString());
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8735c28f/tool/src/main/java/org/apache/kylin/tool/util/ToolUtil.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/util/ToolUtil.java b/tool/src/main/java/org/apache/kylin/tool/util/ToolUtil.java
index 08f21e5..1312ca4 100644
--- a/tool/src/main/java/org/apache/kylin/tool/util/ToolUtil.java
+++ b/tool/src/main/java/org/apache/kylin/tool/util/ToolUtil.java
@@ -21,6 +21,8 @@ package org.apache.kylin.tool.util;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
@@ -70,4 +72,27 @@ public class ToolUtil {
         return null;
     }
 
+    public static String getHostName() {
+        String hostname = System.getenv("COMPUTERNAME");
+        if (StringUtils.isEmpty(hostname)) {
+            InetAddress address = null;
+            try {
+                address = InetAddress.getLocalHost();
+                hostname = address.getHostName();
+                if (StringUtils.isEmpty(hostname)) {
+                    hostname = address.getHostAddress();
+                }
+            } catch (UnknownHostException uhe) {
+                String host = uhe.getMessage(); // host = "hostname: hostname"
+                if (host != null) {
+                    int colon = host.indexOf(':');
+                    if (colon > 0) {
+                        return host.substring(0, colon);
+                    }
+                }
+                hostname = "Unknown";
+            }
+        }
+        return hostname;
+    }
 }


[30/55] [abbrv] kylin git commit: minor, fix create count source table error when using beeline

Posted by sh...@apache.org.
minor, fix create count source table error when using beeline

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


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 55f5ea3ff981d4f051cbcfa53db6bc54b97aab0d
Parents: 8378577
Author: Yiming Liu <li...@gmail.com>
Authored: Tue Jul 19 10:59:57 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Jul 19 11:02:47 2016 +0800

----------------------------------------------------------------------
 core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/55f5ea3f/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java b/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
index 31c92b6..1b8df14 100644
--- a/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
@@ -137,8 +137,8 @@ public class JoinedFlatTable {
         final Map<String, String> tableAliasMap = buildTableAliasMap(intermediateTableDesc.getDataModel());
         final StringBuilder sql = new StringBuilder();
         final String factTbl = intermediateTableDesc.getDataModel().getFactTable();
-        sql.append("dfs -mkdir -p " + outputDir + ";");
-        sql.append("INSERT OVERWRITE DIRECTORY '" + outputDir + "' SELECT count(*) from " + factTbl + " " + tableAliasMap.get(factTbl) + "\n");
+        sql.append("dfs -mkdir -p " + outputDir + ";\n");
+        sql.append("INSERT OVERWRITE DIRECTORY '" + outputDir + "' SELECT count(*) FROM " + factTbl + " " + tableAliasMap.get(factTbl) + "\n");
         appendWhereStatement(intermediateTableDesc, sql, tableAliasMap);
         return sql.toString();
     }


[54/55] [abbrv] kylin git commit: KYLIN-1672 support kylin on cdh 5.7

Posted by sh...@apache.org.
KYLIN-1672 support kylin on cdh 5.7

Signed-off-by: Li Yang <li...@apache.org>


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 80283a7853332051631696f4d25fdc61a068ff98
Parents: 7d9a59b
Author: Lynne Jiang <ly...@hotmail.com>
Authored: Mon May 16 03:33:27 2016 -0700
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Jul 26 09:49:00 2016 +0800

----------------------------------------------------------------------
 dev-support/test_all_against_hdp_2_2_4_2_2.sh   |   0
 .../kylin/engine/mr/steps/MockupMapContext.java |  15 +-
 examples/test_case_data/sandbox/core-site.xml   | 146 +++---
 examples/test_case_data/sandbox/hbase-site.xml  | 162 ++----
 examples/test_case_data/sandbox/hdfs-site.xml   | 259 ++--------
 examples/test_case_data/sandbox/mapred-site.xml | 396 ++++++---------
 examples/test_case_data/sandbox/yarn-site.xml   | 496 ++-----------------
 pom.xml                                         |  18 +-
 server/pom.xml                                  |  36 ++
 .../storage/hbase/steps/MockupMapContext.java   |  19 +-
 10 files changed, 418 insertions(+), 1129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/dev-support/test_all_against_hdp_2_2_4_2_2.sh
----------------------------------------------------------------------
diff --git a/dev-support/test_all_against_hdp_2_2_4_2_2.sh b/dev-support/test_all_against_hdp_2_2_4_2_2.sh
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
index 847071d..9900465 100644
--- a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
@@ -77,6 +77,7 @@ public class MockupMapContext {
                     outKV[0] = key;
                     outKV[1] = value;
                 }
+
             }
 
             @Override
@@ -99,6 +100,7 @@ public class MockupMapContext {
                 throw new NotImplementedException();
             }
 
+
             @Override
             public float getProgress() {
                 throw new NotImplementedException();
@@ -195,17 +197,17 @@ public class MockupMapContext {
             }
 
             @Override
-            public RawComparator<?> getSortComparator() {
+            public boolean userClassesTakesPrecedence() {
                 throw new NotImplementedException();
             }
 
             @Override
-            public String getJar() {
+            public RawComparator<?> getSortComparator() {
                 throw new NotImplementedException();
             }
 
             @Override
-            public RawComparator<?> getGroupingComparator() {
+            public String getJar() {
                 throw new NotImplementedException();
             }
 
@@ -221,7 +223,7 @@ public class MockupMapContext {
 
             @Override
             public boolean getProfileEnabled() {
-                throw new NotImplementedException();
+                return false;
             }
 
             @Override
@@ -308,6 +310,11 @@ public class MockupMapContext {
             public RawComparator<?> getCombinerKeyGroupingComparator() {
                 throw new NotImplementedException();
             }
+
+            @Override
+            public RawComparator<?> getGroupingComparator() {
+                return null;
+            }
         });
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/examples/test_case_data/sandbox/core-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/core-site.xml b/examples/test_case_data/sandbox/core-site.xml
index 9aa588c..6162406 100644
--- a/examples/test_case_data/sandbox/core-site.xml
+++ b/examples/test_case_data/sandbox/core-site.xml
@@ -14,152 +14,146 @@
   See the License for the specific language governing permissions and
   limitations under the License.
 -->
+<!--Autogenerated by Cloudera Manager-->
 <configuration>
-
     <property>
         <name>fs.defaultFS</name>
-        <value>hdfs://sandbox.hortonworks.com:8020</value>
-        <final>true</final>
+        <value>hdfs://quickstart.cloudera:8020</value>
     </property>
-
     <property>
         <name>fs.trash.interval</name>
-        <value>360</value>
+        <value>1</value>
     </property>
-
     <property>
-        <name>ha.failover-controller.active-standby-elector.zk.op.retries</name>
-        <value>120</value>
+        <name>io.compression.codecs</name>
+        <value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.DeflateCodec,org.apache.hadoop.io.compress.SnappyCodec,org.apache.hadoop.io.compress.Lz4Codec</value>
     </property>
-
     <property>
-        <name>hadoop.http.authentication.simple.anonymous.allowed</name>
-        <value>true</value>
+        <name>hadoop.security.authentication</name>
+        <value>simple</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.falcon.groups</name>
-        <value>users</value>
+        <name>hadoop.security.authorization</name>
+        <value>false</value>
+    </property>
+    <property>
+        <name>hadoop.rpc.protection</name>
+        <value>authentication</value>
+    </property>
+    <property>
+        <name>hadoop.security.auth_to_local</name>
+        <value>DEFAULT</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.falcon.hosts</name>
+        <name>hadoop.proxyuser.oozie.hosts</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hbase.groups</name>
-        <value>users</value>
+        <name>hadoop.proxyuser.oozie.groups</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hbase.hosts</name>
+        <name>hadoop.proxyuser.mapred.hosts</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hcat.groups</name>
+        <name>hadoop.proxyuser.mapred.groups</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hcat.hosts</name>
-        <value>sandbox.hortonworks.com</value>
+        <name>hadoop.proxyuser.flume.hosts</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hive.groups</name>
-        <value>users</value>
+        <name>hadoop.proxyuser.flume.groups</name>
+        <value>*</value>
+    </property>
+    <property>
+        <name>hadoop.proxyuser.HTTP.hosts</name>
+        <value>*</value>
+    </property>
+    <property>
+        <name>hadoop.proxyuser.HTTP.groups</name>
+        <value>*</value>
     </property>
-
     <property>
         <name>hadoop.proxyuser.hive.hosts</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hue.groups</name>
+        <name>hadoop.proxyuser.hive.groups</name>
         <value>*</value>
     </property>
-
     <property>
         <name>hadoop.proxyuser.hue.hosts</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.oozie.groups</name>
+        <name>hadoop.proxyuser.hue.groups</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.oozie.hosts</name>
-        <value>sandbox.hortonworks.com</value>
+        <name>hadoop.proxyuser.httpfs.hosts</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.root.groups</name>
+        <name>hadoop.proxyuser.httpfs.groups</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.root.hosts</name>
+        <name>hadoop.proxyuser.hdfs.groups</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.security.auth_to_local</name>
-        <value>DEFAULT</value>
+        <name>hadoop.proxyuser.hdfs.hosts</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.security.authentication</name>
-        <value>simple</value>
+        <name>hadoop.proxyuser.yarn.hosts</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.security.authorization</name>
-        <value>false</value>
+        <name>hadoop.proxyuser.yarn.groups</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>io.compression.codecs</name>
-        <value>org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.SnappyCodec</value>
+        <name>hadoop.security.group.mapping</name>
+        <value>org.apache.hadoop.security.ShellBasedUnixGroupsMapping</value>
     </property>
-
     <property>
-        <name>io.file.buffer.size</name>
-        <value>131072</value>
+        <name>hadoop.security.instrumentation.requires.admin</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>io.serializations</name>
-        <value>org.apache.hadoop.io.serializer.WritableSerialization</value>
+        <name>net.topology.script.file.name</name>
+        <value>/etc/hadoop/conf.cloudera.yarn/topology.py</value>
     </property>
-
     <property>
-        <name>ipc.client.connect.max.retries</name>
-        <value>50</value>
+        <name>io.file.buffer.size</name>
+        <value>65536</value>
     </property>
-
     <property>
-        <name>ipc.client.connection.maxidletime</name>
-        <value>30000</value>
+        <name>hadoop.ssl.enabled</name>
+        <value>false</value>
+    </property>
+    <property>
+        <name>hadoop.ssl.require.client.cert</name>
+        <value>false</value>
+        <final>true</final>
     </property>
-
     <property>
-        <name>ipc.client.idlethreshold</name>
-        <value>8000</value>
+        <name>hadoop.ssl.keystores.factory.class</name>
+        <value>org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory</value>
+        <final>true</final>
     </property>
-
     <property>
-        <name>ipc.server.tcpnodelay</name>
-        <value>true</value>
+        <name>hadoop.ssl.server.conf</name>
+        <value>ssl-server.xml</value>
+        <final>true</final>
     </property>
-
     <property>
-        <name>mapreduce.jobtracker.webinterface.trusted</name>
-        <value>false</value>
+        <name>hadoop.ssl.client.conf</name>
+        <value>ssl-client.xml</value>
+        <final>true</final>
     </property>
-
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/examples/test_case_data/sandbox/hbase-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hbase-site.xml b/examples/test_case_data/sandbox/hbase-site.xml
index 734908e..58c6223 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -15,180 +15,104 @@
   limitations under the License.
 -->
 <configuration>
-
-    <property>
-        <name>dfs.domain.socket.path</name>
-        <value>/var/lib/hadoop-hdfs/dn_socket</value>
-    </property>
-
     <property>
-        <name>hbase.client.keyvalue.maxsize</name>
-        <value>10485760</value>
-    </property>
-
-    <property>
-        <name>hbase.client.scanner.caching</name>
-        <value>100</value>
+        <name>hbase.rootdir</name>
+        <value>hdfs://quickstart.cloudera:8020/hbase</value>
     </property>
-
     <property>
-        <name>hbase.cluster.distributed</name>
+        <name>hbase.replication</name>
         <value>true</value>
     </property>
-
-    <property>
-        <name>hbase.coprocessor.master.classes</name>
-        <value>com.xasecure.authorization.hbase.XaSecureAuthorizationCoprocessor</value>
-    </property>
-
     <property>
-        <name>hbase.coprocessor.region.classes</name>
-        <value>com.xasecure.authorization.hbase.XaSecureAuthorizationCoprocessor</value>
+        <name>hbase.client.write.buffer</name>
+        <value>2097152</value>
     </property>
-
-    <property>
-        <name>hbase.defaults.for.version.skip</name>
-        <value>true</value>
-    </property>
-
     <property>
-        <name>hbase.hregion.majorcompaction</name>
-        <value>604800000</value>
+        <name>hbase.client.pause</name>
+        <value>100</value>
     </property>
-
     <property>
-        <name>hbase.hregion.majorcompaction.jitter</name>
-        <value>0.50</value>
+        <name>hbase.client.retries.number</name>
+        <value>35</value>
     </property>
-
     <property>
-        <name>hbase.hregion.max.filesize</name>
-        <value>10737418240</value>
+        <name>hbase.client.scanner.caching</name>
+        <value>100</value>
     </property>
-
     <property>
-        <name>hbase.hregion.memstore.block.multiplier</name>
-        <value>4</value>
+        <name>hbase.client.keyvalue.maxsize</name>
+        <value>10485760</value>
     </property>
-
     <property>
-        <name>hbase.hregion.memstore.flush.size</name>
-        <value>134217728</value>
-    </property>
-
-    <property>
-        <name>hbase.hregion.memstore.mslab.enabled</name>
+        <name>hbase.ipc.client.allowsInterrupt</name>
         <value>true</value>
     </property>
-
     <property>
-        <name>hbase.hstore.blockingStoreFiles</name>
+        <name>hbase.client.primaryCallTimeout.get</name>
         <value>10</value>
     </property>
-
-    <property>
-        <name>hbase.hstore.compactionThreshold</name>
-        <value>3</value>
-    </property>
-
     <property>
-        <name>hbase.local.dir</name>
-        <value>${hbase.tmp.dir}/local</value>
+        <name>hbase.client.primaryCallTimeout.multiget</name>
+        <value>10</value>
     </property>
-
     <property>
-        <name>hbase.master.info.bindAddress</name>
-        <value>0.0.0.0</value>
+        <name>hbase.regionserver.thrift.http</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>hbase.master.info.port</name>
-        <value>60010</value>
+        <name>hbase.thrift.support.proxyuser</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>hbase.master.port</name>
+        <name>hbase.rpc.timeout</name>
         <value>60000</value>
     </property>
-
-    <property>
-        <name>hbase.regionserver.global.memstore.lowerLimit</name>
-        <value>0.38</value>
-    </property>
-
     <property>
-        <name>hbase.regionserver.global.memstore.upperLimit</name>
-        <value>0.4</value>
-    </property>
-
-    <property>
-        <name>hbase.regionserver.handler.count</name>
-        <value>60</value>
+        <name>hbase.snapshot.enabled</name>
+        <value>true</value>
     </property>
-
     <property>
-        <name>hbase.regionserver.info.port</name>
-        <value>60030</value>
+        <name>hbase.snapshot.master.timeoutMillis</name>
+        <value>60000</value>
     </property>
-
     <property>
-        <name>hbase.rootdir</name>
-        <value>hdfs://sandbox.hortonworks.com:8020/apps/hbase/data</value>
+        <name>hbase.snapshot.region.timeout</name>
+        <value>60000</value>
     </property>
-
     <property>
-        <name>hbase.rpc.protection</name>
-        <value>PRIVACY</value>
+        <name>hbase.snapshot.master.timeout.millis</name>
+        <value>60000</value>
     </property>
-
     <property>
         <name>hbase.security.authentication</name>
         <value>simple</value>
     </property>
-
     <property>
-        <name>hbase.security.authorization</name>
-        <value>true</value>
+        <name>hbase.rpc.protection</name>
+        <value>authentication</value>
     </property>
-
     <property>
-        <name>hbase.superuser</name>
-        <value>hbase</value>
+        <name>zookeeper.session.timeout</name>
+        <value>60000</value>
     </property>
-
     <property>
-        <name>hbase.tmp.dir</name>
-        <value>/hadoop/hbase</value>
+        <name>zookeeper.znode.parent</name>
+        <value>/hbase</value>
     </property>
-
     <property>
-        <name>hbase.zookeeper.property.clientPort</name>
-        <value>2181</value>
+        <name>zookeeper.znode.rootserver</name>
+        <value>root-region-server</value>
     </property>
-
     <property>
         <name>hbase.zookeeper.quorum</name>
-        <value>sandbox.hortonworks.com</value>
+        <value>quickstart.cloudera</value>
     </property>
-
     <property>
-        <name>hbase.zookeeper.useMulti</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>hfile.block.cache.size</name>
-        <value>0.40</value>
-    </property>
-
-    <property>
-        <name>zookeeper.session.timeout</name>
-        <value>30000</value>
+        <name>hbase.zookeeper.property.clientPort</name>
+        <value>2181</value>
     </property>
-
     <property>
-        <name>zookeeper.znode.parent</name>
-        <value>/hbase-unsecure</value>
+        <name>hbase.rest.ssl.enabled</name>
+        <value>false</value>
     </property>
-
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/examples/test_case_data/sandbox/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hdfs-site.xml b/examples/test_case_data/sandbox/hdfs-site.xml
index 1175fff..05854bd 100644
--- a/examples/test_case_data/sandbox/hdfs-site.xml
+++ b/examples/test_case_data/sandbox/hdfs-site.xml
@@ -15,271 +15,68 @@
   limitations under the License.
 -->
 <configuration>
-
-    <property>
-        <name>dfs.block.access.token.enable</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>dfs.block.size</name>
-        <value>34217472</value>
-    </property>
-
-    <property>
-        <name>dfs.blockreport.initialDelay</name>
-        <value>120</value>
-    </property>
-
-    <property>
-        <name>dfs.blocksize</name>
-        <value>134217728</value>
-    </property>
-
-    <property>
-        <name>dfs.client.read.shortcircuit</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.client.read.shortcircuit.streams.cache.size</name>
-        <value>4096</value>
-    </property>
-
-    <property>
-        <name>dfs.cluster.administrators</name>
-        <value>hdfs</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.address</name>
-        <value>0.0.0.0:50010</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.balance.bandwidthPerSec</name>
-        <value>6250000</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.data.dir</name>
-        <value>/hadoop/hdfs/data</value>
-        <final>true</final>
-    </property>
-
-    <property>
-        <name>dfs.datanode.data.dir.perm</name>
-        <value>750</value>
-    </property>
-
     <property>
-        <name>dfs.datanode.du.reserved</name>
-        <value>1073741824</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.failed.volumes.tolerated</name>
-        <value>0</value>
-        <final>true</final>
-    </property>
-
-    <property>
-        <name>dfs.datanode.http.address</name>
-        <value>0.0.0.0:50075</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.https.address</name>
-        <value>0.0.0.0:50475</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.ipc.address</name>
-        <value>0.0.0.0:8010</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.max.transfer.threads</name>
-        <value>1024</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.max.xcievers</name>
-        <value>1024</value>
-    </property>
-
-    <property>
-        <name>dfs.domain.socket.path</name>
-        <value>/var/lib/hadoop-hdfs/dn_socket</value>
-    </property>
-
-    <property>
-        <name>dfs.heartbeat.interval</name>
-        <value>3</value>
+        <name>dfs.namenode.name.dir</name>
+        <value>file:///var/lib/hadoop-hdfs/cache/hdfs/dfs/name</value>
     </property>
-
     <property>
-        <name>dfs.hosts.exclude</name>
-        <value>/etc/hadoop/conf/dfs.exclude</value>
+        <name>dfs.namenode.servicerpc-address</name>
+        <value>quickstart.cloudera:8022</value>
     </property>
-
     <property>
-        <name>dfs.http.policy</name>
-        <value>HTTP_ONLY</value>
+        <name>dfs.https.address</name>
+        <value>quickstart.cloudera:50470</value>
     </property>
-
     <property>
         <name>dfs.https.port</name>
         <value>50470</value>
     </property>
-
-    <property>
-        <name>dfs.journalnode.edits.dir</name>
-        <value>/hadoop/hdfs/journalnode</value>
-    </property>
-
-    <property>
-        <name>dfs.journalnode.http-address</name>
-        <value>0.0.0.0:8480</value>
-    </property>
-
-    <property>
-        <name>dfs.journalnode.https-address</name>
-        <value>0.0.0.0:8481</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.accesstime.precision</name>
-        <value>3600000</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.avoid.read.stale.datanode</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.avoid.write.stale.datanode</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.checkpoint.dir</name>
-        <value>/hadoop/hdfs/namesecondary</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.checkpoint.edits.dir</name>
-        <value>${dfs.namenode.checkpoint.dir}</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.checkpoint.period</name>
-        <value>21600</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.checkpoint.txns</name>
-        <value>1000000</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.handler.count</name>
-        <value>100</value>
-    </property>
-
     <property>
         <name>dfs.namenode.http-address</name>
-        <value>sandbox.hortonworks.com:50070</value>
-        <final>true</final>
-    </property>
-
-    <property>
-        <name>dfs.namenode.https-address</name>
-        <value>sandbox.hortonworks.com:50470</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.name.dir</name>
-        <value>/hadoop/hdfs/namenode</value>
-        <final>true</final>
-    </property>
-
-    <property>
-        <name>dfs.namenode.name.dir.restore</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.safemode.threshold-pct</name>
-        <value>1.0f</value>
+        <value>quickstart.cloudera:50070</value>
     </property>
-
     <property>
-        <name>dfs.namenode.secondary.http-address</name>
-        <value>sandbox.hortonworks.com:50090</value>
+        <name>dfs.replication</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>dfs.namenode.stale.datanode.interval</name>
-        <value>30000</value>
+        <name>dfs.blocksize</name>
+        <value>134217728</value>
     </property>
-
     <property>
-        <name>dfs.namenode.startup.delay.block.deletion.sec</name>
-        <value>3600</value>
+        <name>dfs.client.use.datanode.hostname</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.namenode.write.stale.datanode.ratio</name>
-        <value>1.0f</value>
+        <name>fs.permissions.umask-mode</name>
+        <value>022</value>
     </property>
-
     <property>
-        <name>dfs.nfs.exports.allowed.hosts</name>
-        <value>* rw</value>
+        <name>dfs.namenode.acls.enabled</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.nfs3.dump.dir</name>
-        <value>/tmp/.hdfs-nfs</value>
+        <name>dfs.client.use.legacy.blockreader</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.permissions.enabled</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.permissions.superusergroup</name>
-        <value>hdfs</value>
+        <name>dfs.client.read.shortcircuit</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.replication</name>
-        <value>1</value>
+        <name>dfs.domain.socket.path</name>
+        <value>/var/run/hdfs-sockets/dn</value>
     </property>
-
     <property>
-        <name>dfs.replication.max</name>
-        <value>50</value>
+        <name>dfs.client.read.shortcircuit.skip.checksum</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.support.append</name>
-        <value>true</value>
-        <final>true</final>
+        <name>dfs.client.domain.socket.data.traffic</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.webhdfs.enabled</name>
+        <name>dfs.datanode.hdfs-blocks-metadata.enabled</name>
         <value>true</value>
-        <final>true</final>
     </property>
-
-    <property>
-        <name>fs.permissions.umask-mode</name>
-        <value>022</value>
-    </property>
-
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/examples/test_case_data/sandbox/mapred-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/mapred-site.xml b/examples/test_case_data/sandbox/mapred-site.xml
index ff1c7eb..e8abf50 100644
--- a/examples/test_case_data/sandbox/mapred-site.xml
+++ b/examples/test_case_data/sandbox/mapred-site.xml
@@ -14,241 +14,169 @@
   See the License for the specific language governing permissions and
   limitations under the License.
 -->
-<configuration>
-
-    <property>
-        <name>io.sort.mb</name>
-        <value>64</value>
-    </property>
-
-    <property>
-        <name>mapred.child.java.opts</name>
-        <value>-Xmx200m</value>
-    </property>
-
-    <property>
-        <name>mapred.job.map.memory.mb</name>
-        <value>250</value>
-    </property>
-
-    <property>
-        <name>mapred.job.reduce.memory.mb</name>
-        <value>250</value>
-    </property>
-
-    <property>
-        <name>mapreduce.admin.map.child.java.opts</name>
-        <value>-server -XX:NewRatio=8 -Djava.net.preferIPv4Stack=true -Dhdp.version=${hdp.version}</value>
-    </property>
-
-    <property>
-        <name>mapreduce.admin.reduce.child.java.opts</name>
-        <value>-server -XX:NewRatio=8 -Djava.net.preferIPv4Stack=true -Dhdp.version=${hdp.version}</value>
-    </property>
-
-    <property>
-        <name>mapreduce.admin.user.env</name>
-        <value>LD_LIBRARY_PATH=/usr/hdp/${hdp.version}/hadoop/lib/native:/usr/hdp/${hdp.version}/hadoop/lib/native/Linux-amd64-64</value>
-    </property>
-
-    <property>
-        <name>mapreduce.am.max-attempts</name>
-        <value>2</value>
-    </property>
-
-    <property>
-        <name>mapreduce.application.classpath</name>
-        <value>/tmp/kylin/*,$HADOOP_CONF_DIR,/usr/hdp/${hdp.version}/hbase/lib/hbase-common.jar,/usr/hdp/current/hive-client/conf/,$PWD/mr-framework/hadoop/share/hadoop/mapreduce/*:$PWD/mr-framework/hadoop/share/hadoop/mapreduce/lib/*:$PWD/mr-framework/hadoop/share/hadoop/common/*:$PWD/mr-framework/hadoop/share/hadoop/common/lib/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/lib/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/lib/*:/usr/hdp/${hdp.version}/hadoop/lib/hadoop-lzo-0.6.0.${hdp.version}.jar:/etc/hadoop/conf/secure</value>
-    </property>
-
-    <property>
-        <name>mapreduce.application.framework.path</name>
-        <value>/hdp/apps/${hdp.version}/mapreduce/mapreduce.tar.gz#mr-framework</value>
-    </property>
-
-    <property>
-        <name>mapreduce.cluster.administrators</name>
-        <value>hadoop</value>
-    </property>
-
-    <property>
-        <name>mapreduce.framework.name</name>
-        <value>yarn</value>
-    </property>
-
-    <property>
-        <name>mapreduce.job.emit-timeline-data</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>mapreduce.job.reduce.slowstart.completedmaps</name>
-        <value>0.05</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.address</name>
-        <value>sandbox.hortonworks.com:10020</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.bind-host</name>
-        <value>0.0.0.0</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.done-dir</name>
-        <value>/mr-history/done</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.intermediate-done-dir</name>
-        <value>/mr-history/tmp</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.webapp.address</name>
-        <value>sandbox.hortonworks.com:19888</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.java.opts</name>
-        <value>-Xmx200m</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.log.level</name>
-        <value>INFO</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.memory.mb</name>
-        <value>250</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.output.compress</name>
-        <value>false</value>
-    </property>
 
-    <property>
-        <name>mapreduce.map.sort.spill.percent</name>
-        <value>0.7</value>
-    </property>
 
-    <property>
-        <name>mapreduce.map.speculative</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>mapreduce.output.fileoutputformat.compress</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>mapreduce.output.fileoutputformat.compress.type</name>
-        <value>BLOCK</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.input.buffer.percent</name>
-        <value>0.0</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.java.opts</name>
-        <value>-Xmx200m</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.log.level</name>
-        <value>INFO</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.memory.mb</name>
-        <value>250</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.fetch.retry.enabled</name>
-        <value>1</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.fetch.retry.interval-ms</name>
-        <value>1000</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.fetch.retry.timeout-ms</name>
-        <value>30000</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.input.buffer.percent</name>
-        <value>0.7</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.merge.percent</name>
-        <value>0.66</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.parallelcopies</name>
-        <value>30</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.speculative</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>mapreduce.shuffle.port</name>
-        <value>13562</value>
-    </property>
-
-    <property>
-        <name>mapreduce.task.io.sort.factor</name>
-        <value>100</value>
-    </property>
-
-    <property>
-        <name>mapreduce.task.io.sort.mb</name>
-        <value>64</value>
-    </property>
-
-    <property>
-        <name>mapreduce.task.timeout</name>
-        <value>300000</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.admin-command-opts</name>
-        <value>-Dhdp.version=${hdp.version}</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.command-opts</name>
-        <value>-Xmx200m</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.log.level</name>
-        <value>INFO</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.resource.mb</name>
-        <value>250</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.staging-dir</name>
-        <value>/user</value>
-    </property>
 
+<configuration>
+<property>
+    <name>mapreduce.job.split.metainfo.maxsize</name>
+    <value>10000000</value>
+</property>
+<property>
+    <name>mapreduce.job.counters.max</name>
+    <value>120</value>
+</property>
+<property>
+    <name>mapreduce.output.fileoutputformat.compress</name>
+    <value>false</value>
+</property>
+<property>
+    <name>mapreduce.output.fileoutputformat.compress.type</name>
+    <value>BLOCK</value>
+</property>
+<property>
+    <name>mapreduce.output.fileoutputformat.compress.codec</name>
+    <value>org.apache.hadoop.io.compress.DefaultCodec</value>
+</property>
+<property>
+    <name>mapreduce.map.output.compress.codec</name>
+    <value>org.apache.hadoop.io.compress.SnappyCodec</value>
+</property>
+<property>
+    <name>mapreduce.map.output.compress</name>
+    <value>true</value>
+</property>
+<property>
+    <name>zlib.compress.level</name>
+    <value>DEFAULT_COMPRESSION</value>
+</property>
+<property>
+    <name>mapreduce.task.io.sort.factor</name>
+    <value>64</value>
+</property>
+<property>
+    <name>mapreduce.map.sort.spill.percent</name>
+    <value>0.8</value>
+</property>
+<property>
+    <name>mapreduce.reduce.shuffle.parallelcopies</name>
+    <value>10</value>
+</property>
+<property>
+    <name>mapreduce.task.timeout</name>
+    <value>600000</value>
+</property>
+<property>
+    <name>mapreduce.client.submit.file.replication</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.job.reduces</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.task.io.sort.mb</name>
+    <value>16</value>
+</property>
+<property>
+    <name>mapreduce.map.speculative</name>
+    <value>false</value>
+</property>
+<property>
+    <name>mapreduce.reduce.speculative</name>
+    <value>false</value>
+</property>
+<property>
+    <name>mapreduce.job.reduce.slowstart.completedmaps</name>
+    <value>0.8</value>
+</property>
+<property>
+    <name>mapreduce.jobhistory.address</name>
+    <value>quickstart.cloudera:10020</value>
+</property>
+<property>
+    <name>mapreduce.jobhistory.webapp.address</name>
+    <value>quickstart.cloudera:19888</value>
+</property>
+<property>
+    <name>mapreduce.jobhistory.webapp.https.address</name>
+    <value>quickstart.cloudera:19890</value>
+</property>
+<property>
+    <name>mapreduce.jobhistory.admin.address</name>
+    <value>quickstart.cloudera:10033</value>
+</property>
+<property>
+    <name>mapreduce.framework.name</name>
+    <value>yarn</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.staging-dir</name>
+    <value>/user</value>
+</property>
+<property>
+    <name>mapreduce.am.max-attempts</name>
+    <value>2</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.resource.mb</name>
+    <value>128</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.resource.cpu-vcores</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.job.ubertask.enable</name>
+    <value>false</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.command-opts</name>
+    <value>-Djava.net.preferIPv4Stack=true -Xmx52428800</value>
+</property>
+<property>
+    <name>mapreduce.map.java.opts</name>
+    <value>-Djava.net.preferIPv4Stack=true -Xmx52428800</value>
+</property>
+<property>
+    <name>mapreduce.reduce.java.opts</name>
+    <value>-Djava.net.preferIPv4Stack=true -Xmx52428800</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.admin.user.env</name>
+    <value>LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH</value>
+</property>
+<property>
+    <name>mapreduce.map.memory.mb</name>
+    <value>128</value>
+</property>
+<property>
+    <name>mapreduce.map.cpu.vcores</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.reduce.memory.mb</name>
+    <value>128</value>
+</property>
+<property>
+    <name>mapreduce.reduce.cpu.vcores</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.job.heap.memory-mb.ratio</name>
+    <value>0.8</value>
+</property>
+<property>
+    <name>mapreduce.application.classpath</name>
+    <value>/tmp/kylin/*,/usr/lib/hadoop-mapreduce/lib/*,/etc/hadoop/conf:/usr/lib/hadoop/lib/*:/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/*:/usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-yarn/.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*,/usr/lib/hbase/hbase-common.jar,/etc/hive/conf</value>
+</property>
+<property>
+    <name>mapreduce.admin.user.env</name>
+    <value>LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH</value>
+</property>
+<property>
+    <name>mapreduce.shuffle.max.connections</name>
+    <value>80</value>
+</property>
 </configuration>
+

http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/examples/test_case_data/sandbox/yarn-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/yarn-site.xml b/examples/test_case_data/sandbox/yarn-site.xml
index 8256158..8988d4a 100644
--- a/examples/test_case_data/sandbox/yarn-site.xml
+++ b/examples/test_case_data/sandbox/yarn-site.xml
@@ -15,520 +15,128 @@
   limitations under the License.
 -->
 <configuration>
-
-    <property>
-        <name>hadoop.registry.rm.enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>hadoop.registry.zk.quorum</name>
-        <value>sandbox.hortonworks.com:2181</value>
-    </property>
-
     <property>
         <name>yarn.acl.enable</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.admin.acl</name>
-        <value></value>
-    </property>
-
-    <property>
-        <name>yarn.application.classpath</name>
-        <value>$HADOOP_CONF_DIR,/usr/hdp/current/hadoop-client/*,/usr/hdp/current/hadoop-client/lib/*,/usr/hdp/current/hadoop-hdfs-client/*,/usr/hdp/current/hadoop-hdfs-client/lib/*,/usr/hdp/current/hadoop-yarn-client/*,/usr/hdp/current/hadoop-yarn-client/lib/*</value>
-    </property>
-
-    <property>
-        <name>yarn.client.nodemanager-connect.max-wait-ms</name>
-        <value>60000</value>
-    </property>
-
-    <property>
-        <name>yarn.client.nodemanager-connect.retry-interval-ms</name>
-        <value>10000</value>
-    </property>
-
-    <property>
-        <name>yarn.http.policy</name>
-        <value>HTTP_ONLY</value>
-    </property>
-
-    <property>
-        <name>yarn.log-aggregation-enable</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>yarn.log-aggregation.retain-seconds</name>
-        <value>2592000</value>
-    </property>
-
-    <property>
-        <name>yarn.log.server.url</name>
-        <value>http://sandbox.hortonworks.com:19888/jobhistory/logs</value>
-    </property>
-
-    <property>
-        <name>yarn.node-labels.fs-store.retry-policy-spec</name>
-        <value>2000, 500</value>
-    </property>
-
-    <property>
-        <name>yarn.node-labels.fs-store.root-dir</name>
-        <value>/system/yarn/node-labels</value>
-    </property>
-
-    <property>
-        <name>yarn.node-labels.manager-class</name>
-        <value>org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.address</name>
-        <value>0.0.0.0:45454</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.admin-env</name>
-        <value>MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.aux-services</name>
-        <value>mapreduce_shuffle</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
-        <value>org.apache.hadoop.mapred.ShuffleHandler</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.bind-host</name>
-        <value>0.0.0.0</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.container-executor.class</name>
-        <value>org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.container-monitor.interval-ms</name>
-        <value>3000</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.delete.debug-delay-sec</name>
-        <value>0</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage</name>
-        <value>90</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb</name>
-        <value>1000</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.disk-health-checker.min-healthy-disks</name>
-        <value>0.25</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.health-checker.interval-ms</name>
-        <value>135000</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.health-checker.script.timeout-ms</name>
-        <value>60000</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.cgroups.hierarchy</name>
-        <value>hadoop-yarn</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.cgroups.mount</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.group</name>
-        <value>hadoop</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.resources-handler.class</name>
-        <value>org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.local-dirs</name>
-        <value>/hadoop/yarn/local</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-aggregation.compression-type</name>
-        <value>gz</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-aggregation.debug-enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-aggregation.num-log-files-per-app</name>
-        <value>30</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds</name>
-        <value>-1</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-dirs</name>
-        <value>/hadoop/yarn/log</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log.retain-second</name>
-        <value>604800</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.pmem-check-enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.recovery.dir</name>
-        <value>/var/log/hadoop-yarn/nodemanager/recovery-state</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.recovery.enabled</name>
         <value>true</value>
     </property>
-
-    <property>
-        <name>yarn.nodemanager.remote-app-log-dir</name>
-        <value>/app-logs</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.remote-app-log-dir-suffix</name>
-        <value>logs</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.resource.cpu-vcores</name>
-        <value>8</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.resource.memory-mb</name>
-        <value>9216</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.resource.percentage-physical-cpu-limit</name>
-        <value>100</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.vmem-check-enabled</name>
-        <value>false</value>
-    </property>
-
     <property>
-        <name>yarn.nodemanager.vmem-pmem-ratio</name>
-        <value>10</value>
+        <name>yarn.admin.acl</name>
+        <value>*</value>
     </property>
-
     <property>
         <name>yarn.resourcemanager.address</name>
-        <value>sandbox.hortonworks.com:8050</value>
+        <value>quickstart.cloudera:8032</value>
     </property>
-
     <property>
         <name>yarn.resourcemanager.admin.address</name>
-        <value>sandbox.hortonworks.com:8141</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.am.max-attempts</name>
-        <value>2</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.bind-host</name>
-        <value>0.0.0.0</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.connect.max-wait.ms</name>
-        <value>900000</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.connect.retry-interval.ms</name>
-        <value>30000</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.fs.state-store.retry-policy-spec</name>
-        <value>2000, 500</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.fs.state-store.uri</name>
-        <value></value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.ha.enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.hostname</name>
-        <value>sandbox.hortonworks.com</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.nodes.exclude-path</name>
-        <value>/etc/hadoop/conf/yarn.exclude</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.recovery.enabled</name>
-        <value>true</value>
+        <value>quickstart.cloudera:8033</value>
     </property>
-
-    <property>
-        <name>yarn.resourcemanager.resource-tracker.address</name>
-        <value>sandbox.hortonworks.com:8025</value>
-    </property>
-
     <property>
         <name>yarn.resourcemanager.scheduler.address</name>
-        <value>sandbox.hortonworks.com:8030</value>
+        <value>quickstart.cloudera:8030</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.scheduler.class</name>
-        <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.state-store.max-completed-applications</name>
-        <value>${yarn.resourcemanager.max-completed-applications}</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.store.class</name>
-        <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size</name>
-        <value>10</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
-        <value>true</value>
+        <name>yarn.resourcemanager.resource-tracker.address</name>
+        <value>quickstart.cloudera:8031</value>
     </property>
-
     <property>
         <name>yarn.resourcemanager.webapp.address</name>
-        <value>sandbox.hortonworks.com:8088</value>
+        <value>quickstart.cloudera:8088</value>
     </property>
-
-    <property>
-        <name>yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled</name>
-        <value>false</value>
-    </property>
-
     <property>
         <name>yarn.resourcemanager.webapp.https.address</name>
-        <value>localhost:8090</value>
+        <value>quickstart.cloudera:8090</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.webapp.proxyuser.hcat.groups</name>
-        <value>*</value>
+        <name>yarn.resourcemanager.client.thread-count</name>
+        <value>50</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.webapp.proxyuser.hcat.hosts</name>
-        <value>*</value>
+        <name>yarn.resourcemanager.scheduler.client.thread-count</name>
+        <value>50</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.webapp.proxyuser.oozie.groups</name>
-        <value>*</value>
+        <name>yarn.resourcemanager.admin.client.thread-count</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.webapp.proxyuser.oozie.hosts</name>
-        <value>*</value>
+        <name>yarn.scheduler.minimum-allocation-mb</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.work-preserving-recovery.enabled</name>
-        <value>true</value>
+        <name>yarn.scheduler.increment-allocation-mb</name>
+        <value>512</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms</name>
-        <value>10000</value>
+        <name>yarn.scheduler.maximum-allocation-mb</name>
+        <value>2816</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-acl</name>
-        <value>world:anyone:rwcda</value>
+        <name>yarn.scheduler.minimum-allocation-vcores</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-address</name>
-        <value>localhost:2181</value>
+        <name>yarn.scheduler.increment-allocation-vcores</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-num-retries</name>
-        <value>1000</value>
+        <name>yarn.scheduler.maximum-allocation-vcores</name>
+        <value>2</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-retry-interval-ms</name>
+        <name>yarn.resourcemanager.amliveliness-monitor.interval-ms</name>
         <value>1000</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-state-store.parent-path</name>
-        <value>/rmstore</value>
+        <name>yarn.am.liveness-monitor.expiry-interval-ms</name>
+        <value>600000</value>
     </property>
-
-    <property>
-        <name>yarn.resourcemanager.zk-timeout-ms</name>
-        <value>10000</value>
-    </property>
-
-    <property>
-        <name>yarn.scheduler.maximum-allocation-mb</name>
-        <value>9216</value>
-    </property>
-
     <property>
-        <name>yarn.scheduler.minimum-allocation-mb</name>
-        <value>1536</value>
+        <name>yarn.resourcemanager.am.max-attempts</name>
+        <value>2</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.address</name>
-        <value>sandbox.hortonworks.com:10200</value>
+        <name>yarn.resourcemanager.container.liveness-monitor.interval-ms</name>
+        <value>600000</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.bind-host</name>
-        <value>0.0.0.0</value>
+        <name>yarn.resourcemanager.nm.liveness-monitor.interval-ms</name>
+        <value>1000</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.client.max-retries</name>
-        <value>30</value>
+        <name>yarn.nm.liveness-monitor.expiry-interval-ms</name>
+        <value>600000</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.client.retry-interval-ms</name>
-        <value>1000</value>
+        <name>yarn.resourcemanager.resource-tracker.client.thread-count</name>
+        <value>50</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.enabled</name>
-        <value>true</value>
+        <name>yarn.application.classpath</name>
+        <value>$HADOOP_CLIENT_CONF_DIR,$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_YARN_HOME/*,$HADOOP_YARN_HOME/lib/*</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.generic-application-history.store-class</name>
-        <value>org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore</value>
+        <name>yarn.resourcemanager.scheduler.class</name>
+        <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.http-authentication.simple.anonymous.allowed</name>
+        <name>yarn.scheduler.fair.user-as-default-queue</name>
         <value>true</value>
     </property>
-
-    <property>
-        <name>yarn.timeline-service.http-authentication.type</name>
-        <value>simple</value>
-    </property>
-
     <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.path</name>
-        <value>/hadoop/yarn/timeline</value>
+        <name>yarn.scheduler.fair.preemption</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.read-cache-size</name>
-        <value>104857600</value>
+        <name>yarn.scheduler.fair.sizebasedweight</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size</name>
-        <value>10000</value>
+        <name>yarn.scheduler.fair.assignmultiple</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size</name>
+        <name>yarn.resourcemanager.max-completed-applications</name>
         <value>10000</value>
     </property>
-
-    <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms</name>
-        <value>300000</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.store-class</name>
-        <value>org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.ttl-enable</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.ttl-ms</name>
-        <value>2678400000</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.webapp.address</name>
-        <value>sandbox.hortonworks.com:8188</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.webapp.https.address</name>
-        <value>sandbox.hortonworks.com:8190</value>
-    </property>
-
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 025e31a..7297f5c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -47,13 +47,13 @@
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
         <!-- Hadoop versions -->
-        <hadoop2.version>2.7.1</hadoop2.version>
-        <yarn.version>2.7.1</yarn.version>
-        <zookeeper.version>3.4.6</zookeeper.version>
-        <hive.version>1.2.1</hive.version>
-        <hive-hcatalog.version>1.2.1</hive-hcatalog.version>
-        <hbase-hadoop2.version>1.1.1</hbase-hadoop2.version>
-        <kafka.version>0.8.1</kafka.version>
+        <hadoop2.version>2.6.0-cdh5.7.0</hadoop2.version>
+        <yarn.version>2.6.0-cdh5.7.0</yarn.version>
+        <zookeeper.version>3.4.5-cdh5.7.0</zookeeper.version>
+        <hive.version>1.1.0-cdh5.7.0</hive.version>
+        <hive-hcatalog.version>1.1.0-cdh5.7.0</hive-hcatalog.version>
+        <hbase-hadoop2.version>1.2.0-cdh5.7.0</hbase-hadoop2.version>
+        <kafka.version>0.8.0</kafka.version>
 
         <!-- Dependency versions -->
         <antlr.version>3.4</antlr.version>
@@ -489,6 +489,10 @@
             <id>conjars</id>
             <url>http://conjars.org/repo/</url>
         </repository>
+        <repository>
+            <id>cloudera</id>
+            <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+        </repository>
     </repositories>
 
     <build>

http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index bca91fb..a0a9c5a 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -146,6 +146,10 @@
                     <groupId>javax.servlet</groupId>
                     <artifactId>servlet-api</artifactId>
                 </exclusion>
+	        <exclusion>
+		    <groupId>com.google.protobuf</groupId>
+		    <artifactId>protobuf-java</artifactId>
+	        </exclusion>
                 <exclusion>
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
@@ -165,6 +169,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -180,6 +188,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -195,6 +207,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -210,6 +226,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -233,6 +253,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -248,6 +272,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -264,6 +292,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         
@@ -301,6 +333,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/80283a78/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
index d5c3f60..5adf327 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
@@ -100,11 +100,6 @@ public class MockupMapContext {
             }
 
             @Override
-            public float getProgress() {
-                throw new NotImplementedException();
-            }
-
-            @Override
             public Counter getCounter(Enum<?> counterName) {
                 throw new NotImplementedException();
             }
@@ -165,6 +160,11 @@ public class MockupMapContext {
             }
 
             @Override
+            public boolean userClassesTakesPrecedence() {
+                return false;
+            }
+
+            @Override
             public Class<? extends InputFormat<?, ?>> getInputFormatClass() throws ClassNotFoundException {
                 throw new NotImplementedException();
             }
@@ -214,10 +214,6 @@ public class MockupMapContext {
                 throw new NotImplementedException();
             }
 
-            @Override
-            public boolean getTaskCleanupNeeded() {
-                throw new NotImplementedException();
-            }
 
             @Override
             public boolean getProfileEnabled() {
@@ -230,11 +226,6 @@ public class MockupMapContext {
             }
 
             @Override
-            public IntegerRanges getProfileTaskRange(boolean isMap) {
-                throw new NotImplementedException();
-            }
-
-            @Override
             public String getUser() {
                 throw new NotImplementedException();
             }


[16/55] [abbrv] kylin git commit: minor, use upper in hql

Posted by sh...@apache.org.
minor, use upper in hql

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: a08b77dd342e4b6b755b55f40383c7b180e429a8
Parents: 2224eec
Author: shaofengshi <sh...@apache.org>
Authored: Mon Jul 11 09:58:08 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 11 09:59:20 2016 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/job/JoinedFlatTable.java | 12 +++---------
 1 file changed, 3 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a08b77dd/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java b/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
index 41c301a..31c92b6 100644
--- a/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
@@ -42,8 +42,7 @@ import org.w3c.dom.Document;
 import org.w3c.dom.NodeList;
 
 /**
- * @author George Song (ysong1)
- * 
+ *
  */
 
 public class JoinedFlatTable {
@@ -115,11 +114,6 @@ public class JoinedFlatTable {
         return sql.toString();
     }
 
-    public static String generateRedistributeDataStatement(IJoinedFlatTableDesc intermediateTableDesc) {
-        final String tableName = intermediateTableDesc.getTableName();
-        return "INSERT OVERWRITE TABLE " + tableName + " SELECT * FROM " + tableName + " distribute by rand();\n";
-    }
-
     public static String generateSelectDataStatement(IJoinedFlatTableDesc intermediateTableDesc) {
         StringBuilder sql = new StringBuilder();
         sql.append("SELECT" + "\n");
@@ -222,9 +216,9 @@ public class JoinedFlatTable {
 
         if (distDcol != null) {
             String tblAlias = tableAliasMap.get(distDcol.getTable());
-            sql.append(" distribute by ").append(tblAlias).append(".").append(distDcol.getName());
+            sql.append(" DISTRIBUTE BY ").append(tblAlias).append(".").append(distDcol.getName());
         } else {
-            sql.append(" distribute by rand()");
+            sql.append(" DISTRIBUTE BY RAND()");
         }
     }
 


[31/55] [abbrv] kylin git commit: KYLIN-1848 web fix sort cubes issue

Posted by sh...@apache.org.
KYLIN-1848 web fix sort cubes issue

Signed-off-by: Jason <ji...@163.com>


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 53face5a4eac5666a951e90db7b1b8e3405bccfc
Parents: 55f5ea3
Author: zx chen <34...@qq.com>
Authored: Tue Jul 19 13:50:42 2016 +0800
Committer: Jason <ji...@163.com>
Committed: Tue Jul 19 14:07:12 2016 +0800

----------------------------------------------------------------------
 webapp/app/js/config.js              |  4 +++
 webapp/app/js/controllers/cubes.js   | 41 +++++++++++--------------------
 webapp/app/partials/cubes/cubes.html | 18 +++++++-------
 3 files changed, 28 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/53face5a/webapp/app/js/config.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/config.js b/webapp/app/js/config.js
index 46fdecd..c4a693f 100644
--- a/webapp/app/js/config.js
+++ b/webapp/app/js/config.js
@@ -116,6 +116,10 @@ KylinApp.config(function ($routeProvider, $httpProvider, $locationProvider, $log
   });
 
 // This runs when all code has loaded, and loads the config and route json manifests, before bootstrapping angular.
+
+
+
+
 window.onload = function () {
 
   // Files to load initially.

http://git-wip-us.apache.org/repos/asf/kylin/blob/53face5a/webapp/app/js/controllers/cubes.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubes.js b/webapp/app/js/controllers/cubes.js
index c103e54..85c7825 100644
--- a/webapp/app/js/controllers/cubes.js
+++ b/webapp/app/js/controllers/cubes.js
@@ -185,7 +185,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
     };
 
 //    Cube Action
-    $scope.enable = function (cube, cubeIndex) {
+    $scope.enable = function (cube) {
       SweetAlert.swal({
         title: '',
         text: 'Are you sure to enable the cube? Please note: if cube schema is changed in the disabled period, all segments of the cube will be discarded due to data and schema mismatch.',
@@ -203,7 +203,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
             loadingRequest.hide();
             $scope.refreshCube(cube).then(function(_cube){
               if(_cube && _cube.name){
-                $scope.cubeList.cubes[cubeIndex] = _cube;
+                $scope.cubeList.cubes[$scope.cubeList.cubes.indexOf(cube)] = _cube;
               }
             });
             SweetAlert.swal('Success!', 'Enable job was submitted successfully', 'success');
@@ -222,7 +222,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
       });
     };
 
-    $scope.purge = function (cube, cubeIndex) {
+    $scope.purge = function (cube) {
       SweetAlert.swal({
         title: '',
         text: 'Are you sure to purge the cube? ',
@@ -239,9 +239,9 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
 
             loadingRequest.hide();
             $scope.refreshCube(cube).then(function(_cube){
-              if(_cube && _cube.name){
-                $scope.cubeList.cubes[cubeIndex] = _cube;
-              }
+             if(_cube && _cube.name){
+                $scope.cubeList.cubes[$scope.cubeList.cubes.indexOf(cube)] = _cube;
+             }
             });
             SweetAlert.swal('Success!', 'Purge job was submitted successfully', 'success');
           },function(e){
@@ -258,7 +258,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
       });
     }
 
-    $scope.disable = function (cube, cubeIndex) {
+    $scope.disable = function (cube) {
 
       SweetAlert.swal({
         title: '',
@@ -277,7 +277,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
             loadingRequest.hide();
             $scope.refreshCube(cube).then(function(_cube){
               if(_cube && _cube.name){
-                $scope.cubeList.cubes[cubeIndex] = _cube;
+                $scope.cubeList.cubes[$scope.cubeList.cubes.indexOf(cube)] = _cube;
               }
             });
             SweetAlert.swal('Success!', 'Disable job was submitted successfully', 'success');
@@ -297,9 +297,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
       });
     };
 
-
-
-    $scope.dropCube = function (cube, cubeIndex) {
+    $scope.dropCube = function (cube) {
 
       SweetAlert.swal({
         title: '',
@@ -316,7 +314,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
           CubeService.drop({cubeId: cube.name}, {}, function (result) {
             loadingRequest.hide();
             SweetAlert.swal('Success!', 'Cube drop is done successfully', 'success');
-            $scope.cubeList.cubes.splice(cubeIndex,1);
+            $scope.cubeList.cubes.splice($scope.cubeList.cubes.indexOf(cube),1);
           },function(e){
 
             loadingRequest.hide();
@@ -333,7 +331,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
       });
     };
 
-    $scope.startJobSubmit = function (cube, cubeIndex) {
+    $scope.startJobSubmit = function (cube) {
       $scope.loadDetail(cube);
       // for streaming cube build tip
       if(cube.streaming){
@@ -366,9 +364,6 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
               },
               scope:function(){
                 return $scope;
-              },
-              cubeIndex:function(){
-                return cubeIndex;
               }
             }
           });
@@ -417,7 +412,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
       }
     };
 
-    $scope.startRefresh = function (cube, cubeIndex) {
+    $scope.startRefresh = function (cube) {
       $scope.metaModel={
         model:modelsManager.getModelByCube(cube.name)
       };
@@ -436,9 +431,6 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
           },
           scope:function(){
             return $scope;
-          },
-          cubeIndex:function(){
-            return cubeIndex;
           }
         }
       });
@@ -464,7 +456,7 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
     $scope.cubeEdit = function (cube) {
       $location.path("cubes/edit/" + cube.name);
     }
-    $scope.startMerge = function (cube, cubeIndex) {
+    $scope.startMerge = function (cube) {
       $scope.metaModel={
         model:modelsManager.getModelByCube(cube.name)
       };
@@ -483,9 +475,6 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
           },
           scope:function(){
             return $scope;
-          },
-          cubeIndex:function(){
-            return cubeIndex;
           }
         }
       });
@@ -550,7 +539,7 @@ var cubeCloneCtrl = function ($scope, $modalInstance, CubeService, MessageServic
 }
 
 
-var jobSubmitCtrl = function ($scope, $modalInstance, CubeService, MessageService, $location, cube, metaModel, buildType, SweetAlert, loadingRequest, scope, cubeIndex, CubeList) {
+var jobSubmitCtrl = function ($scope, $modalInstance, CubeService, MessageService, $location, cube, metaModel, buildType, SweetAlert, loadingRequest, scope, CubeList) {
   $scope.cubeList = CubeList;
   $scope.cube = cube;
   $scope.metaModel = metaModel;
@@ -578,7 +567,7 @@ var jobSubmitCtrl = function ($scope, $modalInstance, CubeService, MessageServic
       $modalInstance.dismiss('cancel');
       SweetAlert.swal('Success!', 'Rebuild job was submitted successfully', 'success');
       scope.refreshCube(cube).then(function(_cube){
-          $scope.cubeList.cubes[cubeIndex] = _cube;
+          $scope.cubeList.cubes[$scope.cubeList.cubes.indexOf(cube)] = _cube;
         });
     }, function (e) {
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/53face5a/webapp/app/partials/cubes/cubes.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubes/cubes.html b/webapp/app/partials/cubes/cubes.html
index 6f33ed7..72a3222 100644
--- a/webapp/app/partials/cubes/cubes.html
+++ b/webapp/app/partials/cubes/cubes.html
@@ -49,7 +49,7 @@
         </tr>
         </thead>
         <!--Body-->
-        <tbody ng-repeat="cube in cubeList.cubes track by $index| orderObjectBy:state.filterAttr:state.filterReverse">
+        <tbody ng-repeat="cube in cubeList.cubes | orderObjectBy:state.filterAttr:state.filterReverse ">
         <tr ng-class="{accordion:true}" style="cursor: pointer"  ng-click="cube.showDetail=!cube.showDetail;loadDetail(cube)">
             <td>
                 <i ng-show="!cube.showDetail" class="fa fa-chevron-circle-right blue"></i>
@@ -81,16 +81,16 @@
                     </button>
                     <ul class="dropdown-menu" role="menu">
                         <li ng-if="cube.status=='DISABLED' && userService.hasRole('ROLE_ADMIN') ">
-                            <a ng-click="dropCube(cube, $index)" tooltip="Drop the cube, related jobs and data permanently.">Drop</a></li>
+                            <a ng-click="dropCube(cube)" tooltip="Drop the cube, related jobs and data permanently.">Drop</a></li>
                         <li ng-if="cube.status=='DISABLED' && (userService.hasRole('ROLE_ADMIN') || hasPermission(cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))">
                             <a ng-click="cubeEdit(cube);">Edit</a></li>
-                      <li ng-if="cube.streaming && cube.status=='DISABLED' && (userService.hasRole('ROLE_ADMIN') || hasPermission(cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))">
-                        <li><a ng-click="startJobSubmit(cube, $index);">Build</a></li>
-                        <li><a ng-click="startRefresh(cube, $index)">Refresh</a></li>
-                        <li><a ng-click="startMerge(cube, $index)">Merge</a></li>
-                        <li ng-if="cube.status!='DISABLED'"><a ng-click="disable(cube, $index)">Disable</a></li>
-                        <li ng-if="cube.status=='DISABLED'"><a ng-click="enable(cube, $index)">Enable</a></li>
-                        <li ng-if="cube.status=='DISABLED'"><a ng-click="purge(cube, $index)">Purge</a></li>
+                      <li ng-if="cube.streaming && cube.status=='DISABLED' && (userService.hasRole('ROLE_ADMIN') || hasPermission(cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"></li>
+                        <li><a ng-click="startJobSubmit(cube);">Build</a></li>
+                        <li><a ng-click="startRefresh(cube)">Refresh</a></li>
+                        <li><a ng-click="startMerge(cube)">Merge</a></li>
+                        <li ng-if="cube.status!='DISABLED'"><a ng-click="disable(cube)">Disable</a></li>
+                        <li ng-if="cube.status=='DISABLED'"><a ng-click="enable(cube)">Enable</a></li>
+                        <li ng-if="cube.status=='DISABLED'"><a ng-click="purge(cube)">Purge</a></li>
                         <li><a ng-click="cloneCube(cube)">Clone</a></li>
 
                     </ul>


[43/55] [abbrv] kylin git commit: KYLIN-1911 NPE when extended column has NULL value

Posted by sh...@apache.org.
KYLIN-1911 NPE when extended column has NULL value


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 1f79762cd838040756516b2662867cf0351e08c5
Parents: 543e9f9
Author: Hongbin Ma <ma...@apache.org>
Authored: Sun Jul 24 20:19:34 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Sun Jul 24 20:28:15 2016 +0800

----------------------------------------------------------------------
 .../kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/1f79762c/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java
index d566331..dc718d1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java
@@ -147,6 +147,11 @@ public class ExtendedColumnMeasureType extends MeasureType<ByteArray> {
 
             @Override
             public void reload(Object measureValue) {
+                if (measureValue == null) {
+                    value = null;
+                    return;
+                }
+                
                 ByteArray byteArray = (ByteArray) measureValue;
                 //the array in ByteArray is guaranteed to be completed owned by the ByteArray
                 value = Bytes.toString(byteArray.array());


[44/55] [abbrv] kylin git commit: [maven-release-plugin] prepare release kylin-1.5.3

Posted by sh...@apache.org.
[maven-release-plugin] prepare release kylin-1.5.3


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 4cdc07ea34d0c2fb12951500e0fa28130659affb
Parents: 1f79762
Author: shaofengshi <sh...@apache.org>
Authored: Mon Jul 25 01:57:29 2016 +0000
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 25 01:57:29 2016 +0000

----------------------------------------------------------------------
 assembly/pom.xml         |  2 +-
 atopcalcite/pom.xml      |  2 +-
 core-common/pom.xml      |  2 +-
 core-cube/pom.xml        |  2 +-
 core-dictionary/pom.xml  |  2 +-
 core-job/pom.xml         |  2 +-
 core-metadata/pom.xml    |  2 +-
 core-storage/pom.xml     |  2 +-
 engine-mr/pom.xml        |  2 +-
 engine-spark/pom.xml     |  2 +-
 engine-streaming/pom.xml |  2 +-
 jdbc/pom.xml             |  2 +-
 kylin-it/pom.xml         | 13 ++++++-------
 pom.xml                  |  6 +++---
 query/pom.xml            |  2 +-
 server-base/pom.xml      |  2 +-
 server/pom.xml           |  2 +-
 source-hive/pom.xml      |  2 +-
 source-kafka/pom.xml     |  2 +-
 storage-hbase/pom.xml    |  2 +-
 tool/pom.xml             |  5 ++---
 21 files changed, 29 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index fc1ef63..976f8b2 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>kylin</artifactId>
         <groupId>org.apache.kylin</groupId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/atopcalcite/pom.xml
----------------------------------------------------------------------
diff --git a/atopcalcite/pom.xml b/atopcalcite/pom.xml
index be28f1c..4a849b5 100644
--- a/atopcalcite/pom.xml
+++ b/atopcalcite/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/core-common/pom.xml
----------------------------------------------------------------------
diff --git a/core-common/pom.xml b/core-common/pom.xml
index ca4c7a9..8e75b18 100644
--- a/core-common/pom.xml
+++ b/core-common/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/core-cube/pom.xml
----------------------------------------------------------------------
diff --git a/core-cube/pom.xml b/core-cube/pom.xml
index 0fc2875..2d4ca47 100644
--- a/core-cube/pom.xml
+++ b/core-cube/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/core-dictionary/pom.xml
----------------------------------------------------------------------
diff --git a/core-dictionary/pom.xml b/core-dictionary/pom.xml
index b0f400e..f14d639 100644
--- a/core-dictionary/pom.xml
+++ b/core-dictionary/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/core-job/pom.xml
----------------------------------------------------------------------
diff --git a/core-job/pom.xml b/core-job/pom.xml
index 8a1211f..bc3f684 100644
--- a/core-job/pom.xml
+++ b/core-job/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/core-metadata/pom.xml
----------------------------------------------------------------------
diff --git a/core-metadata/pom.xml b/core-metadata/pom.xml
index 0a7cb40..1a2c5e8 100644
--- a/core-metadata/pom.xml
+++ b/core-metadata/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
         
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/core-storage/pom.xml
----------------------------------------------------------------------
diff --git a/core-storage/pom.xml b/core-storage/pom.xml
index 27ee61c..23a6e44 100644
--- a/core-storage/pom.xml
+++ b/core-storage/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/engine-mr/pom.xml
----------------------------------------------------------------------
diff --git a/engine-mr/pom.xml b/engine-mr/pom.xml
index 12a6b73..23c881d 100644
--- a/engine-mr/pom.xml
+++ b/engine-mr/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/engine-spark/pom.xml
----------------------------------------------------------------------
diff --git a/engine-spark/pom.xml b/engine-spark/pom.xml
index 207fc4d..b7d450f 100644
--- a/engine-spark/pom.xml
+++ b/engine-spark/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/engine-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/engine-streaming/pom.xml b/engine-streaming/pom.xml
index 7110377..5307466 100644
--- a/engine-streaming/pom.xml
+++ b/engine-streaming/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index a663fb3..e86998e 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/kylin-it/pom.xml
----------------------------------------------------------------------
diff --git a/kylin-it/pom.xml b/kylin-it/pom.xml
index a96dcd9..e9765f1 100644
--- a/kylin-it/pom.xml
+++ b/kylin-it/pom.xml
@@ -18,12 +18,11 @@
 -->
 
 
-<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/xsd/maven-4.0.0.xsd">
+<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/xsd/maven-4.0.0.xsd">
     <parent>
         <artifactId>kylin</artifactId>
         <groupId>org.apache.kylin</groupId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -31,8 +30,8 @@
     <name>Kylin:IT</name>
 
     <properties>
-        <hdp.version/>
-        <fastBuildMode/>
+        <hdp.version />
+        <fastBuildMode />
     </properties>
 
     <!-- Dependencies. -->
@@ -322,7 +321,7 @@
                                         <argument>-Dhdp.version=${hdp.version}</argument>
                                         <argument>-DfastBuildMode=${fastBuildMode}</argument>
                                         <argument>-classpath</argument>
-                                        <classpath/>
+                                        <classpath />
                                         <argument>org.apache.kylin.provision.BuildCubeWithEngine</argument>
                                     </arguments>
                                     <workingDirectory>${project.basedir}</workingDirectory>
@@ -343,7 +342,7 @@
                                         <argument>-Dhdp.version=${hdp.version}</argument>
                                         <argument>-DfastBuildMode=${fastBuildMode}</argument>
                                         <argument>-classpath</argument>
-                                        <classpath/>
+                                        <classpath />
                                         <argument>org.apache.kylin.provision.BuildCubeWithStream</argument>
                                     </arguments>
                                     <workingDirectory>${project.basedir}</workingDirectory>

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 94f4e33..0715e87 100644
--- a/pom.xml
+++ b/pom.xml
@@ -31,7 +31,7 @@
     <groupId>org.apache.kylin</groupId>
     <artifactId>kylin</artifactId>
     <packaging>pom</packaging>
-    <version>1.5.3-SNAPSHOT</version>
+    <version>1.5.3</version>
 
     <name>Kylin:HadoopOLAPEngine</name>
     <url>http://kylin.apache.org</url>
@@ -168,7 +168,7 @@
         <connection>scm:git:https://git-wip-us.apache.org/repos/asf/kylin.git</connection>
         <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/kylin.git</developerConnection>
         <url>https://git-wip-us.apache.org/repos/asf/kylin.git</url>
-      <tag>HEAD</tag>
+      <tag>kylin-1.5.3</tag>
   </scm>
 
     <dependencyManagement>
@@ -653,7 +653,7 @@
                                         </goals>
                                     </pluginExecutionFilter>
                                     <action>
-                                        <ignore></ignore>
+                                        <ignore />
                                     </action>
                                 </pluginExecution>
                             </pluginExecutions>

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/query/pom.xml
----------------------------------------------------------------------
diff --git a/query/pom.xml b/query/pom.xml
index a0b8d94..dac7db3 100644
--- a/query/pom.xml
+++ b/query/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/server-base/pom.xml
----------------------------------------------------------------------
diff --git a/server-base/pom.xml b/server-base/pom.xml
index e6a4cc5..e1f42ab 100644
--- a/server-base/pom.xml
+++ b/server-base/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index e662909..bca91fb 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/source-hive/pom.xml
----------------------------------------------------------------------
diff --git a/source-hive/pom.xml b/source-hive/pom.xml
index a7c68eb..b2e6b33 100644
--- a/source-hive/pom.xml
+++ b/source-hive/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/source-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/source-kafka/pom.xml b/source-kafka/pom.xml
index cf92762..2e983da 100644
--- a/source-kafka/pom.xml
+++ b/source-kafka/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/storage-hbase/pom.xml b/storage-hbase/pom.xml
index 0cb42db..39782de 100644
--- a/storage-hbase/pom.xml
+++ b/storage-hbase/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4cdc07ea/tool/pom.xml
----------------------------------------------------------------------
diff --git a/tool/pom.xml b/tool/pom.xml
index 171644a..5342336 100644
--- a/tool/pom.xml
+++ b/tool/pom.xml
@@ -17,12 +17,11 @@
  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/xsd/maven-4.0.0.xsd">
+<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/xsd/maven-4.0.0.xsd">
     <parent>
         <artifactId>kylin</artifactId>
         <groupId>org.apache.kylin</groupId>
-        <version>1.5.3-SNAPSHOT</version>
+        <version>1.5.3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 


[06/55] [abbrv] kylin git commit: KYLIN-1858 remove all ii related code

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BitMapContainer.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BitMapContainer.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BitMapContainer.java
deleted file mode 100644
index 0ee16b8..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BitMapContainer.java
+++ /dev/null
@@ -1,227 +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.index;
-
-//import it.uniroma3.mat.extendedset.intset.ConciseSet;
-//
-//import java.nio.ByteBuffer;
-//import java.nio.IntBuffer;
-//import java.util.ArrayList;
-//import java.util.Arrays;
-//import java.util.List;
-//
-//import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-//
-//import org.apache.kylin.common.util.BytesUtil;
-//import org.apache.kylin.dict.Dictionary;
-
-/**
- * @author yangli9
- */
-//public class BitMapContainer implements ColumnValueContainer {
-//
-//    int valueLen;
-//    int nValues;
-//    int size;
-//    ConciseSet[] sets;
-//    boolean closedForChange;
-//
-//    transient byte[] temp;
-//
-//    public BitMapContainer(TableRecordInfoDigest digest, int col) {
-//        this.valueLen = digest.length(col);
-//        this.size = 0;
-//        this.nValues = digest.getMaxID(col) + 1;
-//        this.sets = null;
-//        this.closedForChange = false;
-//
-//        this.temp = new byte[valueLen];
-//    }
-//
-//    @Override
-//    public void append(ImmutableBytesWritable valueBytes) {
-//        int value = BytesUtil.readUnsigned(valueBytes.get(), valueBytes.getOffset(), valueLen);
-//        append(value);
-//    }
-//
-//    public void append(int value) {
-//        checkUpdateMode();
-//        if (value == Dictionary.NULL_ID[valueLen]) {
-//            value = nValues; // set[nValues] holds NULL
-//        }
-//        sets[value].add(size);
-//        size++;
-//    }
-//
-//    @Override
-//    public void getValueAt(int i, ImmutableBytesWritable valueBytes) {
-//        int value = getValueIntAt(i);
-//        BytesUtil.writeUnsigned(value, temp, 0, valueLen);
-//        valueBytes.set(temp, 0, valueLen);
-//    }
-//
-//    @Override
-//    public ConciseSet getBitMap(Integer startId, Integer endId) {
-//        if (startId == null && endId == null) {
-//            return sets[this.nValues];
-//        }
-//
-//        int start = 0;
-//        int end = this.nValues - 1;
-//        if (startId != null) {
-//            start = startId;
-//        }
-//        if (endId != null) {
-//            end = endId;
-//        }
-//
-//        ConciseSet ret = new ConciseSet();
-//        for (int i = start; i <= end; ++i) {
-//            ConciseSet temp = getBitMap(i);
-//            ret.addAll(temp);
-//        }
-//        return ret;
-//    }
-//
-//    private ConciseSet getBitMap(int valueId) {
-//        if (valueId >= 0 && valueId <= getMaxValueId())
-//            return sets[valueId];
-//        else
-//            return sets[this.nValues];
-//    }
-//
-//    @Override
-//    public int getMaxValueId() {
-//        return this.nValues - 1;
-//    }
-//
-//    public int getValueIntAt(int i) {
-//        for (int v = 0; v < nValues; v++) {
-//            if (sets[v].contains(i)) {
-//                return v;
-//            }
-//        }
-//        // if v is not in [0..nValues-1], then it must be nValue (NULL)
-//        return Dictionary.NULL_ID[valueLen];
-//    }
-//
-//    private void checkUpdateMode() {
-//        if (isClosedForChange()) {
-//            throw new IllegalStateException();
-//        }
-//        if (sets == null) {
-//            sets = new ConciseSet[nValues + 1];
-//            for (int i = 0; i <= nValues; i++) {
-//                sets[i] = new ConciseSet();
-//            }
-//        }
-//    }
-//
-//    private boolean isClosedForChange() {
-//        return closedForChange;
-//    }
-//
-//    @Override
-//    public void closeForChange() {
-//        closedForChange = true;
-//    }
-//
-//    @Override
-//    public int getSize() {
-//        return size;
-//    }
-//
-//    public List<ImmutableBytesWritable> toBytes() {
-//        if (isClosedForChange() == false)
-//            closeForChange();
-//
-//        List<ImmutableBytesWritable> r = new ArrayList<ImmutableBytesWritable>(nValues + 1);
-//        for (int i = 0; i <= nValues; i++) {
-//            r.add(setToBytes(sets[i]));
-//        }
-//        return r;
-//    }
-//
-//    public void fromBytes(List<ImmutableBytesWritable> bytes) {
-//        assert nValues + 1 == bytes.size();
-//        sets = new ConciseSet[nValues + 1];
-//        size = 0;
-//        for (int i = 0; i <= nValues; i++) {
-//            sets[i] = bytesToSet(bytes.get(i));
-//            size += sets[i].size();
-//        }
-//        closedForChange = true;
-//    }
-//
-//    private ImmutableBytesWritable setToBytes(ConciseSet set) {
-//        byte[] array;
-//        if (set.isEmpty()) // ConciseSet.toByteBuffer() throws exception when
-//                           // set is empty
-//            array = BytesUtil.EMPTY_BYTE_ARRAY;
-//        else
-//            array = set.toByteBuffer().array();
-//        return new ImmutableBytesWritable(array);
-//    }
-//
-//    private ConciseSet bytesToSet(ImmutableBytesWritable bytes) {
-//        if (bytes.get() == null || bytes.getLength() == 0) {
-//            return new ConciseSet();
-//        } else {
-//            IntBuffer intBuffer = ByteBuffer.wrap(bytes.get(), bytes.getOffset(), bytes.getLength()).asIntBuffer();
-//            int[] words = new int[intBuffer.capacity()];
-//            intBuffer.get(words);
-//            return new ConciseSet(words, false);
-//        }
-//    }
-//
-//    @Override
-//    public int hashCode() {
-//        final int prime = 31;
-//        int result = 1;
-//        result = prime * result + (closedForChange ? 1231 : 1237);
-//        result = prime * result + nValues;
-//        result = prime * result + Arrays.hashCode(sets);
-//        result = prime * result + size;
-//        result = prime * result + valueLen;
-//        return result;
-//    }
-//
-//    @Override
-//    public boolean equals(Object obj) {
-//        if (this == obj)
-//            return true;
-//        if (obj == null)
-//            return false;
-//        if (getClass() != obj.getClass())
-//            return false;
-//        BitMapContainer other = (BitMapContainer) obj;
-//        if (closedForChange != other.closedForChange)
-//            return false;
-//        if (nValues != other.nValues)
-//            return false;
-//        if (!Arrays.equals(sets, other.sets))
-//            return false;
-//        if (size != other.size)
-//            return false;
-//        if (valueLen != other.valueLen)
-//            return false;
-//        return true;
-//    }
-//
-//}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ColumnValueContainer.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ColumnValueContainer.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ColumnValueContainer.java
deleted file mode 100644
index ea35bb4..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ColumnValueContainer.java
+++ /dev/null
@@ -1,43 +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.index;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- * @author yangli9
- */
-public interface ColumnValueContainer {
-
-    void append(ImmutableBytesWritable valueBytes);
-
-    void closeForChange();
-
-    int getSize();
-
-    // works only after closeForChange()
-    void getValueAt(int i, ImmutableBytesWritable valueBytes);
-
-    ConciseSet getBitMap(Integer startId, Integer endId);
-
-    int getMaxValueId();
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
deleted file mode 100644
index e395544..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
+++ /dev/null
@@ -1,184 +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.index;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.Dictionary;
-
-import com.ning.compress.lzf.LZFDecoder;
-import com.ning.compress.lzf.LZFEncoder;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- * @author yangli9
- */
-public class CompressedValueContainer implements ColumnValueContainer {
-    int valueLen;
-    int nValues;
-    int cap;
-    int size;
-    byte[] uncompressed;
-    byte[] compressed;
-
-    public CompressedValueContainer(TableRecordInfoDigest digest, int col, int cap) {
-        this(digest.length(col), digest.getMaxID(col) + 1, cap);
-    }
-
-    public CompressedValueContainer(int valueLen, int nValues, int cap) {
-        this.valueLen = valueLen;
-        this.nValues = nValues;
-        this.cap = cap;
-        this.size = 0;
-        this.uncompressed = null;
-        this.compressed = null;
-    }
-
-    @Override
-    public void append(ImmutableBytesWritable valueBytes) {
-        checkUpdateMode();
-        System.arraycopy(valueBytes.get(), valueBytes.getOffset(), uncompressed, valueLen * size, valueLen);
-        size++;
-    }
-
-    @Override
-    public void getValueAt(int i, ImmutableBytesWritable valueBytes) {
-        valueBytes.set(uncompressed, valueLen * i, valueLen);
-    }
-
-    @Override
-    public ConciseSet getBitMap(Integer startId, Integer endId) {
-        ConciseSet ret = new ConciseSet();
-        int nullId = Dictionary.NULL_ID[valueLen];
-
-        if (startId == null && endId == null) {
-            //entry for getting null values 
-            for (int i = 0; i < size; ++i) {
-                int valueID = BytesUtil.readUnsigned(uncompressed, i * valueLen, valueLen);
-                if (nullId == valueID) {
-                    ret.add(i);
-                }
-            }
-            return ret;
-        }
-
-        //normal values
-        for (int i = 0; i < size; ++i) {
-            int valueID = BytesUtil.readUnsigned(uncompressed, i * valueLen, valueLen);
-            if (valueID == nullId) {
-                continue;
-            }
-
-            if (startId != null && valueID < startId) {
-                continue;
-            }
-
-            if (endId != null && valueID > endId) {
-                continue;
-            }
-
-            ret.add(i);
-        }
-        return ret;
-
-    }
-
-    @Override
-    public int getMaxValueId() {
-        return nValues - 1;
-    }
-
-    private void checkUpdateMode() {
-        if (isClosedForChange()) {
-            throw new IllegalArgumentException();
-        }
-        if (uncompressed == null) {
-            uncompressed = new byte[valueLen * cap];
-        }
-    }
-
-    private boolean isClosedForChange() {
-        return compressed != null;
-    }
-
-    @Override
-    public void closeForChange() {
-        checkUpdateMode();
-        try {
-            compressed = LZFEncoder.encode(uncompressed, 0, valueLen * size);
-        } catch (Exception e) {
-            throw new RuntimeException("LZF encode failure", e);
-        }
-    }
-
-    @Override
-    public int getSize() {
-        return size;
-    }
-
-    public ImmutableBytesWritable toBytes() {
-        if (isClosedForChange() == false)
-            closeForChange();
-        return new ImmutableBytesWritable(compressed);
-    }
-
-    public void fromBytes(ImmutableBytesWritable bytes) {
-        try {
-            uncompressed = LZFDecoder.decode(bytes.get(), bytes.getOffset(), bytes.getLength());
-        } catch (IOException e) {
-            throw new RuntimeException("LZF decode failure", e);
-        }
-        size = cap = uncompressed.length / valueLen;
-        compressed = BytesUtil.EMPTY_BYTE_ARRAY; // mark closed
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + size;
-        result = prime * result + valueLen;
-        result = prime * result + Arrays.hashCode(uncompressed);
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        CompressedValueContainer other = (CompressedValueContainer) obj;
-        if (size != other.size)
-            return false;
-        if (valueLen != other.valueLen)
-            return false;
-        if (!Bytes.equals(uncompressed, 0, size * valueLen, uncompressed, 0, size * valueLen))
-            return false;
-        return true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/IncrementalSliceMaker.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/IncrementalSliceMaker.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/IncrementalSliceMaker.java
deleted file mode 100644
index 69e016d..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/IncrementalSliceMaker.java
+++ /dev/null
@@ -1,123 +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.index;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-
-/**
- * @author yangli9
- */
-public class IncrementalSliceMaker {
-
-    TableRecordInfo info;
-    private int nColumns;
-    int nRecordsCap;
-
-    short shard;
-    long sliceTimestamp;
-    int nRecords;
-    private ColumnValueContainer[] containers;
-
-    transient ImmutableBytesWritable temp = new ImmutableBytesWritable();
-
-    public IncrementalSliceMaker(TableRecordInfo info, short shard) {
-        this.info = info;
-        this.nColumns = info.getDigest().getColumnCount();
-        this.nRecordsCap = Math.max(1, info.getDescriptor().getSliceSize());
-
-        this.shard = shard;
-        this.sliceTimestamp = Long.MIN_VALUE;
-        this.nRecords = 0;
-        this.containers = null;
-
-        doneSlice(); // init containers
-    }
-
-    private Slice doneSlice() {
-        Slice r = null;
-        if (nRecords > 0) {
-            for (int i = 0; i < nColumns; i++) {
-                containers[i].closeForChange();
-            }
-            r = new Slice(info.getDigest(), shard, sliceTimestamp, containers);
-        }
-
-        // reset for next slice
-        nRecords = 0;
-        containers = new ColumnValueContainer[nColumns];
-        //        for (int i : info.getDescriptor().getBitmapColumns()) {
-        //            containers[i] = new CompressedValueContainer(info.getDigest(), i,
-        //                    nRecordsCap);
-        //        }
-        for (int i : info.getDescriptor().getValueColumns()) {
-            containers[i] = new CompressedValueContainer(info.getDigest(), i, nRecordsCap);
-        }
-        for (int i : info.getDescriptor().getMetricsColumns()) {
-            containers[i] = new CompressedValueContainer(info.getDigest(), i, nRecordsCap);
-        }
-
-        return r;
-
-    }
-
-    // NOTE: record must be appended in time order
-    public Slice append(TableRecord rec) {
-        if (rec.getShard() != shard)
-            throw new IllegalStateException();
-
-        Slice doneSlice = null;
-
-        if (isFull()) {
-            doneSlice = doneSlice();
-        }
-
-        if (nRecords == 0) {
-            sliceTimestamp = increaseSliceTimestamp(rec.getTimestamp());
-        }
-
-        nRecords++;
-        for (int i = 0; i < nColumns; i++) {
-            rec.getValueBytes(i, temp);
-            containers[i].append(temp);
-        }
-
-        return doneSlice;
-    }
-
-    private long increaseSliceTimestamp(long timestamp) {
-        if (timestamp < sliceTimestamp)
-            throw new IllegalStateException();
-
-        if (timestamp == sliceTimestamp)
-            return ++timestamp; // ensure slice timestamp increases
-        else
-            return timestamp;
-    }
-
-    public Slice close() {
-        Slice doneSlice = doneSlice();
-        this.sliceTimestamp = Long.MIN_VALUE;
-        this.nRecords = 0;
-        return doneSlice;
-    }
-
-    private boolean isFull() {
-        return nRecords >= nRecordsCap;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
deleted file mode 100644
index d42cab0..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
+++ /dev/null
@@ -1,133 +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.index;
-
-import java.util.Arrays;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.dimension.DimensionEncoding;
-import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
-import org.apache.kylin.metadata.datatype.LongMutable;
-
-/**
- */
-public class RawTableRecord implements Cloneable {
-    TableRecordInfoDigest digest;
-    private byte[] buf; // consecutive column value IDs (encoded by dictionary)
-
-    public RawTableRecord(TableRecordInfoDigest info) {
-        this.digest = info;
-        this.buf = new byte[info.getByteFormLen()];
-        reset();
-    }
-
-    public RawTableRecord(RawTableRecord another) {
-        this.digest = another.digest;
-        this.buf = Bytes.copy(another.buf);
-    }
-
-    public void reset() {
-        Arrays.fill(buf, DimensionEncoding.NULL);
-    }
-
-    public boolean isMetric(int col) {
-        return digest.isMetrics(col);
-    }
-
-    public FixedLenMeasureCodec<LongMutable> codec(int col) {
-        return digest.codec(col);
-    }
-
-    public final int length(int col) {
-        return digest.length(col);
-    }
-
-    public final int offset(int col) {
-        return digest.offset(col);
-    }
-
-    public int getColumnCount() {
-        return digest.getColumnCount();
-    }
-
-    public void setValueID(int col, int id) {
-        BytesUtil.writeUnsigned(id, buf, digest.offset(col), digest.length(col));
-    }
-
-    public int getValueID(int col) {
-        return BytesUtil.readUnsigned(buf, digest.offset(col), digest.length(col));
-    }
-
-    public void setValueMetrics(int col, LongMutable value) {
-        digest.codec(col).write(value, buf, digest.offset(col));
-    }
-
-    public String getValueMetric(int col) {
-        digest.codec(col).read(buf, digest.offset(col));
-        return (String) digest.codec(col).getValue();
-    }
-
-    public byte[] getBytes() {
-        return buf;
-    }
-
-    //TODO is it possible to avoid copying?
-    public void setBytes(byte[] bytes, int offset, int length) {
-        assert buf.length == length;
-        System.arraycopy(bytes, offset, buf, 0, length);
-    }
-
-    public void setValueBytes(int col, ImmutableBytesWritable bytes) {
-        System.arraycopy(bytes.get(), bytes.getOffset(), buf, digest.offset(col), digest.length(col));
-    }
-
-    public void getValueBytes(int col, ImmutableBytesWritable bytes) {
-        bytes.set(buf, offset(col), length(col));
-    }
-
-    @Override
-    public Object clone() {
-        return new RawTableRecord(this);
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + Arrays.hashCode(buf);
-        // result = prime * result + ((digest == null) ? 0 : digest.hashCode());
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        RawTableRecord other = (RawTableRecord) obj;
-        if (!Arrays.equals(buf, other.buf))
-            return false;
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ShardingSliceBuilder.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ShardingSliceBuilder.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ShardingSliceBuilder.java
deleted file mode 100644
index 45c9c0a..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ShardingSliceBuilder.java
+++ /dev/null
@@ -1,53 +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.index;
-
-import java.util.List;
-
-import com.google.common.collect.Lists;
-
-public class ShardingSliceBuilder {
-
-    IncrementalSliceMaker[] builders;
-
-    public ShardingSliceBuilder(TableRecordInfo info) {
-        int sharding = info.getDescriptor().getSharding();
-        builders = new IncrementalSliceMaker[sharding];
-        for (short i = 0; i < sharding; i++) {
-            builders[i] = new IncrementalSliceMaker(info, i);
-        }
-    }
-
-    // NOTE: record must be appended in time order
-    public Slice append(TableRecord rec) {
-        short shard = rec.getShard();
-        return builders[shard].append(rec);
-    }
-
-    public List<Slice> close() {
-        List<Slice> result = Lists.newArrayList();
-        for (IncrementalSliceMaker builder : builders) {
-            Slice slice = builder.close();
-            if (slice != null)
-                result.add(slice);
-        }
-        return result;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
deleted file mode 100644
index dc2c5c4..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
+++ /dev/null
@@ -1,215 +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.index;
-
-import java.util.Iterator;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.Dictionary;
-
-import com.google.common.base.Objects;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- * Within a partition (per timestampGranularity), records are further sliced
- * (per sliceLength) to fit into HBASE cell.
- * 
- * @author yangli9
- */
-public class Slice implements Iterable<RawTableRecord>, Comparable<Slice> {
-
-    final TableRecordInfoDigest info;
-    final int nColumns;
-
-    final short shard;
-    final long timestamp;
-    final int nRecords;
-    final ColumnValueContainer[] containers;
-    private Dictionary<?>[] localDictionaries;
-
-    public Slice(TableRecordInfoDigest digest, short shard, long timestamp, ColumnValueContainer[] containers) {
-        this.info = digest;
-        this.nColumns = digest.getColumnCount();
-
-        this.shard = shard;
-        this.timestamp = timestamp;
-        this.nRecords = containers[0].getSize();
-        this.containers = containers;
-
-        assert nColumns == containers.length;
-        for (int i = 0; i < nColumns; i++) {
-            assert nRecords == containers[i].getSize();
-        }
-    }
-
-    public Dictionary<?>[] getLocalDictionaries() {
-        return localDictionaries;
-    }
-
-    public void setLocalDictionaries(Dictionary<?>[] localDictionaries) {
-        this.localDictionaries = localDictionaries;
-    }
-
-    public int getRecordCount() {
-        return this.nRecords;
-    }
-
-    public short getShard() {
-        return shard;
-    }
-
-    public long getTimestamp() {
-        return timestamp;
-    }
-
-    public ColumnValueContainer[] getColumnValueContainers() {
-        return containers;
-    }
-
-    public ColumnValueContainer getColumnValueContainer(int col) {
-        return containers[col];
-    }
-
-    public Iterator<RawTableRecord> iterateWithBitmap(final ConciseSet resultBitMap) {
-        if (resultBitMap == null) {
-            return this.iterator();
-        } else {
-            final RawTableRecord rec = info.createTableRecordBytes();
-            final ImmutableBytesWritable temp = new ImmutableBytesWritable();
-
-            return new Iterator<RawTableRecord>() {
-                int i = 0;
-                int iteratedCount = 0;
-                int resultSize = resultBitMap.size();
-
-                @Override
-                public boolean hasNext() {
-                    return iteratedCount < resultSize;
-                }
-
-                @Override
-                public RawTableRecord next() {
-                    while (!resultBitMap.contains(i)) {
-                        i++;
-                    }
-                    for (int col = 0; col < nColumns; col++) {
-                        containers[col].getValueAt(i, temp);
-                        rec.setValueBytes(col, temp);
-                    }
-                    iteratedCount++;
-                    i++;
-
-                    return rec;
-                }
-
-                @Override
-                public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-
-            };
-        }
-    }
-
-    @Override
-    public Iterator<RawTableRecord> iterator() {
-        return new Iterator<RawTableRecord>() {
-            int i = 0;
-            RawTableRecord rec = info.createTableRecordBytes();
-            ImmutableBytesWritable temp = new ImmutableBytesWritable();
-
-            @Override
-            public boolean hasNext() {
-                return i < nRecords;
-            }
-
-            @Override
-            public RawTableRecord next() {
-                for (int col = 0; col < nColumns; col++) {
-                    containers[col].getValueAt(i, temp);
-                    rec.setValueBytes(col, temp);
-                }
-                i++;
-                return rec;
-            }
-
-            @Override
-            public void remove() {
-                throw new UnsupportedOperationException();
-            }
-
-        };
-    }
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see java.lang.Object#hashCode()
-     */
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((info == null) ? 0 : info.hashCode());
-        result = prime * result + shard;
-        result = prime * result + (int) (timestamp ^ (timestamp >>> 32));
-        return result;
-    }
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see java.lang.Object#equals(java.lang.Object)
-     */
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj) {
-            return true;
-        }
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-        Slice other = (Slice) obj;
-        if (shard != other.shard) {
-            return false;
-        }
-        if (timestamp != other.timestamp) {
-            return false;
-        }
-        return Objects.equal(info, other.info);
-    }
-
-    @Override
-    public int compareTo(Slice o) {
-        int comp = this.shard - o.shard;
-        if (comp != 0)
-            return comp;
-
-        comp = (int) (this.timestamp - o.timestamp);
-        return comp;
-    }
-
-    public TableRecordInfoDigest getInfo() {
-        return info;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/SliceBuilder.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/SliceBuilder.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/SliceBuilder.java
deleted file mode 100644
index 0076919..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/SliceBuilder.java
+++ /dev/null
@@ -1,78 +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.index;
-
-import java.io.IOException;
-import java.util.List;
-
-import javax.annotation.Nullable;
-
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.common.util.StreamingBatch;
-import org.apache.kylin.common.util.StreamingMessage;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.invertedindex.util.IIDictionaryBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-
-/**
- */
-public final class SliceBuilder {
-
-    private static Logger logger = LoggerFactory.getLogger(SliceBuilder.class);
-
-    private final BatchSliceMaker sliceMaker;
-    private final IIDesc iiDesc;
-
-    public SliceBuilder(IIDesc desc, short shard) {
-        this.iiDesc = desc;
-        this.sliceMaker = new BatchSliceMaker(desc, shard);
-    }
-
-    public Slice buildSlice(StreamingBatch microStreamBatch) throws IOException {
-        final List<List<String>> messages = Lists.transform(microStreamBatch.getMessages(), new Function<StreamingMessage, List<String>>() {
-            @Nullable
-            @Override
-            public List<String> apply(@Nullable StreamingMessage input) {
-                return input.getData();
-            }
-        });
-        final Dictionary<?>[] dictionaries = IIDictionaryBuilder.buildDictionary(messages, iiDesc);
-        TableRecordInfo tableRecordInfo = new TableRecordInfo(iiDesc, dictionaries);
-        return build(messages, tableRecordInfo, dictionaries);
-    }
-
-    private Slice build(List<List<String>> table, final TableRecordInfo tableRecordInfo, Dictionary<?>[] localDictionary) {
-        final Slice slice = sliceMaker.makeSlice(tableRecordInfo.getDigest(), Lists.transform(table, new Function<List<String>, TableRecord>() {
-            @Nullable
-            @Override
-            public TableRecord apply(@Nullable List<String> input) {
-                TableRecord result = tableRecordInfo.createTableRecord();
-                for (int i = 0; i < input.size(); i++) {
-                    result.setValueString(i, input.get(i));
-                }
-                return result;
-            }
-        }));
-        slice.setLocalDictionaries(localDictionary);
-        return slice;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
deleted file mode 100644
index 3ee34be..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
+++ /dev/null
@@ -1,193 +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.index;
-
-import java.util.Arrays;
-
-import org.apache.commons.lang.ObjectUtils;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.common.util.ShardingHash;
-import org.apache.kylin.metadata.datatype.LongMutable;
-
-/**
- * TableRecord extends RawTableRecord by decorating it with a TableRecordInfo.
- * 
- * @author yangli9, honma
- */
-public class TableRecord implements Cloneable {
-
-    private TableRecordInfo info;
-    private RawTableRecord rawRecord;
-
-    public static final byte ROWKEY_PLACE_HOLDER_BYTE = 9;
-
-    public TableRecord(RawTableRecord rawRecord, TableRecordInfo info) {
-        this.info = info;
-        this.rawRecord = rawRecord;
-    }
-
-    public TableRecord(TableRecord another) {
-        this.info = another.info;
-        this.rawRecord = (RawTableRecord) another.rawRecord.clone();
-    }
-
-    public TableRecordInfo getInfo() {
-        return info;
-    }
-
-    @Override
-    public Object clone() {
-        return new TableRecord(this);
-    }
-
-    public void reset() {
-        rawRecord.reset();
-    }
-
-    public byte[] getBytes() {
-        return rawRecord.getBytes();
-    }
-
-    public void setBytes(byte[] bytes, int offset, int length) {
-        rawRecord.setBytes(bytes, offset, length);
-    }
-
-    public long getTimestamp() {
-        String str = getValueString(info.getTimestampColumn());
-        return DateFormat.stringToMillis(str);
-    }
-
-    public int length(int col) {
-        return rawRecord.length(col);
-    }
-
-    public void setValueStringWithoutDictionary(int col, String value) {
-        int offset = info.digest.offset(col);
-        int length = info.digest.length(col);
-        byte[] src = value.getBytes();
-        if (length >= src.length) {
-            byte[] dst = rawRecord.getBytes();
-            System.arraycopy(src, 0, dst, offset, src.length);
-            Arrays.fill(dst, offset + src.length, offset + length, ROWKEY_PLACE_HOLDER_BYTE);
-        } else {
-            byte[] dst = rawRecord.getBytes();
-            System.arraycopy(src, 0, dst, offset, length);
-        }
-    }
-
-    public String getValueStringWithoutDictionary(int col) {
-        int offset = info.digest.offset(col);
-        int length = info.digest.length(col);
-        byte[] bytes = rawRecord.getBytes();
-        int i;
-        for (i = 0; i < length; ++i) {
-            if (bytes[offset + i] == ROWKEY_PLACE_HOLDER_BYTE) {
-                break;
-            }
-        }
-        return new String(bytes, offset, i);
-    }
-
-    public void setValueString(int col, String value) {
-        if (rawRecord.isMetric(col)) {
-            LongMutable v = rawRecord.codec(col).valueOf(value);
-            setValueMetrics(col, v);
-        } else {
-            final Dictionary<String> dict = info.dict(col);
-            if (dict != null) {
-                int id = dict.getIdFromValue(value);
-                rawRecord.setValueID(col, id);
-            } else {
-                setValueStringWithoutDictionary(col, value);
-                //                throw new UnsupportedOperationException("cannot set value when there is no dictionary");
-            }
-        }
-    }
-
-    /**
-     * get value of columns which belongs to the original table columns.
-     * i.e. columns like min_xx, max_yy will never appear
-     */
-    public String getValueString(int col) {
-        if (rawRecord.isMetric(col)) {
-            return getValueMetric(col);
-        } else {
-            final Dictionary<String> dict = info.dict(col);
-            if (dict != null) {
-                return dict.getValueFromId(rawRecord.getValueID(col));
-            } else {
-                return getValueStringWithoutDictionary(col);
-                //                throw new UnsupportedOperationException("cannot get value when there is no dictionary");
-            }
-        }
-    }
-
-    public void getValueBytes(int col, ImmutableBytesWritable bytes) {
-        rawRecord.getValueBytes(col, bytes);
-    }
-
-    private void setValueMetrics(int col, LongMutable value) {
-        rawRecord.setValueMetrics(col, value);
-    }
-
-    private String getValueMetric(int col) {
-        return rawRecord.getValueMetric(col);
-    }
-
-    public short getShard() {
-        int timestampID = rawRecord.getValueID(info.getTimestampColumn());
-        return ShardingHash.getShard(timestampID, info.getDescriptor().getSharding());
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder buf = new StringBuilder("[");
-        for (int col = 0; col < rawRecord.getColumnCount(); col++) {
-            if (col > 0)
-                buf.append(",");
-            buf.append(getValueString(col));
-        }
-        buf.append("]");
-        return buf.toString();
-    }
-
-    @Override
-    public int hashCode() {
-        if (rawRecord != null) {
-            return rawRecord.hashCode();
-        } else {
-            return 0;
-        }
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        TableRecord other = (TableRecord) obj;
-        return ObjectUtils.equals(other.rawRecord, this.rawRecord);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
deleted file mode 100644
index 628a08d..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
+++ /dev/null
@@ -1,147 +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.index;
-
-import java.util.List;
-
-import org.apache.kylin.common.util.Array;
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.model.TblColRef;
-
-/**
- * @author yangli9
- *         <p/>
- *         TableRecordInfo stores application-aware knowledges, while
- *         TableRecordInfoDigest only stores byte level knowleges
- */
-public class TableRecordInfo {
-
-    final IIDesc desc;
-
-    final TableRecordInfoDigest digest;
-    final Dictionary<?>[] dictionaries;
-
-    public TableRecordInfo(IISegment iiSegment) {
-        this(iiSegment.getIIDesc());
-    }
-
-    public TableRecordInfo(IIDesc desc) {
-        this(desc, new Dictionary<?>[desc.listAllColumns().size()]);
-    }
-
-    public TableRecordInfo(IIDesc desc, Dictionary<?>[] dictionaries) {
-        this.desc = desc;
-        this.dictionaries = dictionaries;
-        this.digest = createDigest(desc, dictionaries);
-    }
-
-    public TableRecordInfoDigest getDigest() {
-        return digest;
-    }
-
-    private TableRecordInfoDigest createDigest(IIDesc desc, Dictionary<?>[] dictionaryMap) {
-        final List<TblColRef> tblColRefs = desc.listAllColumns();
-        final int nColumns = tblColRefs.size();
-        boolean[] isMetric = new boolean[nColumns];
-        int[] lengths = new int[nColumns];
-        int[] dictMaxIds = new int[nColumns];
-        String[] dataTypes = new String[nColumns];
-        for (int i = 0; i < nColumns; ++i) {
-            final TblColRef tblColRef = tblColRefs.get(i);
-            isMetric[i] = desc.isMetricsCol(i);
-            dataTypes[i] = tblColRef.getDatatype();
-            if (isMetric[i]) {
-                lengths[i] = FixedLenMeasureCodec.get(DataType.getType(tblColRef.getColumnDesc().getDatatype())).getLength();
-            } else {
-                if (Array.isEmpty(dictionaryMap)) {
-                    final DataType dataType = DataType.getType(tblColRef.getColumnDesc().getDatatype());
-                    if (dataType.isNumberFamily()) {
-                        lengths[i] = 16;
-                    } else if (dataType.isStringFamily()) {
-                        lengths[i] = 256;
-                    } else if (dataType.isDateTimeFamily()) {
-                        lengths[i] = 19;
-                    } else {
-                        throw new RuntimeException("invalid data type:" + dataType);
-                    }
-                    dictMaxIds[i] = Integer.MAX_VALUE;
-                } else {
-                    final Dictionary<?> dictionary = dictionaryMap[i];
-                    lengths[i] = dictionary.getSizeOfId();
-                    dictMaxIds[i] = dictionary.getMaxId();
-                }
-            }
-        }
-        // offsets
-        int pos = 0;
-        int[] offsets = new int[nColumns];
-        for (int i = 0; i < nColumns; i++) {
-            offsets[i] = pos;
-            pos += lengths[i];
-        }
-
-        int byteFormLen = pos;
-
-        return new TableRecordInfoDigest(nColumns, byteFormLen, offsets, dictMaxIds, lengths, isMetric, dataTypes);
-    }
-
-    public TableRecord createTableRecord() {
-        return new TableRecord(digest.createTableRecordBytes(), this);
-    }
-
-    public final IIDesc getDescriptor() {
-        return desc;
-    }
-
-    public final List<TblColRef> getColumns() {
-        return desc.listAllColumns();
-    }
-
-    public int findColumn(TblColRef col) {
-        return desc.findColumn(col);
-    }
-
-    public int findFactTableColumn(String columnName) {
-        if (columnName == null)
-            return -1;
-        for (int i = 0; i < getColumns().size(); ++i) {
-            TblColRef tblColRef = getColumns().get(i);
-            if (tblColRef.isSameAs(desc.getFactTableName(), columnName)) {
-                return i;
-            }
-        }
-        return -1;
-    }
-
-    // dimensions go with dictionary
-    @SuppressWarnings("unchecked")
-    public Dictionary<String> dict(int col) {
-        // yes, all dictionaries are string based
-        return (Dictionary<String>) dictionaries[col];
-    }
-
-    public int getTimestampColumn() {
-        return desc.getTimestampColumn();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
deleted file mode 100644
index 9eebdbe..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
+++ /dev/null
@@ -1,175 +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.index;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.kylin.common.util.BytesSerializer;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.LongMutable;
-
-import com.google.common.base.Objects;
-
-/**
- */
-public class TableRecordInfoDigest {
-
-    private String[] metricDataTypes;
-    private int nColumns;
-    private int byteFormLen;
-
-    private int[] offsets;// column offset in byte form row
-    private int[] dictMaxIds;// max id for each of the dict
-    private int[] lengths;// length of each encoded dict
-    private boolean[] isMetric;// whether it's metric or dict
-    private FixedLenMeasureCodec<?>[] measureCodecs;
-
-    public TableRecordInfoDigest(int nColumns, int byteFormLen, //
-            int[] offsets, int[] dictMaxIds, int[] lengths, boolean[] isMetric, String[] metricDataTypes) {
-        this.nColumns = nColumns;
-        this.byteFormLen = byteFormLen;
-        this.offsets = offsets;
-        this.dictMaxIds = dictMaxIds;
-        this.lengths = lengths;
-        this.isMetric = isMetric;
-        this.metricDataTypes = metricDataTypes;
-        this.measureCodecs = new FixedLenMeasureCodec[nColumns];
-        for (int i = 0; i < isMetric.length; i++) {
-            if (isMetric[i]) {
-                measureCodecs[i] = FixedLenMeasureCodec.get(DataType.getType(metricDataTypes[i]));
-            }
-        }
-    }
-
-    private TableRecordInfoDigest() {
-    }
-
-    public int getByteFormLen() {
-        return byteFormLen;
-    }
-
-    public boolean isMetrics(int col) {
-        return isMetric[col];
-    }
-
-    public boolean[] isMetrics() {
-        return isMetric;
-    }
-
-    public int getColumnCount() {
-        return nColumns;
-    }
-
-    public int offset(int col) {
-        return offsets[col];
-    }
-
-    public int length(int col) {
-        return lengths[col];
-    }
-
-    public int getMaxID(int col) {
-        return dictMaxIds[col];
-    }
-
-    public boolean[] getIsMetric() {
-        return isMetric;
-    }
-
-    public String[] getMetricDataTypes() {
-        return metricDataTypes;
-    }
-
-    public RawTableRecord createTableRecordBytes() {
-        return new RawTableRecord(this);
-    }
-
-    @Override
-    public int hashCode() {
-        return com.google.common.base.Objects.hashCode(nColumns, offsets, dictMaxIds, lengths, isMetric, metricDataTypes);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj) {
-            return true;
-        }
-        if (obj instanceof TableRecordInfoDigest) {
-            TableRecordInfoDigest other = (TableRecordInfoDigest) obj;
-            return Objects.equal(this.nColumns, other.nColumns) && Arrays.equals(this.offsets, other.offsets) && Arrays.equals(this.dictMaxIds, other.dictMaxIds) && Arrays.equals(this.lengths, other.lengths) && Arrays.equals(this.isMetric, other.isMetric) && Arrays.equals(this.metricDataTypes, other.metricDataTypes);
-        } else {
-            return false;
-        }
-    }
-
-    // metrics go with fixed-len codec
-    @SuppressWarnings("unchecked")
-    public FixedLenMeasureCodec<LongMutable> codec(int col) {
-        // yes, all metrics are long currently
-        return (FixedLenMeasureCodec<LongMutable>) measureCodecs[col];
-    }
-
-    public static byte[] serialize(TableRecordInfoDigest o) {
-        ByteBuffer buf = ByteBuffer.allocate(BytesSerializer.SERIALIZE_BUFFER_SIZE);
-        serializer.serialize(o, buf);
-        byte[] result = new byte[buf.position()];
-        System.arraycopy(buf.array(), 0, result, 0, buf.position());
-        return result;
-    }
-
-    public static TableRecordInfoDigest deserialize(byte[] bytes) {
-        return serializer.deserialize(ByteBuffer.wrap(bytes));
-    }
-
-    public static TableRecordInfoDigest deserialize(ByteBuffer buffer) {
-        return serializer.deserialize(buffer);
-    }
-
-    private static final BytesSerializer<TableRecordInfoDigest> serializer = new BytesSerializer<TableRecordInfoDigest>() {
-
-        @Override
-        public void serialize(TableRecordInfoDigest value, ByteBuffer out) {
-            BytesUtil.writeVInt(value.nColumns, out);
-            BytesUtil.writeVInt(value.byteFormLen, out);
-            BytesUtil.writeIntArray(value.offsets, out);
-            BytesUtil.writeIntArray(value.dictMaxIds, out);
-            BytesUtil.writeIntArray(value.lengths, out);
-            BytesUtil.writeBooleanArray(value.isMetric, out);
-            BytesUtil.writeAsciiStringArray(value.metricDataTypes, out);
-
-        }
-
-        @Override
-        public TableRecordInfoDigest deserialize(ByteBuffer in) {
-            TableRecordInfoDigest result = new TableRecordInfoDigest();
-            result.nColumns = BytesUtil.readVInt(in);
-            result.byteFormLen = BytesUtil.readVInt(in);
-            result.offsets = BytesUtil.readIntArray(in);
-            result.dictMaxIds = BytesUtil.readIntArray(in);
-            result.lengths = BytesUtil.readIntArray(in);
-            result.isMetric = BytesUtil.readBooleanArray(in);
-            result.metricDataTypes = BytesUtil.readAsciiStringArray(in);
-            return result;
-        }
-
-    };
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
deleted file mode 100644
index 0ec2638..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
+++ /dev/null
@@ -1,78 +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.measure;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
-import org.apache.kylin.metadata.datatype.DataType;
-
-/**
- */
-public class FixedHLLCodec extends FixedLenMeasureCodec<HyperLogLogPlusCounter> {
-
-    private DataType type;
-    private int presision;
-    private HyperLogLogPlusCounter current;
-
-    public FixedHLLCodec(DataType type) {
-        this.type = type;
-        this.presision = type.getPrecision();
-        this.current = new HyperLogLogPlusCounter(this.presision);
-    }
-
-    @Override
-    public int getLength() {
-        return 1 << presision;
-    }
-
-    @Override
-    public DataType getDataType() {
-        return type;
-    }
-
-    @Override
-    public HyperLogLogPlusCounter valueOf(String value) {
-        current.clear();
-        if (value != null)
-            current.add(value.getBytes());
-        return current;
-    }
-
-    @Override
-    public Object getValue() {
-        return current;
-    }
-
-    @Override
-    public HyperLogLogPlusCounter read(byte[] buf, int offset) {
-        current.readRegistersArray(ByteBuffer.wrap(buf, offset, buf.length - offset));
-        return current;
-    }
-
-    @Override
-    public void write(HyperLogLogPlusCounter v, byte[] buf, int offset) {
-        v.writeRegistersArray(ByteBuffer.wrap(buf, offset, buf.length - offset));
-    }
-
-    @Override
-    public HyperLogLogPlusCounter read(ByteBuffer buffer) {
-        current.readRegistersArray(buffer);
-        return current;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
deleted file mode 100644
index bd952a1..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.invertedindex.measure;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.measure.hllc.HLLCMeasureType;
-import org.apache.kylin.metadata.datatype.DataType;
-
-abstract public class FixedLenMeasureCodec<T> {
-
-    public static FixedLenMeasureCodec<?> get(DataType type) {
-        if (HLLCMeasureType.DATATYPE_HLLC.equals(type.getName())) {
-            return new FixedHLLCodec(type);
-        } else {
-            return new FixedPointLongCodec(type);
-        }
-    }
-
-    abstract public int getLength();
-
-    abstract public DataType getDataType();
-
-    abstract public T valueOf(String value);
-
-    abstract public Object getValue();
-
-    abstract public T read(byte[] buf, int offset);
-
-    abstract public void write(T v, byte[] buf, int offset);
-
-    abstract public T read(ByteBuffer buffer);
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
deleted file mode 100644
index 67e5158..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
+++ /dev/null
@@ -1,117 +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.measure;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.LongMutable;
-
-public class FixedPointLongCodec extends FixedLenMeasureCodec<LongMutable> {
-
-    private static final int SIZE = 8;
-    // number of digits after decimal point
-    int scale;
-    DataType type;
-    // avoid massive object creation
-    LongMutable current = new LongMutable();
-
-    public FixedPointLongCodec(DataType type) {
-        this.type = type;
-        this.scale = Math.max(0, type.getScale());
-    }
-
-    @Override
-    public int getLength() {
-        return SIZE;
-    }
-
-    @Override
-    public DataType getDataType() {
-        return type;
-    }
-
-    long getValueIgnoringDecimalPoint(String value) {
-        int index = value.indexOf('.');
-
-        if (index == 0 || index == value.length() - 1) {
-            throw new RuntimeException("Bad decimal format: " + value);
-        } else if (index < 0) {
-            return Long.valueOf(value) * (int) Math.pow(10, scale);
-        } else {
-            StringBuilder sb = new StringBuilder();
-            sb.append(value.substring(0, index));
-
-            //if there are more than scale digits after the decimal point, the tail will be discarded
-            int end = Math.min(value.length(), index + scale + 1);
-            sb.append(value.substring(index + 1, end));
-            int diff = index + scale + 1 - value.length();
-            //if there are less than scale digits after the decimal point, the tail will be compensated
-            for (int i = 0; i < diff; i++) {
-                sb.append('0');
-            }
-            return Long.valueOf(sb.toString());
-        }
-    }
-
-    String restoreDecimalPoint(long value) {
-        if (scale < 0) {
-            throw new RuntimeException("Bad scale: " + scale + " with value: " + value);
-        } else if (scale == 0) {
-            return Long.toString(value);
-        } else {
-            return String.format("%." + scale + "f", value / (Math.pow(10, scale)));
-        }
-    }
-
-    @Override
-    public LongMutable valueOf(String value) {
-        if (value == null)
-            current.set(0L);
-        else
-            current.set(getValueIgnoringDecimalPoint(value));
-        return current;
-    }
-
-    @Override
-    public String getValue() {
-        if (scale == 0)
-            return current.toString();
-        else
-            return restoreDecimalPoint(current.get());
-    }
-
-    @Override
-    public LongMutable read(byte[] buf, int offset) {
-        current.set(BytesUtil.readLong(buf, offset, SIZE));
-        return current;
-    }
-
-    @Override
-    public void write(LongMutable v, byte[] buf, int offset) {
-        BytesUtil.writeLong(v == null ? 0 : v.get(), buf, offset, SIZE);
-    }
-
-    @Override
-    public LongMutable read(ByteBuffer buffer) {
-        current.set(BytesUtil.readLong(buffer, SIZE));
-        return current;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java
deleted file mode 100644
index 66fb67d..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java
+++ /dev/null
@@ -1,418 +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.model;
-
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.commons.net.util.Base64;
-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.util.Bytes;
-import org.apache.kylin.common.util.JsonUtil;
-import org.apache.kylin.common.util.StringUtil;
-import org.apache.kylin.metadata.MetadataConstants;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.DataModelDesc;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.IEngineAware;
-import org.apache.kylin.metadata.model.IStorageAware;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.ModelDimensionDesc;
-import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
-/**
- * @author yangli9
- */
-@SuppressWarnings("serial")
-@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class IIDesc extends RootPersistentEntity {
-
-    public static final String HBASE_FAMILY = "f";
-    public static final String HBASE_QUALIFIER = "c";
-    public static final String HBASE_DICTIONARY = "d";
-    public static final byte[] HBASE_FAMILY_BYTES = Bytes.toBytes(HBASE_FAMILY);
-    public static final byte[] HBASE_QUALIFIER_BYTES = Bytes.toBytes(HBASE_QUALIFIER);
-    public static final byte[] HBASE_DICTIONARY_BYTES = Bytes.toBytes(HBASE_DICTIONARY);
-
-    private KylinConfig config;
-    private DataModelDesc model;
-
-    @JsonProperty("name")
-    private String name;
-    @JsonProperty("model_name")
-    private String modelName;
-    @JsonProperty("timestamp_dimension")
-    private String timestampDimension;
-    @JsonProperty("bitmap_dimensions")
-    private List<ModelDimensionDesc> bitmapDimensions = Collections.emptyList();
-    @JsonProperty("value_dimensions")
-    private List<ModelDimensionDesc> valueDimensions;
-    @JsonProperty("metrics")
-    private String[] metricNames;
-    @JsonProperty("sharding")
-    private short sharding = 1; // parallelism
-    @JsonProperty("slice_size")
-    private int sliceSize = 50000; // no. rows
-
-    @JsonProperty("engine_type")
-    private int engineType = IEngineAware.ID_MR_II;
-
-    @JsonProperty("storage_type")
-    private int storageType = IStorageAware.ID_HBASE;
-
-    @JsonProperty("signature")
-    private String signature;
-
-    // computed
-    private List<TableDesc> allTables = Lists.newArrayList();
-    private List<TblColRef> allColumns = Lists.newArrayList();
-    private List<TblColRef> allDimensions = Lists.newArrayList();
-    private int tsCol;
-    private int[] bitmapCols;
-    private int[] valueCols;
-    private int[] metricsCols;
-    private BitSet metricsColSet;
-    private List<MeasureDesc> measureDescs;
-
-    public void init(MetadataManager metadataManager) {
-
-        config = metadataManager.getConfig();
-
-        if (this.modelName == null || this.modelName.length() == 0) {
-            throw new RuntimeException("The cubeDesc '" + this.getName() + "' doesn't have data model specified.");
-        }
-
-        this.model = MetadataManager.getInstance(config).getDataModelDesc(this.modelName);
-
-        if (this.model == null) {
-            throw new RuntimeException("No data model found with name '" + modelName + "'.");
-        }
-
-        timestampDimension = timestampDimension.toUpperCase();
-
-        // capitalize
-        ModelDimensionDesc.capicalizeStrings(bitmapDimensions);
-        ModelDimensionDesc.capicalizeStrings(valueDimensions);
-        StringUtil.toUpperCaseArray(metricNames, metricNames);
-
-        // retrieve all columns and all tables, and make available measure to ii
-        HashSet<String> allTableNames = Sets.newHashSet();
-        measureDescs = Lists.newArrayList();
-        measureDescs.add(makeCountMeasure());
-        for (ModelDimensionDesc modelDimensionDesc : Iterables.concat(bitmapDimensions, valueDimensions)) {
-            TableDesc tableDesc = this.getTableDesc(modelDimensionDesc.getTable());
-            for (String column : modelDimensionDesc.getColumns()) {
-                ColumnDesc columnDesc = tableDesc.findColumnByName(column);
-                TblColRef tcr = columnDesc.getRef();
-                allColumns.add(tcr);
-                allDimensions.add(tcr);
-                measureDescs.add(makeHLLMeasure(columnDesc, "hllc10"));
-            }
-
-            if (!allTableNames.contains(tableDesc.getIdentity())) {
-                allTableNames.add(tableDesc.getIdentity());
-                allTables.add(tableDesc);
-            }
-        }
-        for (String column : metricNames) {
-            TableDesc tableDesc = this.getTableDesc(this.getFactTableName());
-            ColumnDesc columnDesc = tableDesc.findColumnByName(column);
-            allColumns.add(columnDesc.getRef());
-            measureDescs.add(makeNormalMeasure("SUM", columnDesc));
-            measureDescs.add(makeNormalMeasure("MIN", columnDesc));
-            measureDescs.add(makeNormalMeasure("MAX", columnDesc));
-            if (!allTableNames.contains(tableDesc.getIdentity())) {
-                allTableNames.add(tableDesc.getIdentity());
-                allTables.add(tableDesc);
-            }
-        }
-
-        // indexing for each type of columns
-        bitmapCols = new int[ModelDimensionDesc.getColumnCount(bitmapDimensions)];
-        valueCols = new int[ModelDimensionDesc.getColumnCount(valueDimensions)];
-        metricsCols = new int[metricNames.length];
-        metricsColSet = new BitSet(this.getTableDesc(this.getFactTableName()).getColumnCount());
-
-        int totalIndex = 0;
-        for (int i = 0; i < bitmapCols.length; ++i, ++totalIndex) {
-            bitmapCols[i] = totalIndex;
-        }
-        for (int i = 0; i < valueCols.length; ++i, ++totalIndex) {
-            valueCols[i] = totalIndex;
-        }
-        for (int i = 0; i < metricsCols.length; ++i, ++totalIndex) {
-            metricsCols[i] = totalIndex;
-            metricsColSet.set(totalIndex);
-        }
-
-        // partitioning column
-        tsCol = -1;
-        for (int i = 0; i < allColumns.size(); ++i) {
-            TblColRef col = allColumns.get(i);
-
-            if (col.isSameAs(this.getFactTableName(), this.timestampDimension)) {
-                tsCol = i;
-                break;
-            }
-        }
-        if (tsCol < 0)
-            throw new RuntimeException("timestamp_dimension is not in bitmapDimensions or valueDimensions");
-    }
-
-    private TableDesc getTableDesc(String tableName) {
-        return MetadataManager.getInstance(this.config).getTableDesc(tableName);
-    }
-
-    public String getResourcePath() {
-        return getIIDescResourcePath(name);
-    }
-
-    public static String getIIDescResourcePath(String descName) {
-        return ResourceStore.II_DESC_RESOURCE_ROOT + "/" + descName + MetadataConstants.FILE_SURFIX;
-    }
-
-    public List<MeasureDesc> getMeasures() {
-        return measureDescs;
-    }
-
-    public List<FunctionDesc> listAllFunctions() {
-        List<FunctionDesc> functions = new ArrayList<FunctionDesc>();
-        for (MeasureDesc m : measureDescs) {
-            functions.add(m.getFunction());
-        }
-        return functions;
-    }
-
-    private MeasureDesc makeNormalMeasure(String func, ColumnDesc columnDesc) {
-        String columnName = columnDesc.getName();
-        String returnType = columnDesc.getTypeName();
-        MeasureDesc measureDesc = new MeasureDesc();
-        FunctionDesc f1 = new FunctionDesc();
-        f1.setExpression(func);
-        ParameterDesc p1 = new ParameterDesc();
-        p1.setType("column");
-        p1.setValue(columnName);
-        p1.setColRefs(ImmutableList.of(columnDesc.getRef()));
-        f1.setParameter(p1);
-        f1.setReturnType(returnType);
-        if (f1.isSum() && f1.getReturnDataType().isIntegerFamily()) {
-            f1.setReturnType("bigint");
-        }
-
-        measureDesc.setFunction(f1);
-        measureDesc.setName(func + "_" + columnName);
-        return measureDesc;
-    }
-
-    /**
-     * 
-     * @param hllType represents the precision
-     */
-    private MeasureDesc makeHLLMeasure(ColumnDesc columnDesc, String hllType) {
-        String columnName = columnDesc.getName();
-        MeasureDesc measureDesc = new MeasureDesc();
-        FunctionDesc f1 = new FunctionDesc();
-        f1.setExpression("COUNT_DISTINCT");
-        ParameterDesc p1 = new ParameterDesc();
-        p1.setType("column");
-        p1.setValue(columnName);
-        p1.setColRefs(ImmutableList.of(columnDesc.getRef()));
-        f1.setParameter(p1);
-        f1.setReturnType(hllType);
-        measureDesc.setFunction(f1);
-        measureDesc.setName("COUNT_DISTINCT" + "_" + columnName);
-        return measureDesc;
-    }
-
-    private MeasureDesc makeCountMeasure() {
-        MeasureDesc measureDesc = new MeasureDesc();
-        FunctionDesc f1 = new FunctionDesc();
-        f1.setExpression("COUNT");
-        ParameterDesc p1 = new ParameterDesc();
-        p1.setType("constant");
-        p1.setValue("1");
-        f1.setParameter(p1);
-        f1.setReturnType("bigint");
-        measureDesc.setFunction(f1);
-        measureDesc.setName("COUNT_1");
-        return measureDesc;
-    }
-
-    /**
-     * at first stage the only table in II is fact table, tables
-     * 
-     * @return
-     */
-    public List<TableDesc> listTables() {
-        return allTables;
-    }
-
-    public List<TblColRef> listAllColumns() {
-        return allColumns;
-    }
-
-    public List<TblColRef> listAllDimensions() {
-        return allDimensions;
-    }
-
-    public TblColRef findColumnRef(String table, String column) {
-        ColumnDesc columnDesc = this.getTableDesc(table).findColumnByName(column);
-        return columnDesc.getRef();
-    }
-
-    public int findColumn(TblColRef col) {
-        return this.allColumns.indexOf(col);
-    }
-
-    public KylinConfig getConfig() {
-        return config;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public String getModelName() {
-        return modelName;
-    }
-
-    public void setModelName(String modelName) {
-        this.modelName = modelName;
-    }
-
-    public DataModelDesc getModel() {
-        return model;
-    }
-
-    public void setModel(DataModelDesc model) {
-        this.model = model;
-    }
-
-    public int getTimestampColumn() {
-        return tsCol;
-    }
-
-    public int[] getBitmapColumns() {
-        return bitmapCols;
-    }
-
-    public int[] getValueColumns() {
-        return valueCols;
-    }
-
-    public int[] getMetricsColumns() {
-        return metricsCols;
-    }
-
-    public short getSharding() {
-        return sharding;
-    }
-
-    public int getSliceSize() {
-        return sliceSize;
-    }
-
-    public String getSignature() {
-        return signature;
-    }
-
-    public void setSignature(String signature) {
-        this.signature = signature;
-    }
-
-    public boolean isMetricsCol(TblColRef col) {
-        if (!col.getTable().equalsIgnoreCase(this.getFactTableName()))
-            return false;
-        return isMetricsCol(this.findColumn(col));
-    }
-
-    public boolean isMetricsCol(int index) {
-        return metricsColSet.get(index);
-    }
-
-    /**
-     * the returned fact table name is guaranteed to be in the form of db.table
-     * 
-     * @return
-     */
-    public String getFactTableName() {
-        return this.model.getFactTable().toUpperCase();
-    }
-
-    public String getTimestampDimension() {
-        return timestampDimension;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String calculateSignature() {
-        MessageDigest md = null;
-        try {
-            md = MessageDigest.getInstance("MD5");
-            StringBuilder sigString = new StringBuilder();
-            sigString.append(this.name).append("|").append(this.getFactTableName()).append("|").append(timestampDimension).append("|").append(JsonUtil.writeValueAsString(this.bitmapDimensions)).append("|").append(JsonUtil.writeValueAsString(valueDimensions)).append("|").append(JsonUtil.writeValueAsString(this.metricNames)).append("|").append(sharding).append("|").append(sliceSize);
-
-            byte[] signature = md.digest(sigString.toString().getBytes());
-            return new String(Base64.encodeBase64(signature));
-        } catch (NoSuchAlgorithmException e) {
-            throw new RuntimeException("Failed to calculate signature");
-        } catch (JsonProcessingException e) {
-            throw new RuntimeException("Failed to calculate signature");
-        }
-
-    }
-
-    public int getStorageType() {
-        return storageType;
-    }
-
-    public void setStorageType(int storageType) {
-        this.storageType = storageType;
-    }
-
-    public int getEngineType() {
-        return engineType;
-    }
-
-    public void setEngineType(int engineType) {
-        this.engineType = engineType;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIJoinedFlatTableDesc.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIJoinedFlatTableDesc.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIJoinedFlatTableDesc.java
deleted file mode 100644
index 21e5677..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIJoinedFlatTableDesc.java
+++ /dev/null
@@ -1,68 +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.model;
-
-import java.util.List;
-
-import org.apache.kylin.metadata.model.DataModelDesc;
-import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
-import org.apache.kylin.metadata.model.IntermediateColumnDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-public class IIJoinedFlatTableDesc implements IJoinedFlatTableDesc {
-
-    private IIDesc iiDesc;
-    private String tableName;
-    private List<IntermediateColumnDesc> columnList = Lists.newArrayList();
-
-    public IIJoinedFlatTableDesc(IIDesc iiDesc) {
-        this.iiDesc = iiDesc;
-        parseIIDesc();
-    }
-
-    private void parseIIDesc() {
-        this.tableName = "kylin_intermediate_ii_" + iiDesc.getName();
-
-        int columnIndex = 0;
-        for (TblColRef col : iiDesc.listAllColumns()) {
-            columnList.add(new IntermediateColumnDesc(String.valueOf(columnIndex), col));
-            columnIndex++;
-        }
-    }
-
-    @Override
-    public String getTableName() {
-        return tableName + "_" + "II_Flat";
-    }
-
-    @Override
-    public List<IntermediateColumnDesc> getColumnList() {
-        return columnList;
-    }
-
-    @Override
-    public DataModelDesc getDataModel() {
-        return iiDesc.getModel();
-    }
-
-}


[25/55] [abbrv] kylin git commit: KYLIN-1894 fix test and cleanup tmp files

Posted by sh...@apache.org.
KYLIN-1894 fix test and cleanup tmp files

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


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 28a2b9f7eadc05fdf4e0a9e268b0adb142bdfccf
Parents: f0db336
Author: sunyerui <su...@gmail.com>
Authored: Mon Jul 18 17:50:12 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 18 18:37:46 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/dict/AppendTrieDictionary.java |  2 +-
 .../kylin/dict/AppendTrieDictionaryTest.java    | 16 +++++++--
 .../apache/kylin/dict/CachedTreeMapTest.java    | 34 +++++++++++++-------
 3 files changed, 36 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/28a2b9f7/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
index 4cce586..14980bf 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
@@ -950,8 +950,8 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
             if (dict == null) {
                 dict = new AppendTrieDictionary<T>();
             }
-            dict.flushIndex((CachedTreeMap) mutableDictSliceMap);
             dict.update(baseDir, baseId, maxId, maxValueLength, nValues, bytesConverter, (CachedTreeMap)mutableDictSliceMap);
+            dict.flushIndex((CachedTreeMap) mutableDictSliceMap);
 
             return dict;
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/28a2b9f7/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
index b81a439..4266f2a 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
@@ -26,6 +26,7 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -49,6 +50,8 @@ import org.junit.Test;
  */
 public class AppendTrieDictionaryTest {
 
+    public static final String BASE_DIR = "/tmp/kylin_append_dict";
+
     @BeforeClass
     public static void setUp() {
         KylinConfig.destroyInstance();
@@ -56,7 +59,7 @@ public class AppendTrieDictionaryTest {
         KylinConfig config = KylinConfig.getInstanceFromEnv();
         config.setAppendDictEntrySize(50000);
         config.setAppendDictCacheSize(3);
-        config.setProperty("kylin.hdfs.working.dir", "/tmp/kylin_append_dict");
+        config.setProperty("kylin.hdfs.working.dir", BASE_DIR);
     }
 
     @AfterClass
@@ -66,6 +69,13 @@ public class AppendTrieDictionaryTest {
             FileSystem.get(new Path(workingDir).toUri(), new Configuration()).delete(new Path(workingDir), true);
         } catch (IOException e) {
         }
+        File tmpLocalDir = new File(BASE_DIR);
+        if (tmpLocalDir.exists()) {
+            for (File f : tmpLocalDir.listFiles()) {
+                f.delete();
+            }
+            tmpLocalDir.delete();
+        }
     }
 
     public static final String[] words = new String[] { "paint", "par", "part", "parts", "partition", "partitions", "party", "partie", "parties", "patient", "taste", "tar", "trie", "try", "tries", "\u5b57\u5178", "\u5b57\u5178\u6811", "\u5b57\u6bcd", // non-ascii characters
@@ -122,7 +132,7 @@ public class AppendTrieDictionaryTest {
     @Test
     public void testHugeKeySet() throws IOException {
         BytesConverter converter = new StringBytesConverter();
-        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create("/tmp/kylin_append_dict");
+        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create(BASE_DIR);
         AppendTrieDictionary<String> dict = null;
 
         InputStream is = new FileInputStream("src/test/resources/dict/huge_key");
@@ -152,7 +162,7 @@ public class AppendTrieDictionaryTest {
         }
         BytesConverter converter = new StringBytesConverter();
 
-        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create("/tmp/kylin_append_dict");
+        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create(BASE_DIR);
         AppendTrieDictionary<String> dict = null;
         TreeMap<Integer, String> checkMap = new TreeMap<>();
         int firstAppend = rnd.nextInt(strList.size() / 2);

http://git-wip-us.apache.org/repos/asf/kylin/blob/28a2b9f7/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
index d2af621..3366bf6 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
@@ -3,6 +3,7 @@ package org.apache.kylin.dict;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -81,8 +82,7 @@ public class CachedTreeMapTest {
     public static final String backupDir = "/tmp/kylin_cachedtreemap_test.bak/";
     public static final String tmpDir = "/tmp/kylin_cachedtreemap_test.tmp/";
 
-    @After
-    public void afterTest() {
+    private static void cleanup() {
         File dir = new File(baseDir);
         if (dir.exists()) {
             for (File f : dir.listFiles()) {
@@ -110,9 +110,19 @@ public class CachedTreeMapTest {
         VALUE_WRITE_ERROR_TOGGLE = false;
     }
 
+    @After
+    public void afterTest() {
+        cleanup();
+    }
+
+    @AfterClass
+    public static void tearDown() {
+        cleanup();
+    }
+
     @Test
     public void testCachedTreeMap() throws IOException {
-        CachedTreeMap map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        CachedTreeMap map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(false).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         map.put(Key.of(1), Value.of("a"));
         map.put(Key.of(2), Value.of("b"));
@@ -133,7 +143,7 @@ public class CachedTreeMapTest {
         assertEquals(5, dir.listFiles(new CachedFileFilter()).length);
 
         DataInputStream in = new DataInputStream(new FileInputStream(baseDir+".index"));
-        CachedTreeMap map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        CachedTreeMap map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         map2.readFields(in);
         assertEquals(5, map2.size());
@@ -152,7 +162,7 @@ public class CachedTreeMapTest {
     @Test
     public void testWriteFailed() throws IOException {
         // normal case
-        CachedTreeMap map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        CachedTreeMap map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(false).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         map.put(Key.of(1), Value.of("a"));
         map.put(Key.of(2), Value.of("b"));
@@ -167,14 +177,14 @@ public class CachedTreeMapTest {
         map.commit(false);
 
         DataInputStream in = new DataInputStream(new FileInputStream(baseDir+".index"));
-        CachedTreeMap map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        CachedTreeMap map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         map2.readFields(in);
         assertEquals(3, map2.size());
         assertEquals("a", ((Value)map2.get(Key.of(1))).valueStr);
 
         // suppose write value failed and didn't commit data
-        map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(false).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         VALUE_WRITE_ERROR_TOGGLE = true;
         map.put(Key.of(1), Value.of("aa"));
@@ -191,7 +201,7 @@ public class CachedTreeMapTest {
 
         // read map data should not be modified
         in = new DataInputStream(new FileInputStream(baseDir+".index"));
-        map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         map2.readFields(in);
         assertEquals(3, map2.size());
@@ -203,7 +213,7 @@ public class CachedTreeMapTest {
 
     @Test
     public void testCommit() throws IOException {
-        CachedTreeMap map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        CachedTreeMap map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(false).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         map.put(Key.of(1), Value.of("a"));
         map.put(Key.of(2), Value.of("b"));
@@ -220,7 +230,7 @@ public class CachedTreeMapTest {
         assertFalse(new File(backupDir).exists());
 
         DataInputStream in = new DataInputStream(new FileInputStream(baseDir+".index"));
-        CachedTreeMap map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        CachedTreeMap map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         map2.readFields(in);
         assertEquals(4, map2.size());
@@ -242,7 +252,7 @@ public class CachedTreeMapTest {
         assertEquals(4, new File(baseDir).listFiles(new CachedFileFilter()).length);
 
         in = new DataInputStream(new FileInputStream(baseDir+".index"));
-        map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         map2.readFields(in);
         assertEquals(4, map2.size());
@@ -254,7 +264,7 @@ public class CachedTreeMapTest {
         assertEquals(6, new File(baseDir).listFiles(new CachedFileFilter()).length);
 
         in = new DataInputStream(new FileInputStream(baseDir+".index"));
-        map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+        map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir)
                 .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
         map2.readFields(in);
         assertEquals(6, map2.size());


[55/55] [abbrv] kylin git commit: merge 1.5.3

Posted by sh...@apache.org.
merge 1.5.3


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: b46029fba7e3c5d387acb9ea37e788de5dcaf0be
Parents: 80283a7
Author: shaofengshi <sh...@apache.org>
Authored: Tue Jul 26 11:32:50 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Jul 26 11:32:50 2016 +0800

----------------------------------------------------------------------
 tool/pom.xml | 12 ++++++++++++
 1 file changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b46029fb/tool/pom.xml
----------------------------------------------------------------------
diff --git a/tool/pom.xml b/tool/pom.xml
index 5342336..8a8a088 100644
--- a/tool/pom.xml
+++ b/tool/pom.xml
@@ -42,6 +42,18 @@
 
         <!--Env-->
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-api</artifactId>
+            <version>${yarn.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-common</artifactId>
+            <version>${yarn.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>
             <scope>provided</scope>


[09/55] [abbrv] kylin git commit: KYLIN-1858 bug fix

Posted by sh...@apache.org.
KYLIN-1858 bug fix


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 854ec376c4292a762e2402939fb6e229f8a1b8a4
Parents: 2cc0b9c
Author: Hongbin Ma <ma...@apache.org>
Authored: Fri Jul 8 15:38:10 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Fri Jul 8 17:50:46 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/RealizationRegistryTest.java     | 14 +++++---------
 1 file changed, 5 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/854ec376/assembly/src/test/java/org/apache/kylin/RealizationRegistryTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/RealizationRegistryTest.java b/assembly/src/test/java/org/apache/kylin/RealizationRegistryTest.java
index 7449b67..11723c8 100644
--- a/assembly/src/test/java/org/apache/kylin/RealizationRegistryTest.java
+++ b/assembly/src/test/java/org/apache/kylin/RealizationRegistryTest.java
@@ -18,11 +18,6 @@
 
 package org.apache.kylin;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Set;
-
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.metadata.realization.RealizationRegistry;
@@ -31,6 +26,10 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+
 /**
  */
 public class RealizationRegistryTest extends LocalFileMetadataTestCase {
@@ -48,9 +47,6 @@ public class RealizationRegistryTest extends LocalFileMetadataTestCase {
     public void test() throws Exception {
         final RealizationRegistry registry = RealizationRegistry.getInstance(KylinConfig.getInstanceFromEnv());
         final Set<RealizationType> realizationTypes = registry.getRealizationTypes();
-        assertEquals(RealizationType.values().length, realizationTypes.size());
-        for (RealizationType type : RealizationType.values()) {
-            assertTrue(realizationTypes.contains(type));
-        }
+        assertEquals(RealizationType.values().length - 1, realizationTypes.size());
     }
 }


[52/55] [abbrv] kylin git commit: KYLIN-1749 Allow mandatory only cuboid

Posted by sh...@apache.org.
KYLIN-1749 Allow mandatory only cuboid


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 5d679f1a6d4055286b7e77a91cf18d482aebadaa
Parents: 8e780c2
Author: Yang Li <li...@apache.org>
Authored: Tue May 31 08:12:47 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Jul 26 09:30:19 2016 +0800

----------------------------------------------------------------------
 .../src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java      | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/5d679f1a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java
index 2e64791..b4a56f5 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java
@@ -150,7 +150,6 @@ public class CuboidTest extends LocalFileMetadataTestCase {
         CubeDesc cube = getSSBCubeDesc();
 
         assertEquals(false, Cuboid.isValid(cube, toLong("10000000000")));
-
         // the 4th is mandatory and isMandatoryOnlyValid is true
         assertEquals(true, Cuboid.isValid(cube, toLong("10000001000")));
         assertEquals(true, Cuboid.isValid(cube, toLong("00000001000")));


[40/55] [abbrv] kylin git commit: minor, avoid port conflict

Posted by sh...@apache.org.
minor, avoid port conflict

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 9d48273a468c60f8ac2ddf7c1b7696fe7a3b6ecb
Parents: ae9ccf3
Author: shaofengshi <sh...@apache.org>
Authored: Fri Jul 22 18:13:48 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Jul 22 18:13:48 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/rest/service/CacheServiceTest.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/9d48273a/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java b/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
index d77b9ce..7620f8c 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
@@ -72,12 +72,12 @@ public class CacheServiceTest extends LocalFileMetadataTestCase {
     public static void beforeClass() throws Exception {
         staticCreateTestMetadata();
         configA = KylinConfig.getInstanceFromEnv();
-        configA.setProperty("kylin.rest.servers", "localhost:7070");
+        configA.setProperty("kylin.rest.servers", "localhost:7777");
         configB = KylinConfig.createKylinConfigFromInputStream(KylinConfig.getKylinPropertiesAsInputStream());
-        configB.setProperty("kylin.rest.servers", "localhost:7070");
+        configB.setProperty("kylin.rest.servers", "localhost:7777");
         configB.setMetadataUrl("../examples/test_metadata");
 
-        server = new Server(7070);
+        server = new Server(7777);
         ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
         context.setContextPath("/");
         server.setHandler(context);


[20/55] [abbrv] kylin git commit: minor, fix DebugTomcat

Posted by sh...@apache.org.
minor, fix DebugTomcat


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 6747b2a80c0957cdc7602b81774aebeab523933c
Parents: 416f0a1
Author: lidongsjtu <li...@apache.org>
Authored: Tue Jul 12 18:20:59 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Tue Jul 12 18:22:33 2016 +0800

----------------------------------------------------------------------
 examples/test_case_data/sandbox/kylin.properties | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/6747b2a8/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index 410bd74..9cba448 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -89,6 +89,8 @@ kylin.cube.algorithm=random
 kylin.security.profile=testing
 kylin.storage.cleanup.time.threshold=21600000
 
+kylin.job.controller.lock=org.apache.kylin.job.lock.MockJobLock
+
 ## Config for Restful APP ##
 # database connection settings:
 ldap.server=


[11/55] [abbrv] kylin git commit: KYLIN-1861 change default “hive.auto.convert.join.noconditionaltask.size” to 100Mb

Posted by sh...@apache.org.
KYLIN-1861 change default \u201chive.auto.convert.join.noconditionaltask.size\u201d to 100Mb

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: b999eac8fa77ebf7e6bd3b65d71654aaab24544e
Parents: 687d593
Author: shaofengshi <sh...@apache.org>
Authored: Sat Jul 9 09:14:55 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sat Jul 9 09:14:55 2016 +0800

----------------------------------------------------------------------
 build/conf/kylin_hive_conf.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b999eac8/build/conf/kylin_hive_conf.xml
----------------------------------------------------------------------
diff --git a/build/conf/kylin_hive_conf.xml b/build/conf/kylin_hive_conf.xml
index b95c077..3d6109b 100644
--- a/build/conf/kylin_hive_conf.xml
+++ b/build/conf/kylin_hive_conf.xml
@@ -35,7 +35,7 @@
 
     <property>
         <name>hive.auto.convert.join.noconditionaltask.size</name>
-        <value>300000000</value>
+        <value>100000000</value>
         <description>enable map-side join</description>
     </property>
 


[49/55] [abbrv] kylin git commit: merge 1.5.3

Posted by sh...@apache.org.
merge 1.5.3


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 90a0c5baba0c0d8e7c1f84ea4b94f0886fa8c19e
Parents: 1b6ed08
Author: shaofengshi <sh...@apache.org>
Authored: Mon Jul 25 20:12:00 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 25 20:13:38 2016 +0800

----------------------------------------------------------------------
 .../kylin/provision/BuildCubeWithEngine.java    |  3 +-
 .../rest/security/MockAclHBaseStorage.java      |  4 +-
 .../apache/kylin/rest/service/CubeService.java  |  2 +
 .../apache/kylin/rest/service/QueryService.java |  7 +-
 .../apache/kylin/rest/service/UserService.java  | 14 +---
 .../kylin/storage/hbase/HBaseConnection.java    | 74 ++++++++++++++++----
 .../kylin/storage/hbase/HBaseResourceStore.java | 23 ++----
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     |  2 +-
 .../storage/hbase/steps/HBaseCuboidWriter.java  |  1 -
 .../storage/hbase/util/CubeMigrationCLI.java    | 22 ++----
 .../storage/hbase/util/StorageCleanupJob.java   |  5 +-
 11 files changed, 85 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
index 4cee1ed..2a5979f 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
@@ -58,6 +58,7 @@ 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.HBaseConnection;
 import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
 import org.apache.kylin.storage.hbase.util.StorageCleanupJob;
 import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
@@ -419,7 +420,7 @@ public class BuildCubeWithEngine {
     }
 
     private void checkHFilesInHBase(CubeSegment segment) throws IOException {
-        Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
         String tableName = segment.getStorageLocationIdentifier();
 
         HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
index 492c176..16d6f9f 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
@@ -34,8 +34,8 @@ public class MockAclHBaseStorage implements AclHBaseStorage {
     private static final String aclTableName = "MOCK-ACL-TABLE";
     private static final String userTableName = "MOCK-USER-TABLE";
 
-    private HTableInterface mockedAclTable;
-    private HTableInterface mockedUserTable;
+    private Table mockedAclTable;
+    private Table mockedUserTable;
     private RealAclHBaseStorage realAcl;
 
     public MockAclHBaseStorage() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index 00b07d5..0503236 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -29,6 +29,7 @@ import java.util.Set;
 import java.util.WeakHashMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.HTable;
 import java.util.*;
 import org.apache.kylin.common.KylinConfig;
@@ -65,6 +66,7 @@ import org.apache.kylin.rest.security.AclPermission;
 import org.apache.kylin.source.hive.HiveSourceTableLoader;
 import org.apache.kylin.source.hive.cardinality.HiveColumnCardinalityJob;
 import org.apache.kylin.source.hive.cardinality.HiveColumnCardinalityUpdateJob;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 783616d..d095f2b 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -46,7 +46,6 @@ import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -184,14 +183,10 @@ public class QueryService extends BasicService {
         List<Query> queries = new ArrayList<Query>();
         Table htable = null;
         try {
-<<<<<<< b6296775f7793a63baf7f6a97cf4c0759d654341:server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
-            HConnection conn = HBaseConnection.get(hbaseUrl);
+            org.apache.hadoop.hbase.client.Connection conn = HBaseConnection.get(hbaseUrl);
             HBaseConnection.createHTableIfNeeded(conn, userTableName, USER_QUERY_FAMILY);
 
-            htable = conn.getTable(userTableName);
-=======
             htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
->>>>>>> KYLIN-1528 Create a branch for v1.5 with HBase 1.x API:server/src/main/java/org/apache/kylin/rest/service/QueryService.java
             Get get = new Get(Bytes.toBytes(creator));
             get.addFamily(Bytes.toBytes(USER_QUERY_FAMILY));
             Result result = htable.get(get);

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
index 64c2c7d..e039534 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
@@ -146,17 +146,12 @@ public class UserService implements UserDetailsManager {
     public void updateUser(UserDetails user) {
         Table htable = null;
         try {
-            byte[] userAuthorities = serialize(user.getAuthorities());
             htable = aclHBaseStorage.getTable(userTableName);
 
-<<<<<<< b6296775f7793a63baf7f6a97cf4c0759d654341:server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
             Pair<byte[], byte[]> pair = userToHBaseRow(user);
             Put put = new Put(pair.getKey());
-            put.add(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
-=======
-            Put put = new Put(Bytes.toBytes(user.getUsername()));
-            put.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), userAuthorities);
->>>>>>> KYLIN-1528 Create a branch for v1.5 with HBase 1.x API:server/src/main/java/org/apache/kylin/rest/service/UserService.java
+
+            put.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
 
             htable.put(put);
         } catch (IOException e) {
@@ -219,13 +214,8 @@ public class UserService implements UserDetailsManager {
         Scan s = new Scan();
         s.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN));
 
-<<<<<<< b6296775f7793a63baf7f6a97cf4c0759d654341:server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
         List<UserDetails> all = new ArrayList<UserDetails>();
-        HTableInterface htable = null;
-=======
-        List<String> authorities = new ArrayList<String>();
         Table htable = null;
->>>>>>> KYLIN-1528 Create a branch for v1.5 with HBase 1.x API:server/src/main/java/org/apache/kylin/rest/service/UserService.java
         ResultScanner scanner = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/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
index 05170a0..e7ee2f5 100644
--- 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
@@ -21,7 +21,7 @@ package org.apache.kylin.storage.hbase;
 import java.io.IOException;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.*;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.StorageException;
 import org.apache.kylin.engine.mr.HadoopUtil;
@@ -51,14 +52,20 @@ public class HBaseConnection {
 
     private static final Logger logger = LoggerFactory.getLogger(HBaseConnection.class);
 
-    private static final Map<String, Configuration> ConfigCache = new ConcurrentHashMap<String, Configuration>();
-    private static final Map<String, Connection> ConnPool = new ConcurrentHashMap<String, Connection>();
+    private static final Map<String, Configuration> configCache = new ConcurrentHashMap<String, Configuration>();
+    private static final Map<String, Connection> connPool = new ConcurrentHashMap<String, Connection>();
+
+    private static final ThreadLocal<Configuration> configThreadLocal = new ThreadLocal<>();
+
+    private static ExecutorService coprocessorPool = null;
 
     static {
         Runtime.getRuntime().addShutdownHook(new Thread() {
             @Override
             public void run() {
-                for (Connection conn : ConnPool.values()) {
+                closeCoprocessorPool();
+
+                for (Connection conn : connPool.values()) {
                     try {
                         conn.close();
                     } catch (IOException e) {
@@ -68,19 +75,62 @@ public class HBaseConnection {
             }
         });
     }
+
+    public static ExecutorService getCoprocessorPool() {
+        if (coprocessorPool != null) {
+            return coprocessorPool;
+        }
+
+        synchronized (HBaseConnection.class) {
+            if (coprocessorPool != null) {
+                return coprocessorPool;
+            }
+
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+            // copy from HConnectionImplementation.getBatchPool()
+            int maxThreads = config.getHBaseMaxConnectionThreads();
+            int coreThreads = config.getHBaseCoreConnectionThreads();
+            long keepAliveTime = config.getHBaseConnectionThreadPoolAliveSeconds();
+            LinkedBlockingQueue<Runnable> workQueue = new LinkedBlockingQueue<Runnable>(maxThreads * 100);
+            ThreadPoolExecutor tpe = new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue, //
+                    Threads.newDaemonThreadFactory("kylin-coproc-"));
+            tpe.allowCoreThreadTimeOut(true);
+
+            logger.info("Creating coprocessor thread pool with max of {}, core of {}", maxThreads, coreThreads);
+
+            coprocessorPool = tpe;
+            return coprocessorPool;
+        }
+    }
+
+    private static void closeCoprocessorPool() {
+        if (coprocessorPool == null)
+            return;
+
+        coprocessorPool.shutdown();
+        try {
+            if (!coprocessorPool.awaitTermination(10, TimeUnit.SECONDS)) {
+                coprocessorPool.shutdownNow();
+            }
+        } catch (InterruptedException e) {
+            coprocessorPool.shutdownNow();
+        }
+    }
+
     
     public static void clearConnCache() {
-        ConnPool.clear();
+        connPool.clear();
     }
 
     private static final ThreadLocal<Configuration> hbaseConfig = new ThreadLocal<>();
 
     public static Configuration getCurrentHBaseConfiguration() {
-        if (hbaseConfig.get() == null) {
+        if (configThreadLocal.get() == null) {
             String storageUrl = KylinConfig.getInstanceFromEnv().getStorageUrl();
-            hbaseConfig.set(newHBaseConfiguration(storageUrl));
+            configThreadLocal.set(newHBaseConfiguration(storageUrl));
         }
-        return hbaseConfig.get();
+        return configThreadLocal.get();
     }
 
     private static Configuration newHBaseConfiguration(String url) {
@@ -128,20 +178,20 @@ public class HBaseConnection {
     @SuppressWarnings("resource")
     public static Connection get(String url) {
         // find configuration
-        Configuration conf = ConfigCache.get(url);
+        Configuration conf = configCache.get(url);
         if (conf == null) {
             conf = newHBaseConfiguration(url);
-            ConfigCache.put(url, conf);
+            configCache.put(url, conf);
         }
 
-        Connection connection = ConnPool.get(url);
+        Connection 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 = ConnectionFactory.createConnection(conf);
-                    ConnPool.put(url, connection);
+                    connPool.put(url, connection);
                 }
 
                 if (connection == null || connection.isClosed()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/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
index f988dea..aa7a4d4 100644
--- 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
@@ -32,14 +32,9 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.*;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
@@ -286,7 +281,6 @@ public class HBaseResourceStore extends ResourceStore {
 
             Delete del = new Delete(Bytes.toBytes(resPath));
             table.delete(del);
-            table.flushCommits();
 
             if (hdfsResourceExist) { // remove hdfs cell value
                 Path redirectPath = bigCellHDFSPath(resPath);
@@ -308,7 +302,7 @@ public class HBaseResourceStore extends ResourceStore {
     }
 
     private Result getFromHTable(String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             return internalGetFromHTable(table, path, fetchContent, fetchTimestamp);
         } finally {
@@ -316,7 +310,7 @@ public class HBaseResourceStore extends ResourceStore {
         }
     }
 
-    private Result internalGetFromHTable(HTableInterface table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
+    private Result internalGetFromHTable(Table table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
         byte[] rowkey = Bytes.toBytes(path);
 
         Get get = new Get(rowkey);
@@ -330,14 +324,9 @@ public class HBaseResourceStore extends ResourceStore {
                 get.addColumn(B_FAMILY, B_COLUMN_TS);
         }
 
-        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
-        try {
-            Result result = table.get(get);
-            boolean exists = result != null && (!result.isEmpty() || (result.getExists() != null && result.getExists()));
-            return exists ? result : null;
-        } finally {
-            IOUtils.closeQuietly(table);
-        }
+        Result result = table.get(get);
+        boolean exists = result != null && (!result.isEmpty() || (result.getExists() != null && result.getExists()));
+        return exists ? result : null;
     }
 
     private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, Table table) throws IOException {

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
index 830aca7..d84074f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
@@ -360,7 +360,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                     final boolean[] abnormalFinish = new boolean[1];
 
                     try {
-                        HTableInterface table = conn.get(cubeSeg.getStorageLocationIdentifier(), HBaseConnection.getCoprocessorPool());
+                        Table table = conn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()), HBaseConnection.getCoprocessorPool());
 
                         final CubeVisitRequest request = builder.build();
                         final byte[] startKey = epRange.getFirst();

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
index 16955dd..c990379 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
@@ -136,7 +136,6 @@ public class HBaseCuboidWriter implements ICuboidWriter {
             logger.info("commit total " + puts.size() + " puts, totally cost:" + (System.currentTimeMillis() - t) + "ms");
             puts.clear();
         }
-        logger.info("commit total " + puts.size() + " puts, totally cost:" + (System.currentTimeMillis() - t) + "ms");
         puts.clear();
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index 6b63e66..dfb7c78 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -27,16 +27,8 @@ import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.*;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.RawResource;
@@ -45,7 +37,6 @@ import org.apache.kylin.common.persistence.Serializer;
 import org.apache.kylin.common.restclient.Broadcaster;
 import org.apache.kylin.common.restclient.RestClient;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -463,11 +454,7 @@ public class CubeMigrationCLI {
                             put.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), CellUtil.cloneValue(cell));
                             destAclHtable.put(put);
                         }
-                        Put put = new Put(Bytes.toBytes(cubeId));
-                        put.add(family, column, value);
-                        destAclHtable.put(put);
                     }
-                    destAclHtable.flushCommits();
                 } finally {
                     IOUtils.closeQuietly(srcAclHtable);
                     IOUtils.closeQuietly(destAclHtable);
@@ -533,13 +520,12 @@ public class CubeMigrationCLI {
         case COPY_ACL: {
             String cubeId = (String) opt.params[0];
             String modelId = (String) opt.params[1];
-            HTableInterface destAclHtable = null;
+            Table destAclHtable = null;
             try {
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
 
                 destAclHtable.delete(new Delete(Bytes.toBytes(cubeId)));
                 destAclHtable.delete(new Delete(Bytes.toBytes(modelId)));
-                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(destAclHtable);
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/90a0c5ba/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
index d7f49df..874121d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.CliCommandExecutor;
@@ -167,10 +168,10 @@ public class StorageCleanupJob extends AbstractHadoopJob {
     }
 
     class DeleteHTableRunnable implements Callable {
-        HBaseAdmin hbaseAdmin;
+        Admin hbaseAdmin;
         String htableName;
 
-        DeleteHTableRunnable(HBaseAdmin hbaseAdmin, String htableName) {
+        DeleteHTableRunnable(Admin hbaseAdmin, String htableName) {
             this.hbaseAdmin = hbaseAdmin;
             this.htableName = htableName;
         }


[27/55] [abbrv] kylin git commit: KYLIN-1786 extended column UI update

Posted by sh...@apache.org.
KYLIN-1786 extended column UI update


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 761db1e90649b78255931626cecfe0ee9293cac8
Parents: b7fdafd
Author: Jason <ji...@163.com>
Authored: Mon Jul 18 23:19:51 2016 +0800
Committer: Jason <ji...@163.com>
Committed: Mon Jul 18 23:19:51 2016 +0800

----------------------------------------------------------------------
 webapp/app/js/controllers/cubeEdit.js          |  57 ++---------
 webapp/app/js/controllers/cubeMeasures.js      |  69 ++++++++-----
 webapp/app/partials/cubeDesigner/measures.html | 105 +-------------------
 3 files changed, 55 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/761db1e9/webapp/app/js/controllers/cubeEdit.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeEdit.js b/webapp/app/js/controllers/cubeEdit.js
index 8475025..3fba2c9 100755
--- a/webapp/app/js/controllers/cubeEdit.js
+++ b/webapp/app/js/controllers/cubeEdit.js
@@ -86,40 +86,7 @@ KylinApp.controller('CubeEditCtrl', function ($scope, $q, $routeParams, $locatio
     return avaColObject;
   };
 
-  $scope.getCommonMetricColumns = function (measure) {
-    var nonCustomMeasures = ['SUM','MIN','MAX','COUNT'];
-    var expression = measure.function.expression;
-
-    //metric from model
-    var me_columns = [];
-    if($scope.metaModel.model.metrics){
-      angular.forEach($scope.metaModel.model.metrics,function(metric,index){
-        me_columns.push(metric);
-      })
-    }
-
-    if(nonCustomMeasures.indexOf(expression)!==-1){
-      return me_columns;
-    }
-
-    //add cube dimension column for specific measure
-    angular.forEach($scope.cubeMetaFrame.dimensions,function(dimension,index){
-      if(dimension.column && dimension.derived == null){
-        me_columns.push(dimension.column);
-      }
-
-      if(dimension.derived&&dimension.derived.length>=1){
-        me_columns = me_columns.concat(dimension.derived);
-      }
-
-    });
-
-    return me_columns;
-
-  };
-
-
-  $scope.getExtendedColumns = function (measure) {
+  $scope.getCommonMetricColumns = function () {
     //metric from model
     var me_columns = [];
     if($scope.metaModel.model.metrics){
@@ -127,31 +94,23 @@ KylinApp.controller('CubeEditCtrl', function ($scope, $q, $routeParams, $locatio
         me_columns.push(metric);
       })
     }
-    angular.forEach($scope.metaModel.model.dimensions,function(dimension,index){
-        if(dimension.columns){
-          me_columns = me_columns.concat(dimension.columns);
-        }
-    })
 
     return me_columns;
-
   };
 
-  $scope.getExtendedFactColumns = function (measure) {
+  $scope.getExtendedHostColumn = function(){
     var me_columns = [];
-    angular.forEach($scope.metaModel.model.dimensions,function(dimension,index){
+    //add cube dimension column for specific measure
+    angular.forEach($scope.cubeMetaFrame.dimensions,function(dimension,index){
       if($scope.metaModel.model.fact_table !== dimension.table){
         return;
       }
-
-      if(dimension.columns){
-        me_columns = me_columns.concat(dimension.columns);
+      if(dimension.column && dimension.derived == null){
+        me_columns.push(dimension.column);
       }
-    })
-
+    });
     return me_columns;
-
-  };
+  }
 
 
   $scope.getFactColumns = function () {

http://git-wip-us.apache.org/repos/asf/kylin/blob/761db1e9/webapp/app/js/controllers/cubeMeasures.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeMeasures.js b/webapp/app/js/controllers/cubeMeasures.js
index bb22a42..8db1356 100644
--- a/webapp/app/js/controllers/cubeMeasures.js
+++ b/webapp/app/js/controllers/cubeMeasures.js
@@ -20,37 +20,28 @@
 
 KylinApp.controller('CubeMeasuresCtrl', function ($scope, $modal,MetaModel,cubesManager,CubeDescModel,SweetAlert) {
 
+  $scope.initUpdateMeasureStatus = function(){
+    $scope.updateMeasureStatus = {
+      isEdit:false,
+      editIndex:-1
+    }
+  };
+  $scope.initUpdateMeasureStatus();
 
-  $scope.addNewMeasure = function (measure) {
+  $scope.addNewMeasure = function (measure, index) {
+    if(measure&&index>=0){
+      $scope.updateMeasureStatus.isEdit = true;
+      $scope.updateMeasureStatus.editIndex = index;
+    }
     $scope.nextParameters = [];
-    $scope.newMeasure = (!!measure)? measure:CubeDescModel.createMeasure();
-    //if(!!measure){
-    //  $scope.convertNextParameters();
-    //}
+    $scope.newMeasure = (!!measure)? jQuery.extend(true, {},measure):CubeDescModel.createMeasure();
     if(!!measure && measure.function.parameter.next_parameter){
       $scope.nextPara.value = measure.function.parameter.next_parameter.value;
     }
   };
-  //$scope.convertNextParameters = function(){
-  //  $scope.nextParameters = [];
-  //  var paramater = jQuery.extend(true, {}, $scope.newMeasure.function.parameter);
-  //  while(paramater.next_parameter){
-  //    var paraMeter =
-  //    {
-  //     "type": paramater.next_parameter.type,
-  //     "value":paramater.next_parameter.value,
-  //      "next_parameter":null
-  //    }
-  //    $scope.nextParameters.push(paraMeter);
-  //
-  //    paramater = paramater.next_parameter;
-  //
-  //  }
-  //
-  //}
+
 
   $scope.updateNextParameter = function(){
-    //jQuery.extend(true, {},$scope.newMeasure.function.parameter.next_parameter)
     for(var i= 0;i<$scope.nextParameters.length-1;i++){
       $scope.nextParameters[i].next_parameter=$scope.nextParameters[i+1];
     }
@@ -106,7 +97,8 @@ KylinApp.controller('CubeMeasuresCtrl', function ($scope, $modal,MetaModel,cubes
 
   $scope.clearNewMeasure = function () {
     $scope.newMeasure = null;
-    $scope.nextPara.value = "";
+    $scope.initUpdateMeasureStatus();
+    $scope.nextParameterInit();
   };
 
   $scope.saveNewMeasure = function () {
@@ -116,22 +108,45 @@ KylinApp.controller('CubeMeasuresCtrl', function ($scope, $modal,MetaModel,cubes
       return false;
     }
     if($scope.nextPara.value!=="" && ($scope.newMeasure.function.expression == 'EXTENDED_COLUMN' || $scope.newMeasure.function.expression == 'TOP_N')){
-      $scope.newMeasure.function.parameter.next_parameter = $scope.nextPara;
+      $scope.newMeasure.function.parameter.next_parameter = jQuery.extend(true,{},$scope.nextPara);
     }
 
-    if ($scope.cubeMetaFrame.measures.indexOf($scope.newMeasure) === -1) {
+    if($scope.updateMeasureStatus.isEdit == true){
+      $scope.cubeMetaFrame.measures[$scope.updateMeasureStatus.editIndex] = $scope.newMeasure;
+    }
+    else {
       $scope.cubeMetaFrame.measures.push($scope.newMeasure);
     }
     $scope.newMeasure = null;
+    $scope.initUpdateMeasureStatus();
+    $scope.nextParameterInit();
     return true;
   };
 
+  $scope.nextParameterInit = function(){
+    $scope.nextPara = {
+      "type":"column",
+      "value":"",
+      "next_parameter":null
+    }
+    if($scope.newMeasure){
+      $scope.newMeasure.function.parameter.next_parameter = null;
+    }
+  }
+
   //map right return type for param
   $scope.measureReturnTypeUpdate = function(){
 
-    if($scope.newMeasure.function.expression == 'TOP_N'||$scope.newMeasure.function.expression == 'EXTENDED_COLUMN'){
+    if($scope.newMeasure.function.expression == 'TOP_N'){
+      $scope.newMeasure.function.parameter.type= 'column';
+      $scope.newMeasure.function.returntype = "topn(100)";
+      return;
+    }else if($scope.newMeasure.function.expression == 'EXTENDED_COLUMN'){
       $scope.newMeasure.function.parameter.type= 'column';
+      $scope.newMeasure.function.returntype = "extendedcolumn(100)";
       return;
+    }else{
+      $scope.nextParameterInit();
     }
 
     if($scope.newMeasure.function.expression == 'COUNT'){

http://git-wip-us.apache.org/repos/asf/kylin/blob/761db1e9/webapp/app/partials/cubeDesigner/measures.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/measures.html b/webapp/app/partials/cubeDesigner/measures.html
index 0a39210..a9a3b9c 100755
--- a/webapp/app/partials/cubeDesigner/measures.html
+++ b/webapp/app/partials/cubeDesigner/measures.html
@@ -56,7 +56,7 @@
                           </td>
                           <td ng-if="state.mode=='edit'">
                               <!--Edit Button -->
-                              <button class="btn btn-xs btn-info" ng-click="addNewMeasure(measure)">
+                              <button class="btn btn-xs btn-info" ng-click="addNewMeasure(measure, $index)">
                                   <i class="fa fa-pencil"></i>
                               </button>
                               <!--Remove Button -->
@@ -152,14 +152,14 @@
                                         ng-if="newMeasure.function.parameter.type == 'column' && newMeasure.function.expression !== 'EXTENDED_COLUMN'"
                                         ng-model="newMeasure.function.parameter.value"
                                         ng-change="measureReturnTypeUpdate();"
-                                        ng-options="column as column for column in getCommonMetricColumns(newMeasure)" >
+                                        ng-options="column as column for column in getCommonMetricColumns()" >
                                   <option value="">-- Select a Column --</option>
                                 </select>
                                 <select class="form-control" chosen
                                         ng-if="newMeasure.function.expression == 'EXTENDED_COLUMN'"
                                         ng-model="newMeasure.function.parameter.value"
                                         ng-change="measureReturnTypeUpdate();"
-                                        ng-options="column as column for column in getFactColumns()" >
+                                        ng-options="column as column for column in getExtendedHostColumn()" >
                                   <option value="">-- Select a Column --</option>
                                 </select>
                               </div>
@@ -212,7 +212,7 @@
                           <div class="col-xs-12 col-sm-6">
                             <select class="form-control" chosen ng-if="nextPara.type !== 'constant'" required
                                     ng-model="nextPara.value"
-                                    ng-options="column as column for column in getExtendedFactColumns()" >
+                                    ng-options="column as column for column in getCommonMetricColumns()" >
                               <option value=""></option>
                             </select>
                           </div>
@@ -227,7 +227,7 @@
                           <div class="col-xs-12 col-sm-6">
                             <select class="form-control" chosen ng-if="nextPara.type !== 'constant'" required
                                     ng-model="nextPara.value"
-                                    ng-options="column as column for column in getExtendedColumns(newMeasure)" >
+                                    ng-options="column as column for column in getCommonMetricColumns()" >
                               <option value=""></option>
                             </select>
                           </div>
@@ -260,14 +260,6 @@
                               </tr>
                             </table>
 
-
-
-                            <!--<button class="btn btn-sm btn-info" ng-click="addNextParameter()" ng-show="newMeasure.function.expression == 'TOP_N' && nextParameters.length==0"-->
-                                    <!--ng-show="state.mode=='edit'"><i class="fa fa-plus"> Group by Column</i>-->
-                            <!--</button>-->
-                            <!--<button class="btn btn-sm btn-info" ng-click="addNextParameter()" ng-show="newMeasure.function.expression == 'EXTENDED_COLUMN' && nextParameters.length==0"-->
-                                    <!--ng-show="state.mode=='edit'"><i class="fa fa-plus"> Extended Column On Fact Table</i>-->
-                            <!--</button>-->
                           </div>
                         </div>
                       </div>
@@ -285,93 +277,6 @@
 </div>
 
 
-
-
-<script type="text/ng-template" id="nextParameter.html">
-  <div class="modal-header">
-    <h4 tooltip="submit" ng-if="newmea.measure&&newmea.measure.function.expression == 'EXTENDED_COLUMN'">Add Extended Column</h4>
-    <h4 tooltip="submit" ng-if="newmea.measure&&newmea.measure.function.expression == 'TOP_N'">Select Group By Column</h4>
-  </div>
-  <div class="modal-body" style="background-color: white">
-
-    <div class="row">
-      <div class="col-md-2"></div>
-      <div class="col-md-8">
-        <div class="row">
-          <div class="form-group">
-            <div class="row" ng-if="newmea.measure&&newmea.measure.function.expression !== 'TOP_N' && newmea.measure.function.expression !== 'EXTENDED_COLUMN'">
-              <label class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default"><b>Param Type</b></label>
-              <div class="col-xs-12 col-sm-6">
-                <select class="form-control"
-                        chosen ng-model="nextPara.type" required
-                        ng-options="mpt as mpt for mpt in cubeConfig.measureParamType">
-                  <option value=""></option>
-                </select>
-              </div>
-            </div>
-
-          </div>
-        </div>
-      </div>
-      <div class="col-md-2"></div>
-    </div>
-    <div class="row">
-      <div class="col-md-2"></div>
-      <div class="col-md-8">
-        <div class="row">
-          <div class="form-group">
-
-            <div ng-if="newmea.measure&&newmea.measure.function.expression !== 'TOP_N' && newmea.measure.function.expression !== 'EXTENDED_COLUMN" class="row">
-              <label  class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default"><b>Param Value</b></label>
-                <!--COUNT_DISTINCT-->
-              <div class="col-xs-12 col-sm-6">
-                <select class="form-control" chosen ng-if="nextPara.type !== 'constant'"
-                        ng-model="nextPara.value"
-                        ng-options="column as column for column in getCommonMetricColumns(newmea.measure)" >
-                  <option value=""></option>
-                </select>
-               <span class="font-color-default"
-                     ng-if="nextPara.type == 'constant'"
-                     ng-init="nextPara.value = 1"><b>&nbsp;&nbsp;1</b></span>
-              </div>
-            </div>
-
-            <div ng-if="newmea.measure&&newmea.measure.function.expression == 'TOP_N'" ng-init="nextPara.type='column'" class="row">
-              <label  class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default"><b>Group By Column</b></label>
-              <!--COUNT_DISTINCT-->
-              <div class="col-xs-12 col-sm-6">
-                <select class="form-control" chosen
-                        ng-model="nextPara.value"
-                        ng-options="column as column for column in getExtendedColumns(newmea.measure)" >
-                  <option value=""></option>
-                </select>
-              </div>
-            </div>
-
-            <div ng-if="newmea.measure&&newmea.measure.function.expression == 'EXTENDED_COLUMN'" ng-init="nextPara.type='column'" class="row">
-              <label  class="col-xs-12 col-sm-4 control-label no-padding-right font-color-default"><b>Extended Column On Fact Table</b></label>
-              <!--COUNT_DISTINCT-->
-              <div class="col-xs-12 col-sm-6">
-                <select class="form-control" chosen
-                        ng-model="nextPara.value"
-                        ng-options="column as column for column in getExtendedColumns(newmea.measure)" >
-                  <option value=""></option>
-                </select>
-              </div>
-            </div>
-
-            </div>
-          </div>
-        </div>
-      </div>
-      <div class="col-md-2"></div>
-    </div>
-  </div>
-  <div class="modal-footer">
-    <button class="btn btn-success" ng-click="ok()">OK</button>
-    <button class="btn btn-primary" ng-click="cancel()">Close</button>
-  </div>
-</script>
 <script type="text/ng-template" id="expressionTip.html">
   <p>All cubes have to contain one measure for Count(1), suggest use "_Count_" as name (Has been generated automatically)</p>
 </script>


[24/55] [abbrv] kylin git commit: KYLIN-1894 GlobalDictionary may corrupt when server suddenly crash

Posted by sh...@apache.org.
KYLIN-1894 GlobalDictionary may corrupt when server suddenly crash

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


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: f0db336fc238f88af952de8672806c62c9c57aee
Parents: 9200475
Author: sunyerui <su...@gmail.com>
Authored: Fri Jul 15 12:30:06 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 18 16:41:21 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/dict/AppendTrieDictionary.java |  65 ++---
 .../org/apache/kylin/dict/CachedTreeMap.java    |  83 +++++-
 .../kylin/dict/AppendTrieDictionaryTest.java    |   6 +-
 .../apache/kylin/dict/CachedTreeMapTest.java    | 265 +++++++++++++++++++
 4 files changed, 364 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/f0db336f/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
index 32038bf..4cce586 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
@@ -99,13 +99,7 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
         }
     }
 
-    public void update(String baseDir, int baseId, int maxId, int maxValueLength, int nValues, BytesConverter bytesConverter, byte[] dictMapBytes) throws IOException {
-        ByteArrayInputStream buf = new ByteArrayInputStream(dictMapBytes);
-        DataInputStream input = new DataInputStream(buf);
-        update(baseDir, baseId, maxId, maxValueLength, nValues, bytesConverter, input);
-    }
-
-    public void update(String baseDir, int baseId, int maxId, int maxValueLength, int nValues, BytesConverter bytesConverter, DataInput input) throws IOException {
+    public void update(String baseDir, int baseId, int maxId, int maxValueLength, int nValues, BytesConverter bytesConverter, CachedTreeMap dictMap) throws IOException {
         this.baseDir = baseDir;
         this.baseId = baseId;
         this.maxId = maxId;
@@ -114,11 +108,8 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
         this.bytesConverter = bytesConverter;
 
         int cacheSize = KylinConfig.getInstanceFromEnv().getAppendDictCacheSize();
-        if (dictSliceMap == null) {
-            dictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(cacheSize).baseDir(baseDir).persistent(true).immutable(true).keyClazz(DictSliceKey.class).valueClazz(DictSlice.class).build();
-        }
-        dictSliceMap.clear();
-        ((Writable) dictSliceMap).readFields(input);
+        dictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(cacheSize).baseDir(baseDir).persistent(true).immutable(true).keyClazz(DictSliceKey.class).valueClazz(DictSlice.class).build();
+        ((CachedTreeMap)dictSliceMap).loadEntry(dictMap);
     }
 
     public byte[] writeDictMap() throws IOException {
@@ -777,7 +768,7 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
 
         private AppendTrieDictionary dict;
 
-        private TreeMap<DictSliceKey, DictNode> dictSliceMap;
+        private TreeMap<DictSliceKey, DictNode> mutableDictSliceMap;
         private static int MAX_ENTRY_IN_SLICE = 10_000_000;
         private static final double MAX_ENTRY_OVERHEAD_FACTOR = 1.0;
 
@@ -803,9 +794,9 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
             MAX_ENTRY_IN_SLICE = KylinConfig.getInstanceFromEnv().getAppendDictEntrySize();
             int cacheSize = KylinConfig.getInstanceFromEnv().getAppendDictCacheSize();
             // create a new cached map with baseDir
-            dictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(cacheSize).baseDir(baseDir).keyClazz(DictSliceKey.class).valueClazz(DictNode.class).persistent(true).immutable(false).build();
+            mutableDictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(cacheSize).baseDir(baseDir).keyClazz(DictSliceKey.class).valueClazz(DictNode.class).persistent(true).immutable(false).build();
             if (dictMapBytes != null) {
-                ((Writable) dictSliceMap).readFields(new DataInputStream(new ByteArrayInputStream(dictMapBytes)));
+                ((Writable) mutableDictSliceMap).readFields(new DataInputStream(new ByteArrayInputStream(dictMapBytes)));
             }
         }
 
@@ -819,23 +810,23 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
             }
             maxValueLength = Math.max(maxValueLength, value.length);
 
-            if (dictSliceMap.isEmpty()) {
+            if (mutableDictSliceMap.isEmpty()) {
                 DictNode root = new DictNode(new byte[0], false);
-                dictSliceMap.put(DictSliceKey.wrap(new byte[0]), root);
+                mutableDictSliceMap.put(DictSliceKey.wrap(new byte[0]), root);
             }
-            DictSliceKey sliceKey = dictSliceMap.floorKey(DictSliceKey.wrap(value));
+            DictSliceKey sliceKey = mutableDictSliceMap.floorKey(DictSliceKey.wrap(value));
             if (sliceKey == null) {
-                sliceKey = dictSliceMap.firstKey();
+                sliceKey = mutableDictSliceMap.firstKey();
             }
-            DictNode root = dictSliceMap.get(sliceKey);
+            DictNode root = mutableDictSliceMap.get(sliceKey);
             addValueR(root, value, 0);
             if (root.childrenCount > MAX_ENTRY_IN_SLICE * MAX_ENTRY_OVERHEAD_FACTOR) {
-                dictSliceMap.remove(sliceKey);
+                mutableDictSliceMap.remove(sliceKey);
                 DictNode newRoot = splitNodeTree(root);
                 DictNode.mergeSingleByteNode(root, 1);
                 DictNode.mergeSingleByteNode(newRoot, 0);
-                dictSliceMap.put(DictSliceKey.wrap(root.firstValue()), root);
-                dictSliceMap.put(DictSliceKey.wrap(newRoot.firstValue()), newRoot);
+                mutableDictSliceMap.put(DictSliceKey.wrap(root.firstValue()), root);
+                mutableDictSliceMap.put(DictSliceKey.wrap(newRoot.firstValue()), newRoot);
             }
         }
 
@@ -956,18 +947,11 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
         }
 
         public AppendTrieDictionary<T> build(int baseId) throws IOException {
-            ByteArrayOutputStream buf = new ByteArrayOutputStream();
-            DataOutputStream out = new DataOutputStream(buf);
-            ((Writable) dictSliceMap).write(out);
-            byte[] dictMapBytes = buf.toByteArray();
-            buf.close();
-            out.close();
-
             if (dict == null) {
                 dict = new AppendTrieDictionary<T>();
             }
-            dict.update(baseDir, baseId, maxId, maxValueLength, nValues, bytesConverter, dictMapBytes);
-            dict.flushIndex();
+            dict.flushIndex((CachedTreeMap) mutableDictSliceMap);
+            dict.update(baseDir, baseId, maxId, maxValueLength, nValues, bytesConverter, (CachedTreeMap)mutableDictSliceMap);
 
             return dict;
         }
@@ -1047,24 +1031,25 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
         throw new UnsupportedOperationException("AppendTrieDictionary can't retrive value from id");
     }
 
-    public void flushIndex() throws IOException {
-        Path filePath = new Path(baseDir + "/.index");
+    public void flushIndex(CachedTreeMap dictSliceMap) throws IOException {
+        Path filePath = new Path(dictSliceMap.getCurrentDir() + "/.index");
         Configuration conf = new Configuration();
-        try (FSDataOutputStream indexOut = (FileSystem.get(filePath.toUri(), conf)).create(filePath, true, 8 * 1024 * 1024, (short) 2, 8 * 1024 * 1024 * 8)) {
+        try (FSDataOutputStream indexOut = (FileSystem.get(filePath.toUri(), conf)).create(filePath, true, 8 * 1024 * 1024, (short) 5, 8 * 1024 * 1024 * 8)) {
             indexOut.writeInt(baseId);
             indexOut.writeInt(maxId);
             indexOut.writeInt(maxValueLength);
             indexOut.writeInt(nValues);
             indexOut.writeUTF(bytesConverter.getClass().getName());
-            ((Writable) dictSliceMap).write(indexOut);
+            dictSliceMap.write(indexOut);
         }
+        dictSliceMap.commit(false);
     }
 
     @Override
     public AppendTrieDictionary copyToAnotherMeta(KylinConfig srcConfig, KylinConfig dstConfig) throws IOException {
         Configuration conf = new Configuration();
         AppendTrieDictionary newDict = new AppendTrieDictionary();
-        newDict.update(baseDir.replaceFirst(srcConfig.getHdfsWorkingDirectory(), dstConfig.getHdfsWorkingDirectory()), baseId, maxId, maxValueLength, nValues, bytesConverter, writeDictMap());
+        newDict.update(baseDir.replaceFirst(srcConfig.getHdfsWorkingDirectory(), dstConfig.getHdfsWorkingDirectory()), baseId, maxId, maxValueLength, nValues, bytesConverter, (CachedTreeMap)dictSliceMap);
         logger.info("Copy AppendDict from {} to {}", this.baseDir, newDict.baseDir);
         Path srcPath = new Path(this.baseDir);
         Path dstPath = new Path(newDict.baseDir);
@@ -1081,7 +1066,6 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
     @Override
     public void write(DataOutput out) throws IOException {
         out.writeUTF(baseDir);
-        flushIndex();
     }
 
     @Override
@@ -1103,7 +1087,10 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
                     throw new IOException(e);
                 }
             }
-            update(baseDir, baseId, maxId, maxValueLength, nValues, converter, input);
+            CachedTreeMap dictMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder()
+                    .baseDir(baseDir).persistent(true).immutable(true).keyClazz(DictSliceKey.class).valueClazz(DictSlice.class).build();
+            dictMap.readFields(input);
+            update(baseDir, baseId, maxId, maxValueLength, nValues, converter, dictMap);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/f0db336f/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java b/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
index ec29bb5..1ea3c1c 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -58,6 +59,8 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
     private final TreeSet<String> fileList;
     private final Configuration conf;
     private final String baseDir;
+    private final String tmpDir;
+    private final FileSystem fs;
     private final boolean persistent;
     private final boolean immutable;
     private long writeValueTime = 0;
@@ -110,7 +113,7 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
             return this;
         }
 
-        public CachedTreeMap build() {
+        public CachedTreeMap build() throws IOException {
             if (baseDir == null) {
                 throw new RuntimeException("CachedTreeMap need a baseDir to cache data");
             }
@@ -122,13 +125,19 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
         }
     }
 
-    private CachedTreeMap(int maxCount, Class<K> keyClazz, Class<V> valueClazz, String baseDir, boolean persistent, boolean immutable) {
+    private CachedTreeMap(int maxCount, Class<K> keyClazz, Class<V> valueClazz, String baseDir, boolean persistent, boolean immutable) throws IOException {
         super();
         this.keyClazz = keyClazz;
         this.valueClazz = valueClazz;
         this.fileList = new TreeSet<>();
         this.conf = new Configuration();
-        this.baseDir = baseDir;
+        if (baseDir.endsWith("/")) {
+            this.baseDir = baseDir.substring(0, baseDir.length()-1);
+        } else {
+            this.baseDir = baseDir;
+        }
+        this.tmpDir = this.baseDir + ".tmp";
+        this.fs = FileSystem.get(new Path(baseDir).toUri(), conf);
         this.persistent = persistent;
         this.immutable = immutable;
         CacheBuilder builder = CacheBuilder.newBuilder().removalListener(new RemovalListener<K, V>() {
@@ -140,17 +149,27 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
                     writeValue(notification.getKey(), notification.getValue());
                     break;
                 case EXPLICIT:
-                    // skip delete files to recover from error during dict appending
-                    // deleteValue(notification.getKey());
+                    deleteValue(notification.getKey());
                     break;
                 default:
                     throw new RuntimeException("unexpected evict reason " + notification.getCause());
                 }
             }
-        }).maximumSize(maxCount);
-        // For immutable values, use soft reference to free memory when gc, and just load again when need it
+        });
+        // For immutable values, load all values as much as possible, and evict by soft reference to free memory when gc
         if (this.immutable) {
             builder.softValues();
+        } else {
+            builder.maximumSize(maxCount);
+            // For mutable map, copy all data into tmp and modify on tmp data, avoiding suddenly server crash made data corrupt
+            if (fs.exists(new Path(tmpDir))) {
+                fs.delete(new Path(tmpDir), true);
+            }
+            if (fs.exists(new Path(this.baseDir))) {
+                FileUtil.copy(fs, new Path(this.baseDir), fs, new Path(tmpDir), false, true, conf);
+            } else {
+                fs.mkdirs(new Path(this.baseDir));
+            }
         }
         this.valueCache = builder.build(new CacheLoader<K, V>() {
             @Override
@@ -163,10 +182,47 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
     }
 
     private String generateFileName(K key) {
-        String file = baseDir + "/cached_" + key.toString();
+        String file = (immutable ? baseDir : tmpDir) + "/cached_" + key.toString();
         return file;
     }
 
+    public String getCurrentDir() {
+        return immutable ? baseDir : tmpDir;
+    }
+
+    public void commit(boolean stillMutable) throws IOException {
+        assert !immutable : "Only support commit method with immutable false";
+
+        Path basePath = new Path(baseDir);
+        Path backupPath = new Path(baseDir+".bak");
+        Path tmpPath = new Path(tmpDir);
+        try {
+            fs.rename(basePath, backupPath);
+        } catch (IOException e) {
+            logger.info("CachedTreeMap commit backup basedir failed, " + e, e);
+            throw e;
+        }
+
+        try {
+            if (stillMutable) {
+                FileUtil.copy(fs, tmpPath, fs, basePath, false, true, conf);
+            } else {
+                fs.rename(tmpPath, basePath);
+            }
+            fs.delete(backupPath, true);
+        } catch (IOException e) {
+            fs.rename(backupPath, basePath);
+            logger.info("CachedTreeMap commit move/copy tmpdir failed, " + e, e);
+            throw e;
+        }
+    }
+
+    public void loadEntry(CachedTreeMap other) {
+        for (Object key : other.keySet()) {
+            super.put((K)key, null);
+        }
+    }
+
     private void writeValue(K key, V value) {
         if (immutable) {
             return;
@@ -174,10 +230,10 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
         long t0 = System.currentTimeMillis();
         String fileName = generateFileName(key);
         Path filePath = new Path(fileName);
-        try (FSDataOutputStream out = (FileSystem.get(filePath.toUri(), conf)).create(filePath, true, BUFFER_SIZE, (short) 2, BUFFER_SIZE * 8)) {
+        try (FSDataOutputStream out = fs.create(filePath, true, BUFFER_SIZE, (short) 5, BUFFER_SIZE * 8)) {
             value.write(out);
             if (!persistent) {
-                FileSystem.get(filePath.toUri(), conf).deleteOnExit(filePath);
+                fs.deleteOnExit(filePath);
             }
         } catch (Exception e) {
             logger.error(String.format("write value into %s exception: %s", fileName, e), e);
@@ -192,7 +248,7 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
         long t0 = System.currentTimeMillis();
         String fileName = generateFileName(key);
         Path filePath = new Path(fileName);
-        try (FSDataInputStream input = (FileSystem.get(filePath.toUri(), conf)).open(filePath, BUFFER_SIZE)) {
+        try (FSDataInputStream input = fs.open(filePath, BUFFER_SIZE)) {
             V value = valueClazz.newInstance();
             value.readFields(input);
             return value;
@@ -211,7 +267,6 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
         String fileName = generateFileName(key);
         Path filePath = new Path(fileName);
         try {
-            FileSystem fs = FileSystem.get(filePath.toUri(), conf);
             if (fs.exists(filePath)) {
                 fs.delete(filePath, true);
             }
@@ -224,6 +279,7 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
 
     @Override
     public V put(K key, V value) {
+        assert !immutable : "Only support put method with immutable false";
         super.put(key, null);
         valueCache.put(key, value);
         return null;
@@ -245,6 +301,7 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
 
     @Override
     public V remove(Object key) {
+        assert !immutable : "Only support remove method with immutable false";
         super.remove(key);
         valueCache.invalidate(key);
         return null;
@@ -300,6 +357,7 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
 
         @Override
         public void remove() {
+            assert !immutable : "Only support remove method with immutable false";
             keyIterator.remove();
             valueCache.invalidate(currentKey);
         }
@@ -344,7 +402,6 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
             for (String file : fileList) {
                 try {
                     Path filePath = new Path(file);
-                    FileSystem fs = FileSystem.get(filePath.toUri(), conf);
                     fs.delete(filePath, true);
                 } catch (Throwable t) {
                     //do nothing?

http://git-wip-us.apache.org/repos/asf/kylin/blob/f0db336f/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
index 0ea5ebe..b81a439 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
@@ -56,7 +56,7 @@ public class AppendTrieDictionaryTest {
         KylinConfig config = KylinConfig.getInstanceFromEnv();
         config.setAppendDictEntrySize(50000);
         config.setAppendDictCacheSize(3);
-        config.setProperty("kylin.hdfs.working.dir", "/tmp");
+        config.setProperty("kylin.hdfs.working.dir", "/tmp/kylin_append_dict");
     }
 
     @AfterClass
@@ -122,7 +122,7 @@ public class AppendTrieDictionaryTest {
     @Test
     public void testHugeKeySet() throws IOException {
         BytesConverter converter = new StringBytesConverter();
-        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create("/tmp");
+        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create("/tmp/kylin_append_dict");
         AppendTrieDictionary<String> dict = null;
 
         InputStream is = new FileInputStream("src/test/resources/dict/huge_key");
@@ -152,7 +152,7 @@ public class AppendTrieDictionaryTest {
         }
         BytesConverter converter = new StringBytesConverter();
 
-        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create("/tmp");
+        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create("/tmp/kylin_append_dict");
         AppendTrieDictionary<String> dict = null;
         TreeMap<Integer, String> checkMap = new TreeMap<>();
         int firstAppend = rnd.nextInt(strList.size() / 2);

http://git-wip-us.apache.org/repos/asf/kylin/blob/f0db336f/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
new file mode 100644
index 0000000..d2af621
--- /dev/null
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
@@ -0,0 +1,265 @@
+package org.apache.kylin.dict;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.*;
+
+import static org.junit.Assert.*;
+
+/**
+ * Created by sunyerui on 16/7/12.
+ */
+public class CachedTreeMapTest {
+
+    public static class Key implements WritableComparable {
+        int keyInt;
+
+        public static Key of(int keyInt) {
+            Key newKey = new Key();
+            newKey.keyInt = keyInt;
+            return newKey;
+        }
+
+        @Override
+        public int compareTo(Object o) {
+            return keyInt - ((Key)o).keyInt;
+        }
+
+        @Override
+        public void write(DataOutput out) throws IOException {
+            out.writeInt(keyInt);
+        }
+
+        @Override
+        public void readFields(DataInput in) throws IOException {
+            keyInt = in.readInt();
+        }
+
+        @Override
+        public String toString() {
+            return String.valueOf(keyInt);
+        }
+    }
+
+    public static boolean VALUE_WRITE_ERROR_TOGGLE = false;
+    public static class Value implements Writable {
+        String valueStr;
+
+        public static Value of(String valueStr) {
+            Value newValue = new Value();
+            newValue.valueStr = valueStr;
+            return newValue;
+        }
+
+        @Override
+        public void write(DataOutput out) throws IOException {
+            if (VALUE_WRITE_ERROR_TOGGLE) {
+                out.write(new byte[0]);
+                return;
+            }
+            out.writeUTF(valueStr);
+        }
+
+        @Override
+        public void readFields(DataInput in) throws IOException {
+            valueStr = in.readUTF();
+        }
+    }
+
+    public static class CachedFileFilter implements FileFilter {
+        @Override
+        public boolean accept(File pathname) {
+            return pathname.getName().startsWith("cached_");
+        }
+    }
+
+    public static final String baseDir = "/tmp/kylin_cachedtreemap_test/";
+    public static final String backupDir = "/tmp/kylin_cachedtreemap_test.bak/";
+    public static final String tmpDir = "/tmp/kylin_cachedtreemap_test.tmp/";
+
+    @After
+    public void afterTest() {
+        File dir = new File(baseDir);
+        if (dir.exists()) {
+            for (File f : dir.listFiles()) {
+                f.delete();
+            }
+            dir.delete();
+        }
+
+        dir = new File(tmpDir);
+        if (dir.exists()) {
+            for (File f : dir.listFiles()) {
+                f.delete();
+            }
+            dir.delete();
+        }
+
+        dir = new File(backupDir);
+        if (dir.exists()) {
+            for (File f : dir.listFiles()) {
+                f.delete();
+            }
+            dir.delete();
+        }
+
+        VALUE_WRITE_ERROR_TOGGLE = false;
+    }
+
+    @Test
+    public void testCachedTreeMap() throws IOException {
+        CachedTreeMap map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(false).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        map.put(Key.of(1), Value.of("a"));
+        map.put(Key.of(2), Value.of("b"));
+        map.put(Key.of(3), Value.of("c"));
+        map.put(Key.of(4), Value.of("d"));
+        map.put(Key.of(5), Value.of("e"));
+
+        File dir = new File(tmpDir);
+        assertEquals(3, dir.listFiles(new CachedFileFilter()).length);
+
+        DataOutputStream out = new DataOutputStream(new FileOutputStream(tmpDir+"/.index"));
+        map.write(out);
+        out.flush();
+        out.close();
+        map.commit(false);
+
+        dir = new File(baseDir);
+        assertEquals(5, dir.listFiles(new CachedFileFilter()).length);
+
+        DataInputStream in = new DataInputStream(new FileInputStream(baseDir+".index"));
+        CachedTreeMap map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        map2.readFields(in);
+        assertEquals(5, map2.size());
+        assertEquals("b", ((Value)map2.get(Key.of(2))).valueStr);
+
+        try {
+            map2.put(Key.of(6), Value.of("f"));
+            fail("Should be error when put value into immutable map");
+        } catch (AssertionError error) {
+        }
+
+        assertFalse(new File(tmpDir).exists());
+        assertFalse(new File(backupDir).exists());
+    }
+
+    @Test
+    public void testWriteFailed() throws IOException {
+        // normal case
+        CachedTreeMap map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(false).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        map.put(Key.of(1), Value.of("a"));
+        map.put(Key.of(2), Value.of("b"));
+        map.put(Key.of(3), Value.of("c"));
+        map.remove(Key.of(3));
+        map.put(Key.of(4), Value.of("d"));
+
+        DataOutputStream out = new DataOutputStream(new FileOutputStream(tmpDir+".index"));
+        map.write(out);
+        out.flush();
+        out.close();
+        map.commit(false);
+
+        DataInputStream in = new DataInputStream(new FileInputStream(baseDir+".index"));
+        CachedTreeMap map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        map2.readFields(in);
+        assertEquals(3, map2.size());
+        assertEquals("a", ((Value)map2.get(Key.of(1))).valueStr);
+
+        // suppose write value failed and didn't commit data
+        map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(false).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        VALUE_WRITE_ERROR_TOGGLE = true;
+        map.put(Key.of(1), Value.of("aa"));
+        map.put(Key.of(2), Value.of("bb"));
+        VALUE_WRITE_ERROR_TOGGLE = false;
+        map.put(Key.of(3), Value.of("cc"));
+        map.put(Key.of(4), Value.of("dd"));
+        out = new DataOutputStream(new FileOutputStream(tmpDir+".index"));
+        map.write(out);
+        out.flush();
+        out.close();
+        // suppose write value failed and didn't commit data
+        //map.commit(false);
+
+        // read map data should not be modified
+        in = new DataInputStream(new FileInputStream(baseDir+".index"));
+        map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        map2.readFields(in);
+        assertEquals(3, map2.size());
+        assertEquals("a", ((Value)map2.get(Key.of(1))).valueStr);
+
+        assertTrue(new File(tmpDir).exists());
+        assertFalse(new File(backupDir).exists());
+    }
+
+    @Test
+    public void testCommit() throws IOException {
+        CachedTreeMap map = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(false).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        map.put(Key.of(1), Value.of("a"));
+        map.put(Key.of(2), Value.of("b"));
+        map.put(Key.of(3), Value.of("c"));
+        map.put(Key.of(4), Value.of("d"));
+
+        DataOutputStream out = new DataOutputStream(new FileOutputStream(tmpDir+".index"));
+        map.write(out);
+        out.flush();
+        out.close();
+        map.commit(true);
+
+        assertTrue(new File(tmpDir).exists());
+        assertFalse(new File(backupDir).exists());
+
+        DataInputStream in = new DataInputStream(new FileInputStream(baseDir+".index"));
+        CachedTreeMap map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        map2.readFields(in);
+        assertEquals(4, map2.size());
+        assertEquals("a", ((Value)map2.get(Key.of(1))).valueStr);
+
+        // continue modify map, but not commit
+        map.put(Key.of(1), Value.of("aa"));
+        map.put(Key.of(2), Value.of("bb"));
+        map.put(Key.of(3), Value.of("cc"));
+        map.put(Key.of(5), Value.of("e"));
+        map.put(Key.of(6), Value.of("f"));
+        out = new DataOutputStream(new FileOutputStream(tmpDir+".index"));
+        map.write(out);
+        out.flush();
+        out.close();
+
+        assertTrue(new File(tmpDir).exists());
+        assertEquals(6, new File(tmpDir).listFiles(new CachedFileFilter()).length);
+        assertEquals(4, new File(baseDir).listFiles(new CachedFileFilter()).length);
+
+        in = new DataInputStream(new FileInputStream(baseDir+".index"));
+        map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        map2.readFields(in);
+        assertEquals(4, map2.size());
+        assertEquals("a", ((Value)map2.get(Key.of(1))).valueStr);
+
+        // commit data
+        map.commit(false);
+        assertFalse(new File(tmpDir).exists());
+        assertEquals(6, new File(baseDir).listFiles(new CachedFileFilter()).length);
+
+        in = new DataInputStream(new FileInputStream(baseDir+".index"));
+        map2 = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir("file://"+baseDir)
+                .persistent(true).immutable(true).maxSize(2).keyClazz(Key.class).valueClazz(Value.class).build();
+        map2.readFields(in);
+        assertEquals(6, map2.size());
+        assertEquals("aa", ((Value)map2.get(Key.of(1))).valueStr);
+        assertEquals("f", ((Value)map2.get(Key.of(6))).valueStr);
+    }
+}
+


[03/55] [abbrv] kylin git commit: KYLIN-1858 remove all ii related code

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/LocalDictionary.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/LocalDictionary.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/LocalDictionary.java
deleted file mode 100644
index 6bf261f..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/LocalDictionary.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint;
-
-import java.util.Map;
-
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.dimension.DictionaryDimEnc;
-import org.apache.kylin.dimension.DimensionEncoding;
-import org.apache.kylin.dimension.FixedLenDimEnc;
-import org.apache.kylin.dimension.IDimensionEncodingMap;
-import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorRowType;
-
-import com.google.common.collect.Maps;
-
-/**
- */
-public class LocalDictionary implements IDimensionEncodingMap {
-
-    private CoprocessorRowType type;
-    private Dictionary<?>[] colDictMap;
-    private TableRecordInfoDigest recordInfo;
-    private Map<TblColRef, DimensionEncoding> encMap;
-
-    public LocalDictionary(Dictionary<?>[] colDictMap, CoprocessorRowType type, TableRecordInfoDigest recordInfo) {
-        this.colDictMap = colDictMap;
-        this.type = type;
-        this.recordInfo = recordInfo;
-        this.encMap = Maps.newHashMap();
-    }
-
-    @Override
-    public DimensionEncoding get(TblColRef col) {
-        DimensionEncoding result = encMap.get(col);
-        if (result == null) {
-            Dictionary<String> dict = getDictionary(col);
-            if (dict == null) {
-                int idx = type.getColIndexByTblColRef(col);
-                int len = recordInfo.length(idx);
-                result = new FixedLenDimEnc(len);
-            } else {
-                result = new DictionaryDimEnc(dict);
-            }
-            encMap.put(col, result);
-        }
-        return result;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public Dictionary<String> getDictionary(TblColRef col) {
-        int idx = type.getColIndexByTblColRef(col);
-        return (Dictionary<String>) this.colDictMap[idx];
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/SliceBitMapProvider.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/SliceBitMapProvider.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/SliceBitMapProvider.java
deleted file mode 100644
index 21c4096..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/SliceBitMapProvider.java
+++ /dev/null
@@ -1,55 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorRowType;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- * <p/>
- * an adapter
- */
-public class SliceBitMapProvider implements BitMapFilterEvaluator.BitMapProvider {
-
-    private Slice slice;
-    private CoprocessorRowType type;
-
-    public SliceBitMapProvider(Slice slice, CoprocessorRowType type) {
-        this.slice = slice;
-        this.type = type;
-    }
-
-    @Override
-    public ConciseSet getBitMap(TblColRef col, Integer startId, Integer endId) {
-        return slice.getColumnValueContainer(type.getColIndexByTblColRef(col)).getBitMap(startId, endId);
-    }
-
-    @Override
-    public int getRecordCount() {
-        return this.slice.getRecordCount();
-    }
-
-    @Override
-    public int getMaxValueId(TblColRef col) {
-        return slice.getColumnValueContainer(type.getColIndexByTblColRef(col)).getMaxValueId();
-    }
-}


[26/55] [abbrv] kylin git commit: KYLIN-1884 Reload metadata automatically after migrating cube

Posted by sh...@apache.org.
KYLIN-1884 Reload metadata automatically after migrating cube

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


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: b7fdafd86b4d1cabd92d4ed0c9762acba19de123
Parents: 28a2b9f
Author: kangkaisen <ka...@live.com>
Authored: Wed Jul 13 17:42:33 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 18 21:33:31 2016 +0800

----------------------------------------------------------------------
 .../kylin/storage/hbase/util/CubeMigrationCLI.java  | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b7fdafd8/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index 32aa4b0..9a83a3f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -42,6 +42,8 @@ import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
+import org.apache.kylin.common.restclient.Broadcaster;
+import org.apache.kylin.common.restclient.RestClient;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.CubeInstance;
@@ -148,6 +150,7 @@ public class CubeMigrationCLI {
         if (realExecute.equalsIgnoreCase("true")) {
             doOpts();
             checkMigrationSuccess(dstConfig, cubeName, true);
+            updateMeta(dstConfig);
         } else {
             showOpts();
         }
@@ -547,4 +550,17 @@ public class CubeMigrationCLI {
         }
         }
     }
+
+    private static void updateMeta(KylinConfig config){
+        String[] nodes = config.getRestServers();
+        for (String node : nodes) {
+            RestClient restClient = new RestClient(node);
+            try {
+                logger.info("update meta cache for " + node);
+                restClient.wipeCache(Broadcaster.TYPE.ALL.getType(), Broadcaster.EVENT.UPDATE.getType(), "all");
+            } catch (IOException e) {
+                logger.error(e.getMessage());
+            }
+        }
+    }
 }


[17/55] [abbrv] kylin git commit: KYLIN-1656 set cap for max and min reducer numbers

Posted by sh...@apache.org.
KYLIN-1656 set cap for max and min reducer numbers

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 3ad49ddbbda5c7d88f7b47d2777ac97b2fa656cf
Parents: a08b77d
Author: shaofengshi <sh...@apache.org>
Authored: Mon Jul 11 14:32:04 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 11 14:34:22 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/source/hive/CreateFlatHiveTableStep.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/3ad49ddb/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java b/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java
index 0297bfa..8a4aaa2 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java
@@ -62,7 +62,8 @@ public class CreateFlatHiveTableStep extends AbstractExecutable {
         int mapperInputRows = config.getHadoopJobMapperInputRows();
 
         int numReducers = Math.round(rowCount / ((float) mapperInputRows));
-        numReducers = Math.max(1, numReducers);
+        numReducers = Math.max(numReducers, config.getHadoopJobMinReducerNumber());
+        numReducers = Math.min(numReducers, config.getHadoopJobMaxReducerNumber());
 
         stepLogger.log("total input rows = " + rowCount);
         stepLogger.log("expected input rows per mapper = " + mapperInputRows);


[21/55] [abbrv] kylin git commit: KYLIN-1786 UI for extended columns as measure

Posted by sh...@apache.org.
KYLIN-1786 UI for extended columns as measure


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 469aa40030c930e64c7a6af5f6d011e3080dfa84
Parents: 6747b2a
Author: Jason <ji...@163.com>
Authored: Tue Jul 12 20:38:46 2016 +0800
Committer: Jason <ji...@163.com>
Committed: Tue Jul 12 20:38:46 2016 +0800

----------------------------------------------------------------------
 webapp/app/js/controllers/cubeEdit.js          |  52 ++++++++
 webapp/app/js/controllers/cubeMeasures.js      |  60 +++++----
 webapp/app/js/directives/directives.js         |  43 +++++++
 webapp/app/js/model/cubeConfig.js              |   2 +-
 webapp/app/partials/cubeDesigner/measures.html | 132 +++++++++++++++-----
 5 files changed, 237 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/469aa400/webapp/app/js/controllers/cubeEdit.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeEdit.js b/webapp/app/js/controllers/cubeEdit.js
index b6fc875..8475025 100755
--- a/webapp/app/js/controllers/cubeEdit.js
+++ b/webapp/app/js/controllers/cubeEdit.js
@@ -119,6 +119,58 @@ KylinApp.controller('CubeEditCtrl', function ($scope, $q, $routeParams, $locatio
   };
 
 
+  $scope.getExtendedColumns = function (measure) {
+    //metric from model
+    var me_columns = [];
+    if($scope.metaModel.model.metrics){
+      angular.forEach($scope.metaModel.model.metrics,function(metric,index){
+        me_columns.push(metric);
+      })
+    }
+    angular.forEach($scope.metaModel.model.dimensions,function(dimension,index){
+        if(dimension.columns){
+          me_columns = me_columns.concat(dimension.columns);
+        }
+    })
+
+    return me_columns;
+
+  };
+
+  $scope.getExtendedFactColumns = function (measure) {
+    var me_columns = [];
+    angular.forEach($scope.metaModel.model.dimensions,function(dimension,index){
+      if($scope.metaModel.model.fact_table !== dimension.table){
+        return;
+      }
+
+      if(dimension.columns){
+        me_columns = me_columns.concat(dimension.columns);
+      }
+    })
+
+    return me_columns;
+
+  };
+
+
+  $scope.getFactColumns = function () {
+    var me_columns = [];
+    angular.forEach($scope.cubeMetaFrame.dimensions,function(dimension,index){
+      if($scope.metaModel.model.fact_table !== dimension.table){
+        return;
+      }
+      if(dimension.column && dimension.derived == null){
+        me_columns.push(dimension.column);
+      }
+
+    });
+
+    return me_columns;
+
+  };
+
+
 
   $scope.getColumnType = function (_column, table) {
     var columns = $scope.getColumnsByTable(table);

http://git-wip-us.apache.org/repos/asf/kylin/blob/469aa400/webapp/app/js/controllers/cubeMeasures.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeMeasures.js b/webapp/app/js/controllers/cubeMeasures.js
index fb9d292..bb22a42 100644
--- a/webapp/app/js/controllers/cubeMeasures.js
+++ b/webapp/app/js/controllers/cubeMeasures.js
@@ -24,27 +24,30 @@ KylinApp.controller('CubeMeasuresCtrl', function ($scope, $modal,MetaModel,cubes
   $scope.addNewMeasure = function (measure) {
     $scope.nextParameters = [];
     $scope.newMeasure = (!!measure)? measure:CubeDescModel.createMeasure();
-    if(!!measure){
-      $scope.convertNextParameters();
+    //if(!!measure){
+    //  $scope.convertNextParameters();
+    //}
+    if(!!measure && measure.function.parameter.next_parameter){
+      $scope.nextPara.value = measure.function.parameter.next_parameter.value;
     }
   };
-  $scope.convertNextParameters = function(){
-    $scope.nextParameters = [];
-    var paramater = jQuery.extend(true, {}, $scope.newMeasure.function.parameter);
-    while(paramater.next_parameter){
-      var paraMeter =
-      {
-       "type": paramater.next_parameter.type,
-       "value":paramater.next_parameter.value,
-        "next_parameter":null
-      }
-      $scope.nextParameters.push(paraMeter);
-
-      paramater = paramater.next_parameter;
-
-    }
-
-  }
+  //$scope.convertNextParameters = function(){
+  //  $scope.nextParameters = [];
+  //  var paramater = jQuery.extend(true, {}, $scope.newMeasure.function.parameter);
+  //  while(paramater.next_parameter){
+  //    var paraMeter =
+  //    {
+  //     "type": paramater.next_parameter.type,
+  //     "value":paramater.next_parameter.value,
+  //      "next_parameter":null
+  //    }
+  //    $scope.nextParameters.push(paraMeter);
+  //
+  //    paramater = paramater.next_parameter;
+  //
+  //  }
+  //
+  //}
 
   $scope.updateNextParameter = function(){
     //jQuery.extend(true, {},$scope.newMeasure.function.parameter.next_parameter)
@@ -70,6 +73,14 @@ KylinApp.controller('CubeMeasuresCtrl', function ($scope, $modal,MetaModel,cubes
     }
     $scope.updateNextParameter();
   }
+
+
+  $scope.nextPara = {
+    "type":"column",
+    "value":"",
+    "next_parameter":null
+  }
+
   $scope.openParameterModal = function (parameter) {
     $modal.open({
       templateUrl: 'nextParameter.html',
@@ -95,13 +106,19 @@ KylinApp.controller('CubeMeasuresCtrl', function ($scope, $modal,MetaModel,cubes
 
   $scope.clearNewMeasure = function () {
     $scope.newMeasure = null;
+    $scope.nextPara.value = "";
   };
 
   $scope.saveNewMeasure = function () {
-    if ($scope.newMeasure.function.expression === 'TOP_N' && $scope.nextParameters.length == 0) {
+
+    if ($scope.newMeasure.function.expression === 'TOP_N' && $scope.nextPara.value == "") {
       SweetAlert.swal('', '[TOP_N] Group by Column is required', 'warning');
       return false;
     }
+    if($scope.nextPara.value!=="" && ($scope.newMeasure.function.expression == 'EXTENDED_COLUMN' || $scope.newMeasure.function.expression == 'TOP_N')){
+      $scope.newMeasure.function.parameter.next_parameter = $scope.nextPara;
+    }
+
     if ($scope.cubeMetaFrame.measures.indexOf($scope.newMeasure) === -1) {
       $scope.cubeMetaFrame.measures.push($scope.newMeasure);
     }
@@ -112,7 +129,8 @@ KylinApp.controller('CubeMeasuresCtrl', function ($scope, $modal,MetaModel,cubes
   //map right return type for param
   $scope.measureReturnTypeUpdate = function(){
 
-    if($scope.newMeasure.function.expression == 'TOP_N'){
+    if($scope.newMeasure.function.expression == 'TOP_N'||$scope.newMeasure.function.expression == 'EXTENDED_COLUMN'){
+      $scope.newMeasure.function.parameter.type= 'column';
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/469aa400/webapp/app/js/directives/directives.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/directives/directives.js b/webapp/app/js/directives/directives.js
index ae90c82..4b83865 100644
--- a/webapp/app/js/directives/directives.js
+++ b/webapp/app/js/directives/directives.js
@@ -332,6 +332,29 @@ KylinApp.directive('kylinPagination', function ($parse, $q) {
       };
     }
   };
+}).directive("extendedcolumntree", function($compile) {
+  return {
+    restrict: "E",
+    transclude: true,
+    scope: {
+      nextpara: '='
+    },
+    template:
+    '<li class="parent_li">Host Column:<b>{{nextpara.value}}</b></b></li>' +
+    '<li class="parent_li">Extended Column:<b>{{nextpara.next_parameter.value}}</b></li>',
+    compile: function(tElement, tAttr, transclude) {
+      var contents = tElement.contents().remove();
+      var compiledContents;
+      return function(scope, iElement, iAttr) {
+        if(!compiledContents) {
+          compiledContents = $compile(contents, transclude);
+        }
+        compiledContents(scope, function(clone, scope) {
+          iElement.append(clone);
+        });
+      };
+    }
+  };
 }).directive('kylinpopover', function ($compile,$templateCache) {
   return {
     restrict: "A",
@@ -355,4 +378,24 @@ KylinApp.directive('kylinPagination', function ($parse, $q) {
       $(element).popover(options);
     }
   };
+}).directive('extendedColumnReturn', function() {
+  return {
+    require: 'ngModel',
+    link: function(scope, element, attrs, ngModelController) {
+
+      var prefix = "extendedcolumn(";
+      var suffix = ")";
+      ngModelController.$parsers.push(function(data) {
+        //convert data from view format to model format
+        return prefix +data+suffix; //converted
+      });
+
+      ngModelController.$formatters.push(function(data) {
+        //convert data from model format to view format
+        var prefixIndex = data.indexOf("(")+1;
+        var suffixIndex = data.indexOf(")");
+        return data.substring(prefixIndex,suffixIndex); //converted
+      });
+    }
+  }
 });

http://git-wip-us.apache.org/repos/asf/kylin/blob/469aa400/webapp/app/js/model/cubeConfig.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/cubeConfig.js b/webapp/app/js/model/cubeConfig.js
index 82dfca4..88e2133 100644
--- a/webapp/app/js/model/cubeConfig.js
+++ b/webapp/app/js/model/cubeConfig.js
@@ -20,7 +20,7 @@ KylinApp.constant('cubeConfig', {
 
   //~ Define metadata & class
   measureParamType: ['column', 'constant'],
-  measureExpressions: ['SUM', 'MIN', 'MAX', 'COUNT', 'COUNT_DISTINCT',"TOP_N", 'RAW'],
+  measureExpressions: ['SUM', 'MIN', 'MAX', 'COUNT', 'COUNT_DISTINCT',"TOP_N", 'RAW','EXTENDED_COLUMN'],
   dimensionDataTypes: ["string", "tinyint", "int", "bigint", "date"],
   cubePartitionTypes: ['APPEND'],
   joinTypes: [

http://git-wip-us.apache.org/repos/asf/kylin/blob/469aa400/webapp/app/partials/cubeDesigner/measures.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/measures.html b/webapp/app/partials/cubeDesigner/measures.html
index 5778003..0a39210 100755
--- a/webapp/app/partials/cubeDesigner/measures.html
+++ b/webapp/app/partials/cubeDesigner/measures.html
@@ -43,8 +43,9 @@
                           <td>
                             <div class="paraTree">
                               <ul>
-                                <parametertree ng-if="measure.function.parameter!=null && measure.function.expression!=='TOP_N'" nextpara="measure.function.parameter"></parametertree>
+                                <parametertree ng-if="measure.function.parameter!=null && measure.function.expression!=='TOP_N' && measure.function.expression!=='EXTENDED_COLUMN'" nextpara="measure.function.parameter"></parametertree>
                                 <topntree ng-if="measure.function.parameter!=null && measure.function.expression=='TOP_N'" nextpara="measure.function.parameter"></topntree>
+                                <extendedcolumntree ng-if="measure.function.parameter!=null && measure.function.expression=='EXTENDED_COLUMN'" nextpara="measure.function.parameter"></extendedcolumntree>
                               </ul>
                             </div>
                         <!--<span ng-if="measure.function.parameter.next_parameter!=null">{{measure.function.parameter.next_parameter |json}}</span>-->
@@ -111,7 +112,7 @@
                           </div>
                       </div>
                       <!--Param Type-->
-                      <div class="form-group">
+                      <div class="form-group" ng-if="newMeasure.function.expression !== 'EXTENDED_COLUMN' && newMeasure.function.expression !== 'TOP_N'">
                           <div class="row">
                               <label class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default"><b>Param Type</b></label>
                               <div class="col-xs-12 col-sm-6">
@@ -133,9 +134,11 @@
                       <div class="form-group middle-popover">
                           <div class="row">
                               <label class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default">
-                                <b>Param Value</b>  <i class="fa fa-info-circle" kylinpopover placement="right" title="Param Value" template="paramvalueTip.html"></i>
+                                <b ng-if="newMeasure.function.expression == 'EXTENDED_COLUMN'">Host column On Fact Table</b>  <i ng-if="newMeasure.function.expression == 'EXTENDED_COLUMN'" title="Host Column" class="fa fa-info-circle" kylinpopover placement="right" template="hostTableTip.html"></i>
+                                <b ng-if="newMeasure.function.expression == 'TOP_N'">ORDER|SUM by Column</b>  <i ng-if="newMeasure.function.expression == 'TOP_N'" class="fa fa-info-circle" title="ORDER|SUM by Column" kylinpopover placement="right" template="topnTip.html"></i>
+                                <b ng-if="newMeasure.function.expression !== 'EXTENDED_COLUMN' && newMeasure.function.expression !== 'TOP_N' ">Param Value</b>  <i ng-if="newMeasure.function.expression !== 'EXTENDED_COLUMN' && newMeasure.function.expression !== 'TOP_N' " class="fa fa-info-circle" kylinpopover placement="right" title="Param Value" template="paramvalueTip.html"></i>
                                 <!--tip for top_n-->
-                                <small ng-if="newMeasure.function.expression == 'TOP_N'" class="help-block" style="color:#3a87ad">(SUM|ORDER BY Column for TOP_N)</small>
+                                <!--<small ng-if="newMeasure.function.expression == 'TOP_N'" class="help-block" style="color:#3a87ad">(SUM|ORDER BY Column for TOP_N)</small>-->
                               </label>
 
 
@@ -146,19 +149,29 @@
                                       ng-init="newMeasure.function.parameter.value = 1"><b>&nbsp;&nbsp;1</b></span>
                                 <!--!COUNT_DISTINCT-->
                                 <select class="form-control" chosen
-                                        ng-if="newMeasure.function.parameter.type == 'column'"
+                                        ng-if="newMeasure.function.parameter.type == 'column' && newMeasure.function.expression !== 'EXTENDED_COLUMN'"
                                         ng-model="newMeasure.function.parameter.value"
                                         ng-change="measureReturnTypeUpdate();"
                                         ng-options="column as column for column in getCommonMetricColumns(newMeasure)" >
                                   <option value="">-- Select a Column --</option>
                                 </select>
+                                <select class="form-control" chosen
+                                        ng-if="newMeasure.function.expression == 'EXTENDED_COLUMN'"
+                                        ng-model="newMeasure.function.parameter.value"
+                                        ng-change="measureReturnTypeUpdate();"
+                                        ng-options="column as column for column in getFactColumns()" >
+                                  <option value="">-- Select a Column --</option>
+                                </select>
                               </div>
                           </div>
                       </div>
                       <!--Return Type-->
-                      <div class="form-group">
+                      <div class="form-group middle-popover">
                           <div class="row">
-                              <label class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default"><b>Return Type</b></label>
+                              <label class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default">
+                                <b ng-if="newMeasure.function.expression !== 'EXTENDED_COLUMN'">Return Type</b>
+                                <b ng-if="newMeasure.function.expression == 'EXTENDED_COLUMN'">Maximum length of extended column</b>  <i ng-if="newMeasure.function.expression == 'EXTENDED_COLUMN'" title="Maximum Length" class="fa fa-info-circle" kylinpopover placement="right" template="extendedTypeTip.html"></i>
+                              </label>
                               <div class="col-xs-12 col-sm-6">
                                   <select class="form-control"
                                       ng-if="newMeasure.function.expression == 'COUNT_DISTINCT'"
@@ -174,13 +187,53 @@
                                         ng-options="ddt.value as ddt.name for ddt in cubeConfig.topNTypes">
                                   <option value=""></option>
                                 </select>
+
+                                <input extended-column-return
+                                  ng-if="newMeasure.function.expression == 'EXTENDED_COLUMN'"
+                                  type="text" placeholder="Kylin won\u2019t save more than this number of bytes" class="form-control"
+                                        tooltip-trigger="focus"
+                                        ng-init="newMeasure.function.returntype=newMeasure.function.returntype?newMeasure.function.returntype:'extendedcolumn(100)'"
+                                       ng-model="newMeasure.function.returntype" required />
+
                                   <span class="font-color-default"
-                                        ng-if="newMeasure.function.expression != 'COUNT_DISTINCT' && newMeasure.function.expression != 'TOP_N'"
+                                        ng-if="newMeasure.function.expression != 'COUNT_DISTINCT' && newMeasure.function.expression != 'TOP_N' && newMeasure.function.expression != 'EXTENDED_COLUMN' "
                                        ><b>&nbsp;&nbsp;{{newMeasure.function.returntype | uppercase}}</b>
                                   </span>
                               </div>
                           </div>
                       </div>
+
+
+                      <div class="form-group middle-popover" ng-if="newMeasure.function.expression == 'EXTENDED_COLUMN'">
+                        <div class="row">
+                          <label class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default">
+                            <b>Extended column on fact table</b>    <i ng-if="newMeasure.function.expression == 'EXTENDED_COLUMN'" title="Extended Column" class="fa fa-info-circle" kylinpopover placement="right" template="extendedColumnTip.html"></i>
+                          </label>
+                          <div class="col-xs-12 col-sm-6">
+                            <select class="form-control" chosen ng-if="nextPara.type !== 'constant'" required
+                                    ng-model="nextPara.value"
+                                    ng-options="column as column for column in getExtendedFactColumns()" >
+                              <option value=""></option>
+                            </select>
+                          </div>
+                        </div>
+                      </div>
+
+                      <div class="form-group" ng-if="newMeasure.function.expression == 'TOP_N'">
+                        <div class="row">
+                          <label class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default">
+                            <b>Group by Column</b>
+                          </label>
+                          <div class="col-xs-12 col-sm-6">
+                            <select class="form-control" chosen ng-if="nextPara.type !== 'constant'" required
+                                    ng-model="nextPara.value"
+                                    ng-options="column as column for column in getExtendedColumns(newMeasure)" >
+                              <option value=""></option>
+                            </select>
+                          </div>
+                        </div>
+                      </div>
+
                       <!--Name-->
                       <div class="form-group">
                         <div class="row">
@@ -207,26 +260,14 @@
                               </tr>
                             </table>
 
-                            <table class="table table-hover table-bordered list" ng-if="nextParameters.length" ng-show="newMeasure.function.expression == 'TOP_N'">
-                              <tr ng-repeat="n_parameter in nextParameters track by $index">
-                                <td><b>Group By Column</b></td>
-                                <td>{{n_parameter.value}}</td>
-                                <td>
-                                  <button class="btn btn-xs btn-info" ng-click="editNextParameter(n_parameter)">
-                                    <i class="fa fa-pencil"></i>
-                                  </button>
-                                  <button class="btn btn-xs btn-info" ng-click="removeParameter(nextParameters, $index)"><i class="fa fa-minus"></i>
-                                  </button>
-
-                                </td>
-
-                              </tr>
-                            </table>
 
 
-                            <button class="btn btn-sm btn-info" ng-click="addNextParameter()" ng-show="newMeasure.function.expression == 'TOP_N' && nextParameters.length==0"
-                                    ng-show="state.mode=='edit'"><i class="fa fa-plus"> Group by Column</i>
-                            </button>
+                            <!--<button class="btn btn-sm btn-info" ng-click="addNextParameter()" ng-show="newMeasure.function.expression == 'TOP_N' && nextParameters.length==0"-->
+                                    <!--ng-show="state.mode=='edit'"><i class="fa fa-plus"> Group by Column</i>-->
+                            <!--</button>-->
+                            <!--<button class="btn btn-sm btn-info" ng-click="addNextParameter()" ng-show="newMeasure.function.expression == 'EXTENDED_COLUMN' && nextParameters.length==0"-->
+                                    <!--ng-show="state.mode=='edit'"><i class="fa fa-plus"> Extended Column On Fact Table</i>-->
+                            <!--</button>-->
                           </div>
                         </div>
                       </div>
@@ -248,7 +289,7 @@
 
 <script type="text/ng-template" id="nextParameter.html">
   <div class="modal-header">
-    <h4 tooltip="submit" ng-if="newmea.measure&&newmea.measure.function.expression !== 'TOP_N'">Add Parameter</h4>
+    <h4 tooltip="submit" ng-if="newmea.measure&&newmea.measure.function.expression == 'EXTENDED_COLUMN'">Add Extended Column</h4>
     <h4 tooltip="submit" ng-if="newmea.measure&&newmea.measure.function.expression == 'TOP_N'">Select Group By Column</h4>
   </div>
   <div class="modal-body" style="background-color: white">
@@ -258,7 +299,7 @@
       <div class="col-md-8">
         <div class="row">
           <div class="form-group">
-            <div class="row" ng-if="newmea.measure&&newmea.measure.function.expression !== 'TOP_N'">
+            <div class="row" ng-if="newmea.measure&&newmea.measure.function.expression !== 'TOP_N' && newmea.measure.function.expression !== 'EXTENDED_COLUMN'">
               <label class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default"><b>Param Type</b></label>
               <div class="col-xs-12 col-sm-6">
                 <select class="form-control"
@@ -280,7 +321,7 @@
         <div class="row">
           <div class="form-group">
 
-            <div ng-if="newmea.measure&&newmea.measure.function.expression !== 'TOP_N'" class="row">
+            <div ng-if="newmea.measure&&newmea.measure.function.expression !== 'TOP_N' && newmea.measure.function.expression !== 'EXTENDED_COLUMN" class="row">
               <label  class="col-xs-12 col-sm-3 control-label no-padding-right font-color-default"><b>Param Value</b></label>
                 <!--COUNT_DISTINCT-->
               <div class="col-xs-12 col-sm-6">
@@ -301,12 +342,23 @@
               <div class="col-xs-12 col-sm-6">
                 <select class="form-control" chosen
                         ng-model="nextPara.value"
-                        ng-options="column as column for column in getCommonMetricColumns(newmea.measure)" >
+                        ng-options="column as column for column in getExtendedColumns(newmea.measure)" >
                   <option value=""></option>
                 </select>
               </div>
             </div>
 
+            <div ng-if="newmea.measure&&newmea.measure.function.expression == 'EXTENDED_COLUMN'" ng-init="nextPara.type='column'" class="row">
+              <label  class="col-xs-12 col-sm-4 control-label no-padding-right font-color-default"><b>Extended Column On Fact Table</b></label>
+              <!--COUNT_DISTINCT-->
+              <div class="col-xs-12 col-sm-6">
+                <select class="form-control" chosen
+                        ng-model="nextPara.value"
+                        ng-options="column as column for column in getExtendedColumns(newmea.measure)" >
+                  <option value=""></option>
+                </select>
+              </div>
+            </div>
 
             </div>
           </div>
@@ -329,3 +381,23 @@
     <li>Distinct Count is approximate, please indicate Error Rate, higher accuracy degree accompanied with larger storage size and longer build time</li>
   </ol>
 </script>
+
+<script type="text/ng-template" id="extendedTypeTip.html">
+  <p>
+    Kylin won\u2019t save more than this number of bytes for each extended column. If exceeded it will be truncated.
+  </p>
+</script>
+
+<script type="text/ng-template" id="topnTip.html">
+  <p>Will use this column for SUM and Order by</p>
+</script>
+
+<script type="text/ng-template" id="hostTableTip.html">
+  <p>Host column is the dimension to derive from, e.g. page_id</p>
+</script>
+<script type="text/ng-template" id="extendedColumnTip.html">
+  <p>
+    Extended column is derived from host, e.g. page_url. No filters on extended column!
+  </p>
+</script>
+


[23/55] [abbrv] kylin git commit: KYLIN-1860 "Models" view is slow to show up

Posted by sh...@apache.org.
KYLIN-1860 "Models" view is slow to show up

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 9200475c472ff7363698186105f7002f1d5d09b1
Parents: 743c9f8
Author: shaofengshi <sh...@apache.org>
Authored: Mon Jul 18 16:40:12 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 18 16:40:12 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/metadata/MetadataManager.java | 17 +----------------
 .../apache/kylin/rest/service/ModelService.java    |  8 +++-----
 2 files changed, 4 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/9200475c/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
index 45acd42..f4fbcd0 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
@@ -368,7 +368,7 @@ public class MetadataManager {
     }
 
     public List<DataModelDesc> getModels() {
-        return new ArrayList<DataModelDesc>(dataModelDescMap.values());
+        return new ArrayList<>(dataModelDescMap.values());
     }
 
     public List<DataModelDesc> getModels(String projectName) throws IOException {
@@ -386,21 +386,6 @@ public class MetadataManager {
             }
         }
 
-        //TODO, list model from realization,compatible with old meta data,will remove
-        RealizationRegistry registry = RealizationRegistry.getInstance(config);
-        for (RealizationEntry realization : projectInstance.getRealizationEntries()) {
-            IRealization rel = registry.getRealization(realization.getType(), realization.getRealization());
-            if (rel != null) {
-                DataModelDesc modelDesc = rel.getDataModelDesc();
-                if (modelDesc != null && !ret.contains(modelDesc)) {
-                    ProjectManager.getInstance(config).updateModelToProject(modelDesc.getName(), projectName);
-                    ret.add(modelDesc);
-                }
-            } else {
-                logger.warn("Realization '" + realization + "' defined under project '" + projectInstance + "' is not found");
-            }
-        }
-
         return new ArrayList<>(ret);
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/9200475c/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
index 4cfa209..bfff619 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
@@ -46,19 +46,18 @@ public class ModelService extends BasicService {
 
     @PostFilter(Constant.ACCESS_POST_FILTER_READ)
     public List<DataModelDesc> listAllModels(final String modelName, final String projectName) throws IOException {
-        List<DataModelDesc> models = null;
+        List<DataModelDesc> models;
         ProjectInstance project = (null != projectName) ? getProjectManager().getProject(projectName) : null;
 
         if (null == project) {
             models = getMetadataManager().getModels();
         } else {
             models = getMetadataManager().getModels(projectName);
-            project.getModels();
         }
 
         List<DataModelDesc> filterModels = new ArrayList<DataModelDesc>();
         for (DataModelDesc modelDesc : models) {
-            boolean isModelMatch = (null == modelName) || modelDesc.getName().toLowerCase().contains(modelName.toLowerCase());
+            boolean isModelMatch = (null == modelName) || modelName.length() == 0 || modelDesc.getName().toLowerCase().equals(modelName.toLowerCase());
 
             if (isModelMatch) {
                 filterModels.add(modelDesc);
@@ -70,8 +69,7 @@ public class ModelService extends BasicService {
 
     public List<DataModelDesc> getModels(final String modelName, final String projectName, final Integer limit, final Integer offset) throws IOException {
 
-        List<DataModelDesc> modelDescs;
-        modelDescs = listAllModels(modelName, projectName);
+        List<DataModelDesc> modelDescs = listAllModels(modelName, projectName);
 
         if (limit == null || offset == null) {
             return modelDescs;


[02/55] [abbrv] kylin git commit: KYLIN-1858 remove all ii related code

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/generated/IIProtos.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/generated/IIProtos.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/generated/IIProtos.java
deleted file mode 100644
index e3f9ca7..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/generated/IIProtos.java
+++ /dev/null
@@ -1,3773 +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.
- */
-
-// Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/protobuf/II.proto
-
-package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated;
-
-public final class IIProtos {
-    private IIProtos() {
-    }
-
-    public static void registerAllExtensions(com.google.protobuf.ExtensionRegistry registry) {
-    }
-
-    public interface IIRequestOrBuilder extends com.google.protobuf.MessageOrBuilder {
-
-        // required bytes type = 1;
-        /**
-         * <code>required bytes type = 1;</code>
-         */
-        boolean hasType();
-
-        /**
-         * <code>required bytes type = 1;</code>
-         */
-        com.google.protobuf.ByteString getType();
-
-        // required bytes filter = 2;
-        /**
-         * <code>required bytes filter = 2;</code>
-         */
-        boolean hasFilter();
-
-        /**
-         * <code>required bytes filter = 2;</code>
-         */
-        com.google.protobuf.ByteString getFilter();
-
-        // required bytes projector = 3;
-        /**
-         * <code>required bytes projector = 3;</code>
-         */
-        boolean hasProjector();
-
-        /**
-         * <code>required bytes projector = 3;</code>
-         */
-        com.google.protobuf.ByteString getProjector();
-
-        // required bytes aggregator = 4;
-        /**
-         * <code>required bytes aggregator = 4;</code>
-         */
-        boolean hasAggregator();
-
-        /**
-         * <code>required bytes aggregator = 4;</code>
-         */
-        com.google.protobuf.ByteString getAggregator();
-
-        // optional bytes tsRange = 5;
-        /**
-         * <code>optional bytes tsRange = 5;</code>
-         */
-        boolean hasTsRange();
-
-        /**
-         * <code>optional bytes tsRange = 5;</code>
-         */
-        com.google.protobuf.ByteString getTsRange();
-    }
-
-    /**
-     * Protobuf type {@code IIRequest}
-     */
-    public static final class IIRequest extends com.google.protobuf.GeneratedMessage implements IIRequestOrBuilder {
-        // Use IIRequest.newBuilder() to construct.
-        private IIRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-            super(builder);
-            this.unknownFields = builder.getUnknownFields();
-        }
-
-        private IIRequest(boolean noInit) {
-            this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance();
-        }
-
-        private static final IIRequest defaultInstance;
-
-        public static IIRequest getDefaultInstance() {
-            return defaultInstance;
-        }
-
-        public IIRequest getDefaultInstanceForType() {
-            return defaultInstance;
-        }
-
-        private final com.google.protobuf.UnknownFieldSet unknownFields;
-
-        @java.lang.Override
-        public final com.google.protobuf.UnknownFieldSet getUnknownFields() {
-            return this.unknownFields;
-        }
-
-        private IIRequest(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-            initFields();
-            int mutable_bitField0_ = 0;
-            com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet.newBuilder();
-            try {
-                boolean done = false;
-                while (!done) {
-                    int tag = input.readTag();
-                    switch (tag) {
-                    case 0:
-                        done = true;
-                        break;
-                    default: {
-                        if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) {
-                            done = true;
-                        }
-                        break;
-                    }
-                    case 10: {
-                        bitField0_ |= 0x00000001;
-                        type_ = input.readBytes();
-                        break;
-                    }
-                    case 18: {
-                        bitField0_ |= 0x00000002;
-                        filter_ = input.readBytes();
-                        break;
-                    }
-                    case 26: {
-                        bitField0_ |= 0x00000004;
-                        projector_ = input.readBytes();
-                        break;
-                    }
-                    case 34: {
-                        bitField0_ |= 0x00000008;
-                        aggregator_ = input.readBytes();
-                        break;
-                    }
-                    case 42: {
-                        bitField0_ |= 0x00000010;
-                        tsRange_ = input.readBytes();
-                        break;
-                    }
-                    }
-                }
-            } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-                throw e.setUnfinishedMessage(this);
-            } catch (java.io.IOException e) {
-                throw new com.google.protobuf.InvalidProtocolBufferException(e.getMessage()).setUnfinishedMessage(this);
-            } finally {
-                this.unknownFields = unknownFields.build();
-                makeExtensionsImmutable();
-            }
-        }
-
-        public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() {
-            return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIRequest_descriptor;
-        }
-
-        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() {
-            return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIRequest_fieldAccessorTable.ensureFieldAccessorsInitialized(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest.class, org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest.Builder.class);
-        }
-
-        public static com.google.protobuf.Parser<IIRequest> PARSER = new com.google.protobuf.AbstractParser<IIRequest>() {
-            public IIRequest parsePartialFrom(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                return new IIRequest(input, extensionRegistry);
-            }
-        };
-
-        @java.lang.Override
-        public com.google.protobuf.Parser<IIRequest> getParserForType() {
-            return PARSER;
-        }
-
-        private int bitField0_;
-        // required bytes type = 1;
-        public static final int TYPE_FIELD_NUMBER = 1;
-        private com.google.protobuf.ByteString type_;
-
-        /**
-         * <code>required bytes type = 1;</code>
-         */
-        public boolean hasType() {
-            return ((bitField0_ & 0x00000001) == 0x00000001);
-        }
-
-        /**
-         * <code>required bytes type = 1;</code>
-         */
-        public com.google.protobuf.ByteString getType() {
-            return type_;
-        }
-
-        // required bytes filter = 2;
-        public static final int FILTER_FIELD_NUMBER = 2;
-        private com.google.protobuf.ByteString filter_;
-
-        /**
-         * <code>required bytes filter = 2;</code>
-         */
-        public boolean hasFilter() {
-            return ((bitField0_ & 0x00000002) == 0x00000002);
-        }
-
-        /**
-         * <code>required bytes filter = 2;</code>
-         */
-        public com.google.protobuf.ByteString getFilter() {
-            return filter_;
-        }
-
-        // required bytes projector = 3;
-        public static final int PROJECTOR_FIELD_NUMBER = 3;
-        private com.google.protobuf.ByteString projector_;
-
-        /**
-         * <code>required bytes projector = 3;</code>
-         */
-        public boolean hasProjector() {
-            return ((bitField0_ & 0x00000004) == 0x00000004);
-        }
-
-        /**
-         * <code>required bytes projector = 3;</code>
-         */
-        public com.google.protobuf.ByteString getProjector() {
-            return projector_;
-        }
-
-        // required bytes aggregator = 4;
-        public static final int AGGREGATOR_FIELD_NUMBER = 4;
-        private com.google.protobuf.ByteString aggregator_;
-
-        /**
-         * <code>required bytes aggregator = 4;</code>
-         */
-        public boolean hasAggregator() {
-            return ((bitField0_ & 0x00000008) == 0x00000008);
-        }
-
-        /**
-         * <code>required bytes aggregator = 4;</code>
-         */
-        public com.google.protobuf.ByteString getAggregator() {
-            return aggregator_;
-        }
-
-        // optional bytes tsRange = 5;
-        public static final int TSRANGE_FIELD_NUMBER = 5;
-        private com.google.protobuf.ByteString tsRange_;
-
-        /**
-         * <code>optional bytes tsRange = 5;</code>
-         */
-        public boolean hasTsRange() {
-            return ((bitField0_ & 0x00000010) == 0x00000010);
-        }
-
-        /**
-         * <code>optional bytes tsRange = 5;</code>
-         */
-        public com.google.protobuf.ByteString getTsRange() {
-            return tsRange_;
-        }
-
-        private void initFields() {
-            type_ = com.google.protobuf.ByteString.EMPTY;
-            filter_ = com.google.protobuf.ByteString.EMPTY;
-            projector_ = com.google.protobuf.ByteString.EMPTY;
-            aggregator_ = com.google.protobuf.ByteString.EMPTY;
-            tsRange_ = com.google.protobuf.ByteString.EMPTY;
-        }
-
-        private byte memoizedIsInitialized = -1;
-
-        public final boolean isInitialized() {
-            byte isInitialized = memoizedIsInitialized;
-            if (isInitialized != -1)
-                return isInitialized == 1;
-
-            if (!hasType()) {
-                memoizedIsInitialized = 0;
-                return false;
-            }
-            if (!hasFilter()) {
-                memoizedIsInitialized = 0;
-                return false;
-            }
-            if (!hasProjector()) {
-                memoizedIsInitialized = 0;
-                return false;
-            }
-            if (!hasAggregator()) {
-                memoizedIsInitialized = 0;
-                return false;
-            }
-            memoizedIsInitialized = 1;
-            return true;
-        }
-
-        public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException {
-            getSerializedSize();
-            if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                output.writeBytes(1, type_);
-            }
-            if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                output.writeBytes(2, filter_);
-            }
-            if (((bitField0_ & 0x00000004) == 0x00000004)) {
-                output.writeBytes(3, projector_);
-            }
-            if (((bitField0_ & 0x00000008) == 0x00000008)) {
-                output.writeBytes(4, aggregator_);
-            }
-            if (((bitField0_ & 0x00000010) == 0x00000010)) {
-                output.writeBytes(5, tsRange_);
-            }
-            getUnknownFields().writeTo(output);
-        }
-
-        private int memoizedSerializedSize = -1;
-
-        public int getSerializedSize() {
-            int size = memoizedSerializedSize;
-            if (size != -1)
-                return size;
-
-            size = 0;
-            if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                size += com.google.protobuf.CodedOutputStream.computeBytesSize(1, type_);
-            }
-            if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                size += com.google.protobuf.CodedOutputStream.computeBytesSize(2, filter_);
-            }
-            if (((bitField0_ & 0x00000004) == 0x00000004)) {
-                size += com.google.protobuf.CodedOutputStream.computeBytesSize(3, projector_);
-            }
-            if (((bitField0_ & 0x00000008) == 0x00000008)) {
-                size += com.google.protobuf.CodedOutputStream.computeBytesSize(4, aggregator_);
-            }
-            if (((bitField0_ & 0x00000010) == 0x00000010)) {
-                size += com.google.protobuf.CodedOutputStream.computeBytesSize(5, tsRange_);
-            }
-            size += getUnknownFields().getSerializedSize();
-            memoizedSerializedSize = size;
-            return size;
-        }
-
-        private static final long serialVersionUID = 0L;
-
-        @java.lang.Override
-        protected java.lang.Object writeReplace() throws java.io.ObjectStreamException {
-            return super.writeReplace();
-        }
-
-        @java.lang.Override
-        public boolean equals(final java.lang.Object obj) {
-            if (obj == this) {
-                return true;
-            }
-            if (!(obj instanceof org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest)) {
-                return super.equals(obj);
-            }
-            org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest other = (org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest) obj;
-
-            boolean result = true;
-            result = result && (hasType() == other.hasType());
-            if (hasType()) {
-                result = result && getType().equals(other.getType());
-            }
-            result = result && (hasFilter() == other.hasFilter());
-            if (hasFilter()) {
-                result = result && getFilter().equals(other.getFilter());
-            }
-            result = result && (hasProjector() == other.hasProjector());
-            if (hasProjector()) {
-                result = result && getProjector().equals(other.getProjector());
-            }
-            result = result && (hasAggregator() == other.hasAggregator());
-            if (hasAggregator()) {
-                result = result && getAggregator().equals(other.getAggregator());
-            }
-            result = result && (hasTsRange() == other.hasTsRange());
-            if (hasTsRange()) {
-                result = result && getTsRange().equals(other.getTsRange());
-            }
-            result = result && getUnknownFields().equals(other.getUnknownFields());
-            return result;
-        }
-
-        private int memoizedHashCode = 0;
-
-        @java.lang.Override
-        public int hashCode() {
-            if (memoizedHashCode != 0) {
-                return memoizedHashCode;
-            }
-            int hash = 41;
-            hash = (19 * hash) + getDescriptorForType().hashCode();
-            if (hasType()) {
-                hash = (37 * hash) + TYPE_FIELD_NUMBER;
-                hash = (53 * hash) + getType().hashCode();
-            }
-            if (hasFilter()) {
-                hash = (37 * hash) + FILTER_FIELD_NUMBER;
-                hash = (53 * hash) + getFilter().hashCode();
-            }
-            if (hasProjector()) {
-                hash = (37 * hash) + PROJECTOR_FIELD_NUMBER;
-                hash = (53 * hash) + getProjector().hashCode();
-            }
-            if (hasAggregator()) {
-                hash = (37 * hash) + AGGREGATOR_FIELD_NUMBER;
-                hash = (53 * hash) + getAggregator().hashCode();
-            }
-            if (hasTsRange()) {
-                hash = (37 * hash) + TSRANGE_FIELD_NUMBER;
-                hash = (53 * hash) + getTsRange().hashCode();
-            }
-            hash = (29 * hash) + getUnknownFields().hashCode();
-            memoizedHashCode = hash;
-            return hash;
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseFrom(com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException {
-            return PARSER.parseFrom(data);
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseFrom(com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-            return PARSER.parseFrom(data, extensionRegistry);
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException {
-            return PARSER.parseFrom(data);
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-            return PARSER.parseFrom(data, extensionRegistry);
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseFrom(java.io.InputStream input) throws java.io.IOException {
-            return PARSER.parseFrom(input);
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseFrom(java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-            return PARSER.parseFrom(input, extensionRegistry);
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException {
-            return PARSER.parseDelimitedFrom(input);
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseDelimitedFrom(java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-            return PARSER.parseDelimitedFrom(input, extensionRegistry);
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException {
-            return PARSER.parseFrom(input);
-        }
-
-        public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parseFrom(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-            return PARSER.parseFrom(input, extensionRegistry);
-        }
-
-        public static Builder newBuilder() {
-            return Builder.create();
-        }
-
-        public Builder newBuilderForType() {
-            return newBuilder();
-        }
-
-        public static Builder newBuilder(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest prototype) {
-            return newBuilder().mergeFrom(prototype);
-        }
-
-        public Builder toBuilder() {
-            return newBuilder(this);
-        }
-
-        @java.lang.Override
-        protected Builder newBuilderForType(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-            Builder builder = new Builder(parent);
-            return builder;
-        }
-
-        /**
-         * Protobuf type {@code IIRequest}
-         */
-        public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder<Builder> implements org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequestOrBuilder {
-            public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() {
-                return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIRequest_descriptor;
-            }
-
-            protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() {
-                return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIRequest_fieldAccessorTable.ensureFieldAccessorsInitialized(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest.class, org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest.Builder.class);
-            }
-
-            // Construct using org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest.newBuilder()
-            private Builder() {
-                maybeForceBuilderInitialization();
-            }
-
-            private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-                super(parent);
-                maybeForceBuilderInitialization();
-            }
-
-            private void maybeForceBuilderInitialization() {
-                if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-                }
-            }
-
-            private static Builder create() {
-                return new Builder();
-            }
-
-            public Builder clear() {
-                super.clear();
-                type_ = com.google.protobuf.ByteString.EMPTY;
-                bitField0_ = (bitField0_ & ~0x00000001);
-                filter_ = com.google.protobuf.ByteString.EMPTY;
-                bitField0_ = (bitField0_ & ~0x00000002);
-                projector_ = com.google.protobuf.ByteString.EMPTY;
-                bitField0_ = (bitField0_ & ~0x00000004);
-                aggregator_ = com.google.protobuf.ByteString.EMPTY;
-                bitField0_ = (bitField0_ & ~0x00000008);
-                tsRange_ = com.google.protobuf.ByteString.EMPTY;
-                bitField0_ = (bitField0_ & ~0x00000010);
-                return this;
-            }
-
-            public Builder clone() {
-                return create().mergeFrom(buildPartial());
-            }
-
-            public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() {
-                return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIRequest_descriptor;
-            }
-
-            public org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest getDefaultInstanceForType() {
-                return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest.getDefaultInstance();
-            }
-
-            public org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest build() {
-                org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest result = buildPartial();
-                if (!result.isInitialized()) {
-                    throw newUninitializedMessageException(result);
-                }
-                return result;
-            }
-
-            public org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest buildPartial() {
-                org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest result = new org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest(this);
-                int from_bitField0_ = bitField0_;
-                int to_bitField0_ = 0;
-                if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-                    to_bitField0_ |= 0x00000001;
-                }
-                result.type_ = type_;
-                if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-                    to_bitField0_ |= 0x00000002;
-                }
-                result.filter_ = filter_;
-                if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-                    to_bitField0_ |= 0x00000004;
-                }
-                result.projector_ = projector_;
-                if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-                    to_bitField0_ |= 0x00000008;
-                }
-                result.aggregator_ = aggregator_;
-                if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-                    to_bitField0_ |= 0x00000010;
-                }
-                result.tsRange_ = tsRange_;
-                result.bitField0_ = to_bitField0_;
-                onBuilt();
-                return result;
-            }
-
-            public Builder mergeFrom(com.google.protobuf.Message other) {
-                if (other instanceof org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest) {
-                    return mergeFrom((org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest) other);
-                } else {
-                    super.mergeFrom(other);
-                    return this;
-                }
-            }
-
-            public Builder mergeFrom(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest other) {
-                if (other == org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest.getDefaultInstance())
-                    return this;
-                if (other.hasType()) {
-                    setType(other.getType());
-                }
-                if (other.hasFilter()) {
-                    setFilter(other.getFilter());
-                }
-                if (other.hasProjector()) {
-                    setProjector(other.getProjector());
-                }
-                if (other.hasAggregator()) {
-                    setAggregator(other.getAggregator());
-                }
-                if (other.hasTsRange()) {
-                    setTsRange(other.getTsRange());
-                }
-                this.mergeUnknownFields(other.getUnknownFields());
-                return this;
-            }
-
-            public final boolean isInitialized() {
-                if (!hasType()) {
-
-                    return false;
-                }
-                if (!hasFilter()) {
-
-                    return false;
-                }
-                if (!hasProjector()) {
-
-                    return false;
-                }
-                if (!hasAggregator()) {
-
-                    return false;
-                }
-                return true;
-            }
-
-            public Builder mergeFrom(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-                org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest parsedMessage = null;
-                try {
-                    parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-                } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-                    parsedMessage = (org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIRequest) e.getUnfinishedMessage();
-                    throw e;
-                } finally {
-                    if (parsedMessage != null) {
-                        mergeFrom(parsedMessage);
-                    }
-                }
-                return this;
-            }
-
-            private int bitField0_;
-
-            // required bytes type = 1;
-            private com.google.protobuf.ByteString type_ = com.google.protobuf.ByteString.EMPTY;
-
-            /**
-             * <code>required bytes type = 1;</code>
-             */
-            public boolean hasType() {
-                return ((bitField0_ & 0x00000001) == 0x00000001);
-            }
-
-            /**
-             * <code>required bytes type = 1;</code>
-             */
-            public com.google.protobuf.ByteString getType() {
-                return type_;
-            }
-
-            /**
-             * <code>required bytes type = 1;</code>
-             */
-            public Builder setType(com.google.protobuf.ByteString value) {
-                if (value == null) {
-                    throw new NullPointerException();
-                }
-                bitField0_ |= 0x00000001;
-                type_ = value;
-                onChanged();
-                return this;
-            }
-
-            /**
-             * <code>required bytes type = 1;</code>
-             */
-            public Builder clearType() {
-                bitField0_ = (bitField0_ & ~0x00000001);
-                type_ = getDefaultInstance().getType();
-                onChanged();
-                return this;
-            }
-
-            // required bytes filter = 2;
-            private com.google.protobuf.ByteString filter_ = com.google.protobuf.ByteString.EMPTY;
-
-            /**
-             * <code>required bytes filter = 2;</code>
-             */
-            public boolean hasFilter() {
-                return ((bitField0_ & 0x00000002) == 0x00000002);
-            }
-
-            /**
-             * <code>required bytes filter = 2;</code>
-             */
-            public com.google.protobuf.ByteString getFilter() {
-                return filter_;
-            }
-
-            /**
-             * <code>required bytes filter = 2;</code>
-             */
-            public Builder setFilter(com.google.protobuf.ByteString value) {
-                if (value == null) {
-                    throw new NullPointerException();
-                }
-                bitField0_ |= 0x00000002;
-                filter_ = value;
-                onChanged();
-                return this;
-            }
-
-            /**
-             * <code>required bytes filter = 2;</code>
-             */
-            public Builder clearFilter() {
-                bitField0_ = (bitField0_ & ~0x00000002);
-                filter_ = getDefaultInstance().getFilter();
-                onChanged();
-                return this;
-            }
-
-            // required bytes projector = 3;
-            private com.google.protobuf.ByteString projector_ = com.google.protobuf.ByteString.EMPTY;
-
-            /**
-             * <code>required bytes projector = 3;</code>
-             */
-            public boolean hasProjector() {
-                return ((bitField0_ & 0x00000004) == 0x00000004);
-            }
-
-            /**
-             * <code>required bytes projector = 3;</code>
-             */
-            public com.google.protobuf.ByteString getProjector() {
-                return projector_;
-            }
-
-            /**
-             * <code>required bytes projector = 3;</code>
-             */
-            public Builder setProjector(com.google.protobuf.ByteString value) {
-                if (value == null) {
-                    throw new NullPointerException();
-                }
-                bitField0_ |= 0x00000004;
-                projector_ = value;
-                onChanged();
-                return this;
-            }
-
-            /**
-             * <code>required bytes projector = 3;</code>
-             */
-            public Builder clearProjector() {
-                bitField0_ = (bitField0_ & ~0x00000004);
-                projector_ = getDefaultInstance().getProjector();
-                onChanged();
-                return this;
-            }
-
-            // required bytes aggregator = 4;
-            private com.google.protobuf.ByteString aggregator_ = com.google.protobuf.ByteString.EMPTY;
-
-            /**
-             * <code>required bytes aggregator = 4;</code>
-             */
-            public boolean hasAggregator() {
-                return ((bitField0_ & 0x00000008) == 0x00000008);
-            }
-
-            /**
-             * <code>required bytes aggregator = 4;</code>
-             */
-            public com.google.protobuf.ByteString getAggregator() {
-                return aggregator_;
-            }
-
-            /**
-             * <code>required bytes aggregator = 4;</code>
-             */
-            public Builder setAggregator(com.google.protobuf.ByteString value) {
-                if (value == null) {
-                    throw new NullPointerException();
-                }
-                bitField0_ |= 0x00000008;
-                aggregator_ = value;
-                onChanged();
-                return this;
-            }
-
-            /**
-             * <code>required bytes aggregator = 4;</code>
-             */
-            public Builder clearAggregator() {
-                bitField0_ = (bitField0_ & ~0x00000008);
-                aggregator_ = getDefaultInstance().getAggregator();
-                onChanged();
-                return this;
-            }
-
-            // optional bytes tsRange = 5;
-            private com.google.protobuf.ByteString tsRange_ = com.google.protobuf.ByteString.EMPTY;
-
-            /**
-             * <code>optional bytes tsRange = 5;</code>
-             */
-            public boolean hasTsRange() {
-                return ((bitField0_ & 0x00000010) == 0x00000010);
-            }
-
-            /**
-             * <code>optional bytes tsRange = 5;</code>
-             */
-            public com.google.protobuf.ByteString getTsRange() {
-                return tsRange_;
-            }
-
-            /**
-             * <code>optional bytes tsRange = 5;</code>
-             */
-            public Builder setTsRange(com.google.protobuf.ByteString value) {
-                if (value == null) {
-                    throw new NullPointerException();
-                }
-                bitField0_ |= 0x00000010;
-                tsRange_ = value;
-                onChanged();
-                return this;
-            }
-
-            /**
-             * <code>optional bytes tsRange = 5;</code>
-             */
-            public Builder clearTsRange() {
-                bitField0_ = (bitField0_ & ~0x00000010);
-                tsRange_ = getDefaultInstance().getTsRange();
-                onChanged();
-                return this;
-            }
-
-            // @@protoc_insertion_point(builder_scope:IIRequest)
-        }
-
-        static {
-            defaultInstance = new IIRequest(true);
-            defaultInstance.initFields();
-        }
-
-        // @@protoc_insertion_point(class_scope:IIRequest)
-    }
-
-    public interface IIResponseInternalOrBuilder extends com.google.protobuf.MessageOrBuilder {
-
-        // repeated .IIResponseInternal.IIRow rows = 1;
-        /**
-         * <code>repeated .IIResponseInternal.IIRow rows = 1;</code>
-         */
-        java.util.List<org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow> getRowsList();
-
-        /**
-         * <code>repeated .IIResponseInternal.IIRow rows = 1;</code>
-         */
-        org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow getRows(int index);
-
-        /**
-         * <code>repeated .IIResponseInternal.IIRow rows = 1;</code>
-         */
-        int getRowsCount();
-
-        /**
-         * <code>repeated .IIResponseInternal.IIRow rows = 1;</code>
-         */
-        java.util.List<? extends org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRowOrBuilder> getRowsOrBuilderList();
-
-        /**
-         * <code>repeated .IIResponseInternal.IIRow rows = 1;</code>
-         */
-        org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRowOrBuilder getRowsOrBuilder(int index);
-
-        // required .IIResponseInternal.Stats stats = 2;
-        /**
-         * <code>required .IIResponseInternal.Stats stats = 2;</code>
-         */
-        boolean hasStats();
-
-        /**
-         * <code>required .IIResponseInternal.Stats stats = 2;</code>
-         */
-        org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats getStats();
-
-        /**
-         * <code>required .IIResponseInternal.Stats stats = 2;</code>
-         */
-        org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.StatsOrBuilder getStatsOrBuilder();
-    }
-
-    /**
-     * Protobuf type {@code IIResponseInternal}
-     */
-    public static final class IIResponseInternal extends com.google.protobuf.GeneratedMessage implements IIResponseInternalOrBuilder {
-        // Use IIResponseInternal.newBuilder() to construct.
-        private IIResponseInternal(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-            super(builder);
-            this.unknownFields = builder.getUnknownFields();
-        }
-
-        private IIResponseInternal(boolean noInit) {
-            this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance();
-        }
-
-        private static final IIResponseInternal defaultInstance;
-
-        public static IIResponseInternal getDefaultInstance() {
-            return defaultInstance;
-        }
-
-        public IIResponseInternal getDefaultInstanceForType() {
-            return defaultInstance;
-        }
-
-        private final com.google.protobuf.UnknownFieldSet unknownFields;
-
-        @java.lang.Override
-        public final com.google.protobuf.UnknownFieldSet getUnknownFields() {
-            return this.unknownFields;
-        }
-
-        private IIResponseInternal(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-            initFields();
-            int mutable_bitField0_ = 0;
-            com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet.newBuilder();
-            try {
-                boolean done = false;
-                while (!done) {
-                    int tag = input.readTag();
-                    switch (tag) {
-                    case 0:
-                        done = true;
-                        break;
-                    default: {
-                        if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) {
-                            done = true;
-                        }
-                        break;
-                    }
-                    case 10: {
-                        if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                            rows_ = new java.util.ArrayList<org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow>();
-                            mutable_bitField0_ |= 0x00000001;
-                        }
-                        rows_.add(input.readMessage(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow.PARSER, extensionRegistry));
-                        break;
-                    }
-                    case 18: {
-                        org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats.Builder subBuilder = null;
-                        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                            subBuilder = stats_.toBuilder();
-                        }
-                        stats_ = input.readMessage(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats.PARSER, extensionRegistry);
-                        if (subBuilder != null) {
-                            subBuilder.mergeFrom(stats_);
-                            stats_ = subBuilder.buildPartial();
-                        }
-                        bitField0_ |= 0x00000001;
-                        break;
-                    }
-                    }
-                }
-            } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-                throw e.setUnfinishedMessage(this);
-            } catch (java.io.IOException e) {
-                throw new com.google.protobuf.InvalidProtocolBufferException(e.getMessage()).setUnfinishedMessage(this);
-            } finally {
-                if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                    rows_ = java.util.Collections.unmodifiableList(rows_);
-                }
-                this.unknownFields = unknownFields.build();
-                makeExtensionsImmutable();
-            }
-        }
-
-        public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() {
-            return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIResponseInternal_descriptor;
-        }
-
-        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() {
-            return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIResponseInternal_fieldAccessorTable.ensureFieldAccessorsInitialized(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.class, org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Builder.class);
-        }
-
-        public static com.google.protobuf.Parser<IIResponseInternal> PARSER = new com.google.protobuf.AbstractParser<IIResponseInternal>() {
-            public IIResponseInternal parsePartialFrom(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                return new IIResponseInternal(input, extensionRegistry);
-            }
-        };
-
-        @java.lang.Override
-        public com.google.protobuf.Parser<IIResponseInternal> getParserForType() {
-            return PARSER;
-        }
-
-        public interface IIRowOrBuilder extends com.google.protobuf.MessageOrBuilder {
-
-            // required bytes columns = 1;
-            /**
-             * <code>required bytes columns = 1;</code>
-             */
-            boolean hasColumns();
-
-            /**
-             * <code>required bytes columns = 1;</code>
-             */
-            com.google.protobuf.ByteString getColumns();
-
-            // optional bytes measures = 2;
-            /**
-             * <code>optional bytes measures = 2;</code>
-             */
-            boolean hasMeasures();
-
-            /**
-             * <code>optional bytes measures = 2;</code>
-             */
-            com.google.protobuf.ByteString getMeasures();
-        }
-
-        /**
-         * Protobuf type {@code IIResponseInternal.IIRow}
-         */
-        public static final class IIRow extends com.google.protobuf.GeneratedMessage implements IIRowOrBuilder {
-            // Use IIRow.newBuilder() to construct.
-            private IIRow(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-                super(builder);
-                this.unknownFields = builder.getUnknownFields();
-            }
-
-            private IIRow(boolean noInit) {
-                this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance();
-            }
-
-            private static final IIRow defaultInstance;
-
-            public static IIRow getDefaultInstance() {
-                return defaultInstance;
-            }
-
-            public IIRow getDefaultInstanceForType() {
-                return defaultInstance;
-            }
-
-            private final com.google.protobuf.UnknownFieldSet unknownFields;
-
-            @java.lang.Override
-            public final com.google.protobuf.UnknownFieldSet getUnknownFields() {
-                return this.unknownFields;
-            }
-
-            private IIRow(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                initFields();
-                int mutable_bitField0_ = 0;
-                com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet.newBuilder();
-                try {
-                    boolean done = false;
-                    while (!done) {
-                        int tag = input.readTag();
-                        switch (tag) {
-                        case 0:
-                            done = true;
-                            break;
-                        default: {
-                            if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) {
-                                done = true;
-                            }
-                            break;
-                        }
-                        case 10: {
-                            bitField0_ |= 0x00000001;
-                            columns_ = input.readBytes();
-                            break;
-                        }
-                        case 18: {
-                            bitField0_ |= 0x00000002;
-                            measures_ = input.readBytes();
-                            break;
-                        }
-                        }
-                    }
-                } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-                    throw e.setUnfinishedMessage(this);
-                } catch (java.io.IOException e) {
-                    throw new com.google.protobuf.InvalidProtocolBufferException(e.getMessage()).setUnfinishedMessage(this);
-                } finally {
-                    this.unknownFields = unknownFields.build();
-                    makeExtensionsImmutable();
-                }
-            }
-
-            public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() {
-                return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIResponseInternal_IIRow_descriptor;
-            }
-
-            protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() {
-                return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIResponseInternal_IIRow_fieldAccessorTable.ensureFieldAccessorsInitialized(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow.class, org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow.Builder.class);
-            }
-
-            public static com.google.protobuf.Parser<IIRow> PARSER = new com.google.protobuf.AbstractParser<IIRow>() {
-                public IIRow parsePartialFrom(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                    return new IIRow(input, extensionRegistry);
-                }
-            };
-
-            @java.lang.Override
-            public com.google.protobuf.Parser<IIRow> getParserForType() {
-                return PARSER;
-            }
-
-            private int bitField0_;
-            // required bytes columns = 1;
-            public static final int COLUMNS_FIELD_NUMBER = 1;
-            private com.google.protobuf.ByteString columns_;
-
-            /**
-             * <code>required bytes columns = 1;</code>
-             */
-            public boolean hasColumns() {
-                return ((bitField0_ & 0x00000001) == 0x00000001);
-            }
-
-            /**
-             * <code>required bytes columns = 1;</code>
-             */
-            public com.google.protobuf.ByteString getColumns() {
-                return columns_;
-            }
-
-            // optional bytes measures = 2;
-            public static final int MEASURES_FIELD_NUMBER = 2;
-            private com.google.protobuf.ByteString measures_;
-
-            /**
-             * <code>optional bytes measures = 2;</code>
-             */
-            public boolean hasMeasures() {
-                return ((bitField0_ & 0x00000002) == 0x00000002);
-            }
-
-            /**
-             * <code>optional bytes measures = 2;</code>
-             */
-            public com.google.protobuf.ByteString getMeasures() {
-                return measures_;
-            }
-
-            private void initFields() {
-                columns_ = com.google.protobuf.ByteString.EMPTY;
-                measures_ = com.google.protobuf.ByteString.EMPTY;
-            }
-
-            private byte memoizedIsInitialized = -1;
-
-            public final boolean isInitialized() {
-                byte isInitialized = memoizedIsInitialized;
-                if (isInitialized != -1)
-                    return isInitialized == 1;
-
-                if (!hasColumns()) {
-                    memoizedIsInitialized = 0;
-                    return false;
-                }
-                memoizedIsInitialized = 1;
-                return true;
-            }
-
-            public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException {
-                getSerializedSize();
-                if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                    output.writeBytes(1, columns_);
-                }
-                if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                    output.writeBytes(2, measures_);
-                }
-                getUnknownFields().writeTo(output);
-            }
-
-            private int memoizedSerializedSize = -1;
-
-            public int getSerializedSize() {
-                int size = memoizedSerializedSize;
-                if (size != -1)
-                    return size;
-
-                size = 0;
-                if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                    size += com.google.protobuf.CodedOutputStream.computeBytesSize(1, columns_);
-                }
-                if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                    size += com.google.protobuf.CodedOutputStream.computeBytesSize(2, measures_);
-                }
-                size += getUnknownFields().getSerializedSize();
-                memoizedSerializedSize = size;
-                return size;
-            }
-
-            private static final long serialVersionUID = 0L;
-
-            @java.lang.Override
-            protected java.lang.Object writeReplace() throws java.io.ObjectStreamException {
-                return super.writeReplace();
-            }
-
-            @java.lang.Override
-            public boolean equals(final java.lang.Object obj) {
-                if (obj == this) {
-                    return true;
-                }
-                if (!(obj instanceof org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow)) {
-                    return super.equals(obj);
-                }
-                org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow other = (org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow) obj;
-
-                boolean result = true;
-                result = result && (hasColumns() == other.hasColumns());
-                if (hasColumns()) {
-                    result = result && getColumns().equals(other.getColumns());
-                }
-                result = result && (hasMeasures() == other.hasMeasures());
-                if (hasMeasures()) {
-                    result = result && getMeasures().equals(other.getMeasures());
-                }
-                result = result && getUnknownFields().equals(other.getUnknownFields());
-                return result;
-            }
-
-            private int memoizedHashCode = 0;
-
-            @java.lang.Override
-            public int hashCode() {
-                if (memoizedHashCode != 0) {
-                    return memoizedHashCode;
-                }
-                int hash = 41;
-                hash = (19 * hash) + getDescriptorForType().hashCode();
-                if (hasColumns()) {
-                    hash = (37 * hash) + COLUMNS_FIELD_NUMBER;
-                    hash = (53 * hash) + getColumns().hashCode();
-                }
-                if (hasMeasures()) {
-                    hash = (37 * hash) + MEASURES_FIELD_NUMBER;
-                    hash = (53 * hash) + getMeasures().hashCode();
-                }
-                hash = (29 * hash) + getUnknownFields().hashCode();
-                memoizedHashCode = hash;
-                return hash;
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseFrom(com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException {
-                return PARSER.parseFrom(data);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseFrom(com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                return PARSER.parseFrom(data, extensionRegistry);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException {
-                return PARSER.parseFrom(data);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                return PARSER.parseFrom(data, extensionRegistry);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseFrom(java.io.InputStream input) throws java.io.IOException {
-                return PARSER.parseFrom(input);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseFrom(java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-                return PARSER.parseFrom(input, extensionRegistry);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException {
-                return PARSER.parseDelimitedFrom(input);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseDelimitedFrom(java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-                return PARSER.parseDelimitedFrom(input, extensionRegistry);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException {
-                return PARSER.parseFrom(input);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parseFrom(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-                return PARSER.parseFrom(input, extensionRegistry);
-            }
-
-            public static Builder newBuilder() {
-                return Builder.create();
-            }
-
-            public Builder newBuilderForType() {
-                return newBuilder();
-            }
-
-            public static Builder newBuilder(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow prototype) {
-                return newBuilder().mergeFrom(prototype);
-            }
-
-            public Builder toBuilder() {
-                return newBuilder(this);
-            }
-
-            @java.lang.Override
-            protected Builder newBuilderForType(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-                Builder builder = new Builder(parent);
-                return builder;
-            }
-
-            /**
-             * Protobuf type {@code IIResponseInternal.IIRow}
-             */
-            public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder<Builder> implements org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRowOrBuilder {
-                public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() {
-                    return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIResponseInternal_IIRow_descriptor;
-                }
-
-                protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() {
-                    return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIResponseInternal_IIRow_fieldAccessorTable.ensureFieldAccessorsInitialized(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow.class, org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow.Builder.class);
-                }
-
-                // Construct using org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow.newBuilder()
-                private Builder() {
-                    maybeForceBuilderInitialization();
-                }
-
-                private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-                    super(parent);
-                    maybeForceBuilderInitialization();
-                }
-
-                private void maybeForceBuilderInitialization() {
-                    if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-                    }
-                }
-
-                private static Builder create() {
-                    return new Builder();
-                }
-
-                public Builder clear() {
-                    super.clear();
-                    columns_ = com.google.protobuf.ByteString.EMPTY;
-                    bitField0_ = (bitField0_ & ~0x00000001);
-                    measures_ = com.google.protobuf.ByteString.EMPTY;
-                    bitField0_ = (bitField0_ & ~0x00000002);
-                    return this;
-                }
-
-                public Builder clone() {
-                    return create().mergeFrom(buildPartial());
-                }
-
-                public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() {
-                    return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIResponseInternal_IIRow_descriptor;
-                }
-
-                public org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow getDefaultInstanceForType() {
-                    return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow.getDefaultInstance();
-                }
-
-                public org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow build() {
-                    org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow result = buildPartial();
-                    if (!result.isInitialized()) {
-                        throw newUninitializedMessageException(result);
-                    }
-                    return result;
-                }
-
-                public org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow buildPartial() {
-                    org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow result = new org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow(this);
-                    int from_bitField0_ = bitField0_;
-                    int to_bitField0_ = 0;
-                    if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-                        to_bitField0_ |= 0x00000001;
-                    }
-                    result.columns_ = columns_;
-                    if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-                        to_bitField0_ |= 0x00000002;
-                    }
-                    result.measures_ = measures_;
-                    result.bitField0_ = to_bitField0_;
-                    onBuilt();
-                    return result;
-                }
-
-                public Builder mergeFrom(com.google.protobuf.Message other) {
-                    if (other instanceof org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow) {
-                        return mergeFrom((org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow) other);
-                    } else {
-                        super.mergeFrom(other);
-                        return this;
-                    }
-                }
-
-                public Builder mergeFrom(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow other) {
-                    if (other == org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow.getDefaultInstance())
-                        return this;
-                    if (other.hasColumns()) {
-                        setColumns(other.getColumns());
-                    }
-                    if (other.hasMeasures()) {
-                        setMeasures(other.getMeasures());
-                    }
-                    this.mergeUnknownFields(other.getUnknownFields());
-                    return this;
-                }
-
-                public final boolean isInitialized() {
-                    if (!hasColumns()) {
-
-                        return false;
-                    }
-                    return true;
-                }
-
-                public Builder mergeFrom(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-                    org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow parsedMessage = null;
-                    try {
-                        parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-                    } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-                        parsedMessage = (org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.IIRow) e.getUnfinishedMessage();
-                        throw e;
-                    } finally {
-                        if (parsedMessage != null) {
-                            mergeFrom(parsedMessage);
-                        }
-                    }
-                    return this;
-                }
-
-                private int bitField0_;
-
-                // required bytes columns = 1;
-                private com.google.protobuf.ByteString columns_ = com.google.protobuf.ByteString.EMPTY;
-
-                /**
-                 * <code>required bytes columns = 1;</code>
-                 */
-                public boolean hasColumns() {
-                    return ((bitField0_ & 0x00000001) == 0x00000001);
-                }
-
-                /**
-                 * <code>required bytes columns = 1;</code>
-                 */
-                public com.google.protobuf.ByteString getColumns() {
-                    return columns_;
-                }
-
-                /**
-                 * <code>required bytes columns = 1;</code>
-                 */
-                public Builder setColumns(com.google.protobuf.ByteString value) {
-                    if (value == null) {
-                        throw new NullPointerException();
-                    }
-                    bitField0_ |= 0x00000001;
-                    columns_ = value;
-                    onChanged();
-                    return this;
-                }
-
-                /**
-                 * <code>required bytes columns = 1;</code>
-                 */
-                public Builder clearColumns() {
-                    bitField0_ = (bitField0_ & ~0x00000001);
-                    columns_ = getDefaultInstance().getColumns();
-                    onChanged();
-                    return this;
-                }
-
-                // optional bytes measures = 2;
-                private com.google.protobuf.ByteString measures_ = com.google.protobuf.ByteString.EMPTY;
-
-                /**
-                 * <code>optional bytes measures = 2;</code>
-                 */
-                public boolean hasMeasures() {
-                    return ((bitField0_ & 0x00000002) == 0x00000002);
-                }
-
-                /**
-                 * <code>optional bytes measures = 2;</code>
-                 */
-                public com.google.protobuf.ByteString getMeasures() {
-                    return measures_;
-                }
-
-                /**
-                 * <code>optional bytes measures = 2;</code>
-                 */
-                public Builder setMeasures(com.google.protobuf.ByteString value) {
-                    if (value == null) {
-                        throw new NullPointerException();
-                    }
-                    bitField0_ |= 0x00000002;
-                    measures_ = value;
-                    onChanged();
-                    return this;
-                }
-
-                /**
-                 * <code>optional bytes measures = 2;</code>
-                 */
-                public Builder clearMeasures() {
-                    bitField0_ = (bitField0_ & ~0x00000002);
-                    measures_ = getDefaultInstance().getMeasures();
-                    onChanged();
-                    return this;
-                }
-
-                // @@protoc_insertion_point(builder_scope:IIResponseInternal.IIRow)
-            }
-
-            static {
-                defaultInstance = new IIRow(true);
-                defaultInstance.initFields();
-            }
-
-            // @@protoc_insertion_point(class_scope:IIResponseInternal.IIRow)
-        }
-
-        public interface StatsOrBuilder extends com.google.protobuf.MessageOrBuilder {
-
-            // optional int32 myShard = 1;
-            /**
-             * <code>optional int32 myShard = 1;</code>
-             */
-            boolean hasMyShard();
-
-            /**
-             * <code>optional int32 myShard = 1;</code>
-             */
-            int getMyShard();
-
-            // optional int64 latestDataTime = 2;
-            /**
-             * <code>optional int64 latestDataTime = 2;</code>
-             */
-            boolean hasLatestDataTime();
-
-            /**
-             * <code>optional int64 latestDataTime = 2;</code>
-             */
-            long getLatestDataTime();
-
-            // optional int64 serviceStartTime = 3;
-            /**
-             * <code>optional int64 serviceStartTime = 3;</code>
-             */
-            boolean hasServiceStartTime();
-
-            /**
-             * <code>optional int64 serviceStartTime = 3;</code>
-             */
-            long getServiceStartTime();
-
-            // optional int64 serviceEndTime = 4;
-            /**
-             * <code>optional int64 serviceEndTime = 4;</code>
-             */
-            boolean hasServiceEndTime();
-
-            /**
-             * <code>optional int64 serviceEndTime = 4;</code>
-             */
-            long getServiceEndTime();
-
-            // optional int32 scannedSlices = 5;
-            /**
-             * <code>optional int32 scannedSlices = 5;</code>
-             */
-            boolean hasScannedSlices();
-
-            /**
-             * <code>optional int32 scannedSlices = 5;</code>
-             */
-            int getScannedSlices();
-        }
-
-        /**
-         * Protobuf type {@code IIResponseInternal.Stats}
-         *
-         * <pre>
-         *all entries in this struct be optional to conveniently add more entries in the future
-         * </pre>
-         */
-        public static final class Stats extends com.google.protobuf.GeneratedMessage implements StatsOrBuilder {
-            // Use Stats.newBuilder() to construct.
-            private Stats(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-                super(builder);
-                this.unknownFields = builder.getUnknownFields();
-            }
-
-            private Stats(boolean noInit) {
-                this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance();
-            }
-
-            private static final Stats defaultInstance;
-
-            public static Stats getDefaultInstance() {
-                return defaultInstance;
-            }
-
-            public Stats getDefaultInstanceForType() {
-                return defaultInstance;
-            }
-
-            private final com.google.protobuf.UnknownFieldSet unknownFields;
-
-            @java.lang.Override
-            public final com.google.protobuf.UnknownFieldSet getUnknownFields() {
-                return this.unknownFields;
-            }
-
-            private Stats(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                initFields();
-                int mutable_bitField0_ = 0;
-                com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet.newBuilder();
-                try {
-                    boolean done = false;
-                    while (!done) {
-                        int tag = input.readTag();
-                        switch (tag) {
-                        case 0:
-                            done = true;
-                            break;
-                        default: {
-                            if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) {
-                                done = true;
-                            }
-                            break;
-                        }
-                        case 8: {
-                            bitField0_ |= 0x00000001;
-                            myShard_ = input.readInt32();
-                            break;
-                        }
-                        case 16: {
-                            bitField0_ |= 0x00000002;
-                            latestDataTime_ = input.readInt64();
-                            break;
-                        }
-                        case 24: {
-                            bitField0_ |= 0x00000004;
-                            serviceStartTime_ = input.readInt64();
-                            break;
-                        }
-                        case 32: {
-                            bitField0_ |= 0x00000008;
-                            serviceEndTime_ = input.readInt64();
-                            break;
-                        }
-                        case 40: {
-                            bitField0_ |= 0x00000010;
-                            scannedSlices_ = input.readInt32();
-                            break;
-                        }
-                        }
-                    }
-                } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-                    throw e.setUnfinishedMessage(this);
-                } catch (java.io.IOException e) {
-                    throw new com.google.protobuf.InvalidProtocolBufferException(e.getMessage()).setUnfinishedMessage(this);
-                } finally {
-                    this.unknownFields = unknownFields.build();
-                    makeExtensionsImmutable();
-                }
-            }
-
-            public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() {
-                return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIResponseInternal_Stats_descriptor;
-            }
-
-            protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() {
-                return org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.internal_static_IIResponseInternal_Stats_fieldAccessorTable.ensureFieldAccessorsInitialized(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats.class, org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats.Builder.class);
-            }
-
-            public static com.google.protobuf.Parser<Stats> PARSER = new com.google.protobuf.AbstractParser<Stats>() {
-                public Stats parsePartialFrom(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                    return new Stats(input, extensionRegistry);
-                }
-            };
-
-            @java.lang.Override
-            public com.google.protobuf.Parser<Stats> getParserForType() {
-                return PARSER;
-            }
-
-            private int bitField0_;
-            // optional int32 myShard = 1;
-            public static final int MYSHARD_FIELD_NUMBER = 1;
-            private int myShard_;
-
-            /**
-             * <code>optional int32 myShard = 1;</code>
-             */
-            public boolean hasMyShard() {
-                return ((bitField0_ & 0x00000001) == 0x00000001);
-            }
-
-            /**
-             * <code>optional int32 myShard = 1;</code>
-             */
-            public int getMyShard() {
-                return myShard_;
-            }
-
-            // optional int64 latestDataTime = 2;
-            public static final int LATESTDATATIME_FIELD_NUMBER = 2;
-            private long latestDataTime_;
-
-            /**
-             * <code>optional int64 latestDataTime = 2;</code>
-             */
-            public boolean hasLatestDataTime() {
-                return ((bitField0_ & 0x00000002) == 0x00000002);
-            }
-
-            /**
-             * <code>optional int64 latestDataTime = 2;</code>
-             */
-            public long getLatestDataTime() {
-                return latestDataTime_;
-            }
-
-            // optional int64 serviceStartTime = 3;
-            public static final int SERVICESTARTTIME_FIELD_NUMBER = 3;
-            private long serviceStartTime_;
-
-            /**
-             * <code>optional int64 serviceStartTime = 3;</code>
-             */
-            public boolean hasServiceStartTime() {
-                return ((bitField0_ & 0x00000004) == 0x00000004);
-            }
-
-            /**
-             * <code>optional int64 serviceStartTime = 3;</code>
-             */
-            public long getServiceStartTime() {
-                return serviceStartTime_;
-            }
-
-            // optional int64 serviceEndTime = 4;
-            public static final int SERVICEENDTIME_FIELD_NUMBER = 4;
-            private long serviceEndTime_;
-
-            /**
-             * <code>optional int64 serviceEndTime = 4;</code>
-             */
-            public boolean hasServiceEndTime() {
-                return ((bitField0_ & 0x00000008) == 0x00000008);
-            }
-
-            /**
-             * <code>optional int64 serviceEndTime = 4;</code>
-             */
-            public long getServiceEndTime() {
-                return serviceEndTime_;
-            }
-
-            // optional int32 scannedSlices = 5;
-            public static final int SCANNEDSLICES_FIELD_NUMBER = 5;
-            private int scannedSlices_;
-
-            /**
-             * <code>optional int32 scannedSlices = 5;</code>
-             */
-            public boolean hasScannedSlices() {
-                return ((bitField0_ & 0x00000010) == 0x00000010);
-            }
-
-            /**
-             * <code>optional int32 scannedSlices = 5;</code>
-             */
-            public int getScannedSlices() {
-                return scannedSlices_;
-            }
-
-            private void initFields() {
-                myShard_ = 0;
-                latestDataTime_ = 0L;
-                serviceStartTime_ = 0L;
-                serviceEndTime_ = 0L;
-                scannedSlices_ = 0;
-            }
-
-            private byte memoizedIsInitialized = -1;
-
-            public final boolean isInitialized() {
-                byte isInitialized = memoizedIsInitialized;
-                if (isInitialized != -1)
-                    return isInitialized == 1;
-
-                memoizedIsInitialized = 1;
-                return true;
-            }
-
-            public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException {
-                getSerializedSize();
-                if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                    output.writeInt32(1, myShard_);
-                }
-                if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                    output.writeInt64(2, latestDataTime_);
-                }
-                if (((bitField0_ & 0x00000004) == 0x00000004)) {
-                    output.writeInt64(3, serviceStartTime_);
-                }
-                if (((bitField0_ & 0x00000008) == 0x00000008)) {
-                    output.writeInt64(4, serviceEndTime_);
-                }
-                if (((bitField0_ & 0x00000010) == 0x00000010)) {
-                    output.writeInt32(5, scannedSlices_);
-                }
-                getUnknownFields().writeTo(output);
-            }
-
-            private int memoizedSerializedSize = -1;
-
-            public int getSerializedSize() {
-                int size = memoizedSerializedSize;
-                if (size != -1)
-                    return size;
-
-                size = 0;
-                if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                    size += com.google.protobuf.CodedOutputStream.computeInt32Size(1, myShard_);
-                }
-                if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                    size += com.google.protobuf.CodedOutputStream.computeInt64Size(2, latestDataTime_);
-                }
-                if (((bitField0_ & 0x00000004) == 0x00000004)) {
-                    size += com.google.protobuf.CodedOutputStream.computeInt64Size(3, serviceStartTime_);
-                }
-                if (((bitField0_ & 0x00000008) == 0x00000008)) {
-                    size += com.google.protobuf.CodedOutputStream.computeInt64Size(4, serviceEndTime_);
-                }
-                if (((bitField0_ & 0x00000010) == 0x00000010)) {
-                    size += com.google.protobuf.CodedOutputStream.computeInt32Size(5, scannedSlices_);
-                }
-                size += getUnknownFields().getSerializedSize();
-                memoizedSerializedSize = size;
-                return size;
-            }
-
-            private static final long serialVersionUID = 0L;
-
-            @java.lang.Override
-            protected java.lang.Object writeReplace() throws java.io.ObjectStreamException {
-                return super.writeReplace();
-            }
-
-            @java.lang.Override
-            public boolean equals(final java.lang.Object obj) {
-                if (obj == this) {
-                    return true;
-                }
-                if (!(obj instanceof org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats)) {
-                    return super.equals(obj);
-                }
-                org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats other = (org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats) obj;
-
-                boolean result = true;
-                result = result && (hasMyShard() == other.hasMyShard());
-                if (hasMyShard()) {
-                    result = result && (getMyShard() == other.getMyShard());
-                }
-                result = result && (hasLatestDataTime() == other.hasLatestDataTime());
-                if (hasLatestDataTime()) {
-                    result = result && (getLatestDataTime() == other.getLatestDataTime());
-                }
-                result = result && (hasServiceStartTime() == other.hasServiceStartTime());
-                if (hasServiceStartTime()) {
-                    result = result && (getServiceStartTime() == other.getServiceStartTime());
-                }
-                result = result && (hasServiceEndTime() == other.hasServiceEndTime());
-                if (hasServiceEndTime()) {
-                    result = result && (getServiceEndTime() == other.getServiceEndTime());
-                }
-                result = result && (hasScannedSlices() == other.hasScannedSlices());
-                if (hasScannedSlices()) {
-                    result = result && (getScannedSlices() == other.getScannedSlices());
-                }
-                result = result && getUnknownFields().equals(other.getUnknownFields());
-                return result;
-            }
-
-            private int memoizedHashCode = 0;
-
-            @java.lang.Override
-            public int hashCode() {
-                if (memoizedHashCode != 0) {
-                    return memoizedHashCode;
-                }
-                int hash = 41;
-                hash = (19 * hash) + getDescriptorForType().hashCode();
-                if (hasMyShard()) {
-                    hash = (37 * hash) + MYSHARD_FIELD_NUMBER;
-                    hash = (53 * hash) + getMyShard();
-                }
-                if (hasLatestDataTime()) {
-                    hash = (37 * hash) + LATESTDATATIME_FIELD_NUMBER;
-                    hash = (53 * hash) + hashLong(getLatestDataTime());
-                }
-                if (hasServiceStartTime()) {
-                    hash = (37 * hash) + SERVICESTARTTIME_FIELD_NUMBER;
-                    hash = (53 * hash) + hashLong(getServiceStartTime());
-                }
-                if (hasServiceEndTime()) {
-                    hash = (37 * hash) + SERVICEENDTIME_FIELD_NUMBER;
-                    hash = (53 * hash) + hashLong(getServiceEndTime());
-                }
-                if (hasScannedSlices()) {
-                    hash = (37 * hash) + SCANNEDSLICES_FIELD_NUMBER;
-                    hash = (53 * hash) + getScannedSlices();
-                }
-                hash = (29 * hash) + getUnknownFields().hashCode();
-                memoizedHashCode = hash;
-                return hash;
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseFrom(com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException {
-                return PARSER.parseFrom(data);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseFrom(com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                return PARSER.parseFrom(data, extensionRegistry);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException {
-                return PARSER.parseFrom(data);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException {
-                return PARSER.parseFrom(data, extensionRegistry);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseFrom(java.io.InputStream input) throws java.io.IOException {
-                return PARSER.parseFrom(input);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseFrom(java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-                return PARSER.parseFrom(input, extensionRegistry);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException {
-                return PARSER.parseDelimitedFrom(input);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseDelimitedFrom(java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-                return PARSER.parseDelimitedFrom(input, extensionRegistry);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException {
-                return PARSER.parseFrom(input);
-            }
-
-            public static org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats parseFrom(com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException {
-                return PARSER.parseFrom(input, extensionRegistry);
-            }
-
-            public static Builder newBuilder() {
-                return Builder.create();
-            }
-
-            public Builder newBuilderForType() {
-                return newBuilder();
-            }
-
-            public static Builder newBuilder(org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos.IIResponseInternal.Stats prototype) {
-                return newBuilder().mergeFrom(prototype);
-            }
-
-            public Builder toBuilder() {
-                return newBuilder(this);
-            }
-
-            @java.lang.Override
-            protected Builder newBuilderForType(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-                Builder builder = new Builder(parent);
-                return builder;
-            }
-
-            /**
-             * Protobuf type {@code IIResponseInternal.Stats}
-             *
-             * <pre>
-             *all entries in this struct be optional to conveniently add more entries in the future
-             * </pre>
-             */
-            public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder<Builder> implements org.apache.kylin.storage.hbase.ii.coprocesso

<TRUNCATED>

[33/55] [abbrv] kylin git commit: minor, add models to project instance in example data

Posted by sh...@apache.org.
minor, add models to project instance in example data


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 5268c5458f2a55abfb43edadbaa872d8e3c4b5d5
Parents: 4bd685a
Author: Hongbin Ma <ma...@apache.org>
Authored: Tue Jul 19 14:49:53 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Jul 19 21:17:52 2016 +0800

----------------------------------------------------------------------
 examples/sample_cube/metadata/project/learn_kylin.json | 13 ++++++++++---
 examples/test_case_data/localmeta/project/default.json | 13 ++++++++++---
 2 files changed, 20 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/5268c545/examples/sample_cube/metadata/project/learn_kylin.json
----------------------------------------------------------------------
diff --git a/examples/sample_cube/metadata/project/learn_kylin.json b/examples/sample_cube/metadata/project/learn_kylin.json
index a4cb1a2..fcfd505 100644
--- a/examples/sample_cube/metadata/project/learn_kylin.json
+++ b/examples/sample_cube/metadata/project/learn_kylin.json
@@ -1,12 +1,19 @@
 {
   "uuid": "1eaca32a-a33e-4b69-83dd-0bb8b1f8c91b",
- 
   "name": "learn_kylin",
   "realizations": [
     {
       "name": "kylin_sales_cube",
       "type": "CUBE",
       "realization": "kylin_sales_cube"
-    }],
-  "tables": ["DEFAULT.KYLIN_SALES", "DEFAULT.KYLIN_CAL_DT", "DEFAULT.KYLIN_CATEGORY_GROUPINGS"]
+    }
+  ],
+  "tables": [
+    "DEFAULT.KYLIN_SALES",
+    "DEFAULT.KYLIN_CAL_DT",
+    "DEFAULT.KYLIN_CATEGORY_GROUPINGS"
+  ],
+  "models": [
+    "kylin_sales_model"
+  ]
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/5268c545/examples/test_case_data/localmeta/project/default.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/project/default.json b/examples/test_case_data/localmeta/project/default.json
index 3026ff8..695f3b7 100644
--- a/examples/test_case_data/localmeta/project/default.json
+++ b/examples/test_case_data/localmeta/project/default.json
@@ -1,5 +1,4 @@
 {
- 
   "uuid": "1eaca32a-a33e-4b69-83dd-0bb8b1f8c91b",
   "name": "default",
   "realizations": [
@@ -43,8 +42,8 @@
       "type": "CUBE",
       "realization": "test_kylin_cube_with_view_inner_join_empty"
     }
-  ], 
-  "tables" : [
+  ],
+  "tables": [
     "DEFAULT.TEST_KYLIN_FACT",
     "EDW.TEST_CAL_DT",
     "EDW.V_TEST_CAL_DT",
@@ -52,5 +51,13 @@
     "EDW.TEST_SITES",
     "EDW.TEST_SELLER_TYPE_DIM",
     "DEFAULT.STREAMING_TABLE"
+  ],
+  "models": [
+    "ssb",
+    "test_kylin_inner_join_model_desc",
+    "test_kylin_inner_join_view_model_desc",
+    "test_kylin_left_join_model_desc",
+    "test_kylin_left_join_view_model_desc",
+    "test_streaming_table_model_desc"
   ]
 }
\ No newline at end of file


[35/55] [abbrv] kylin git commit: kylin 1788 enable arbitrary mandatory size

Posted by sh...@apache.org.
kylin 1788 enable arbitrary mandatory size

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


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: cdbdb551403f22bee5e5ea3d96a5592df6eaa5ea
Parents: 1c257be
Author: Cheng Wang <ch...@kyligence.io>
Authored: Wed Jul 20 18:24:53 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Jul 22 12:34:12 2016 +0800

----------------------------------------------------------------------
 .../kylin/cube/model/validation/rule/AggregationGroupRule.java    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/cdbdb551/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
index ac382e9..958ac2b 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
@@ -110,8 +110,9 @@ public class AggregationGroupRule implements IValidatorRule<CubeDesc> {
             int normalDimSize = normalDims.size();
             int hierarchySize = count(agg.getSelectRule().hierarchy_dims);
             int jointSize = count(agg.getSelectRule().joint_dims);
+            int mandatorySize = mandatoryDims.size() > 0 ? 1 : 0 ;
 
-            if (mandatoryDims.size() + normalDimSize + hierarchySize + jointSize > maxSize) {
+            if ( mandatorySize + normalDimSize + hierarchySize + jointSize > maxSize) {
                 context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " has too many dimensions");
                 continue;
             }


[19/55] [abbrv] kylin git commit: KYLIN-1879 Fix wrong time result in jobs API

Posted by sh...@apache.org.
KYLIN-1879 Fix wrong time result in jobs API


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 416f0a19ce860e0b07e8e7283bebb46f89ac9058
Parents: 8a34d3c
Author: lidongsjtu <li...@apache.org>
Authored: Tue Jul 12 18:06:17 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Tue Jul 12 18:22:20 2016 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/rest/service/JobService.java  | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/416f0a19/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
index c868264..e4fbc98 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -288,7 +288,9 @@ public class JobService extends BasicService {
         result.setType(CubeBuildTypeEnum.BUILD);
         result.setStatus(parseToJobStatus(output.getState()));
         result.setMrWaiting(AbstractExecutable.getExtraInfoAsLong(output, CubingJob.MAP_REDUCE_WAIT_TIME, 0L) / 1000);
-        result.setDuration(AbstractExecutable.getDuration(AbstractExecutable.getStartTime(output), AbstractExecutable.getEndTime(output)) / 1000);
+        result.setExecStartTime(AbstractExecutable.getStartTime(output));
+        result.setExecEndTime(AbstractExecutable.getEndTime(output));
+        result.setDuration(AbstractExecutable.getDuration(result.getExecStartTime(), result.getExecEndTime()) / 1000);
         for (int i = 0; i < cubeJob.getTasks().size(); ++i) {
             AbstractExecutable task = cubeJob.getTasks().get(i);
             result.addStep(parseToJobStep(task, i, outputs.get(task.getId())));


[22/55] [abbrv] kylin git commit: KYLIN-1896 jdbc: add columnClassName to column metadata

Posted by sh...@apache.org.
KYLIN-1896 jdbc: add columnClassName to column metadata

Signed-off-by: lidongsjtu <li...@apache.org>


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 743c9f8d32b46b4c6e5448f4c03011e94bd52975
Parents: 469aa40
Author: kangkaisen <ka...@live.com>
Authored: Fri Jul 15 17:26:10 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Fri Jul 15 18:22:42 2016 +0800

----------------------------------------------------------------------
 jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/743c9f8d/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java
index fe3712b..8890575 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java
@@ -383,9 +383,10 @@ public class KylinClient implements IRemoteClient {
         List<ColumnMetaData> metas = new ArrayList<ColumnMetaData>();
         for (int i = 0; i < queryResp.getColumnMetas().size(); i++) {
             SQLResponseStub.ColumnMetaStub scm = queryResp.getColumnMetas().get(i);
-            ScalarType type = ColumnMetaData.scalar(scm.getColumnType(), scm.getColumnTypeName(), Rep.of(convertType(scm.getColumnType())));
+            Class columnClass = convertType(scm.getColumnType());
+            ScalarType type = ColumnMetaData.scalar(scm.getColumnType(), scm.getColumnTypeName(), Rep.of(columnClass));
 
-            ColumnMetaData meta = new ColumnMetaData(i, scm.isAutoIncrement(), scm.isCaseSensitive(), scm.isSearchable(), scm.isCurrency(), scm.getIsNullable(), scm.isSigned(), scm.getDisplaySize(), scm.getLabel(), scm.getName(), scm.getSchemaName(), scm.getPrecision(), scm.getScale(), scm.getTableName(), scm.getSchemaName(), type, scm.isReadOnly(), scm.isWritable(), scm.isWritable(), null);
+            ColumnMetaData meta = new ColumnMetaData(i, scm.isAutoIncrement(), scm.isCaseSensitive(), scm.isSearchable(), scm.isCurrency(), scm.getIsNullable(), scm.isSigned(), scm.getDisplaySize(), scm.getLabel(), scm.getName(), scm.getSchemaName(), scm.getPrecision(), scm.getScale(), scm.getTableName(), scm.getSchemaName(), type, scm.isReadOnly(), scm.isWritable(), scm.isWritable(), columnClass.getCanonicalName());
 
             metas.add(meta);
         }


[29/55] [abbrv] kylin git commit: fix RAT warnings

Posted by sh...@apache.org.
fix RAT warnings


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 83785777681750673713d6f256b3d665c24a4cd0
Parents: c92f71c
Author: shaofengshi <sh...@apache.org>
Authored: Tue Jul 19 02:35:39 2016 +0000
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Jul 19 02:35:39 2016 +0000

----------------------------------------------------------------------
 .../metadata/measure/TopNMeasureTypeTest.java     | 17 +++++++++++++++++
 .../org/apache/kylin/job/SchedulerFactory.java    | 17 +++++++++++++++++
 .../kylin/rest/request/HiveTableRequest.java      | 17 +++++++++++++++++
 .../apache/kylin/tool/JobInstanceExtractor.java   | 18 ++++++++++++++++++
 4 files changed, 69 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/83785777/core-cube/src/test/java/org/apache/kylin/metadata/measure/TopNMeasureTypeTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/metadata/measure/TopNMeasureTypeTest.java b/core-cube/src/test/java/org/apache/kylin/metadata/measure/TopNMeasureTypeTest.java
index d8708d1..593eb6e 100644
--- a/core-cube/src/test/java/org/apache/kylin/metadata/measure/TopNMeasureTypeTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/metadata/measure/TopNMeasureTypeTest.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
 package org.apache.kylin.metadata.measure;
 
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/kylin/blob/83785777/core-job/src/main/java/org/apache/kylin/job/SchedulerFactory.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/SchedulerFactory.java b/core-job/src/main/java/org/apache/kylin/job/SchedulerFactory.java
index 9504681..4eb76d1 100644
--- a/core-job/src/main/java/org/apache/kylin/job/SchedulerFactory.java
+++ b/core-job/src/main/java/org/apache/kylin/job/SchedulerFactory.java
@@ -1,3 +1,20 @@
+/*
+ * 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;
 
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/kylin/blob/83785777/server-base/src/main/java/org/apache/kylin/rest/request/HiveTableRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/HiveTableRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/HiveTableRequest.java
index c529360..e42c668 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/request/HiveTableRequest.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/HiveTableRequest.java
@@ -1,3 +1,20 @@
+/*
+ * 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.rest.request;
 
 /**

http://git-wip-us.apache.org/repos/asf/kylin/blob/83785777/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java b/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
index 5ad4953..d71494d 100644
--- a/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
+++ b/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
@@ -1,3 +1,21 @@
+/*
+ * 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.tool;
 
 import java.io.File;


[41/55] [abbrv] kylin git commit: KYLIN-1797 when use beeline, must specify HIVE_CONF

Posted by sh...@apache.org.
KYLIN-1797 when use beeline, must specify HIVE_CONF

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 3e553c0e944c4f61d5df7f7ee69315a2916654b6
Parents: 9d48273
Author: shaofengshi <sh...@apache.org>
Authored: Sat Jul 23 18:51:26 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sat Jul 23 18:51:26 2016 +0800

----------------------------------------------------------------------
 build/bin/find-hive-dependency.sh | 11 +++++++++--
 1 file changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/3e553c0e/build/bin/find-hive-dependency.sh
----------------------------------------------------------------------
diff --git a/build/bin/find-hive-dependency.sh b/build/bin/find-hive-dependency.sh
index 5b748fc..067a852 100644
--- a/build/bin/find-hive-dependency.sh
+++ b/build/bin/find-hive-dependency.sh
@@ -16,12 +16,19 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
+dir=$(dirname ${0})
+source ${dir}/check-env.sh
 client_mode=`sh ${KYLIN_HOME}/bin/get-properties.sh kylin.hive.client`
 hive_env=
 
 if [ "${client_mode}" == "beeline" ]
 then
+    # when use beeline, need explicitly provide HIVE_CONF
+    if [ -z "$HIVE_CONF" ]
+    then
+        echo "Please set HIVE_CONF to the path which has hive-site.xml."
+        exit 1
+    fi
     beeline_params=`sh ${KYLIN_HOME}/bin/get-properties.sh kylin.hive.beeline.params`
     hive_env=`beeline ${beeline_params} --outputformat=dsv -e set | grep 'env:CLASSPATH'`
 else
@@ -60,7 +67,7 @@ done
 
 if [ -z "$hive_conf_path" ]
 then
-    echo "Couldn't find hive configuration directory. Please set HIVE_CONF to the path which contains hive-site.xml."
+    echo "Couldn't find hive configuration directory. Please set HIVE_CONF to the path which has hive-site.xml."
     exit 1
 fi
 


[36/55] [abbrv] kylin git commit: KYLIN-1795: fix sample.sh when using beeline

Posted by sh...@apache.org.
KYLIN-1795: fix sample.sh when using beeline

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


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 2ba0478b1ade1acbf8640c956e8f3c449b77787f
Parents: cdbdb55
Author: Yiming Liu <li...@gmail.com>
Authored: Fri Jul 22 09:36:24 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Jul 22 12:34:58 2016 +0800

----------------------------------------------------------------------
 build/bin/sample.sh                           | 8 +++++++-
 examples/sample_cube/create_sample_tables.sql | 6 +++---
 2 files changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/2ba0478b/build/bin/sample.sh
----------------------------------------------------------------------
diff --git a/build/bin/sample.sh b/build/bin/sample.sh
index 7cc50c5..9939c62 100644
--- a/build/bin/sample.sh
+++ b/build/bin/sample.sh
@@ -25,6 +25,10 @@ job_jar=`find -L ${KYLIN_HOME}/lib/ -name kylin-job*.jar`
 echo "Going to create sample tables in hive..."
 cd ${KYLIN_HOME}/sample_cube/data
 
+echo "Loading sample data into HDFS tmp path: /tmp/kylin/sample_cube/data"
+hadoop fs -mkdir -p /tmp/kylin/sample_cube/data
+hadoop fs -put * /tmp/kylin/sample_cube/data/
+
 hive_client_mode=`sh ${KYLIN_HOME}/bin/get-properties.sh kylin.hive.client`
 if [ "${hive_client_mode}" == "beeline" ]
 then
@@ -35,6 +39,8 @@ else
 fi
 
 echo "Sample hive tables are created successfully; Going to create sample cube..."
+hadoop fs -rm -r /tmp/kylin/sample_cube
+
 cd ${KYLIN_HOME}
 hbase org.apache.hadoop.util.RunJar ${job_jar} org.apache.kylin.common.persistence.ResourceTool upload ${KYLIN_HOME}/sample_cube/metadata  || { exit 1; }
-echo "Sample cube is created successfully in project 'learn_kylin'; Restart Kylin server or reload the metadata from web UI to see the change."
\ No newline at end of file
+echo "Sample cube is created successfully in project 'learn_kylin'; Restart Kylin server or reload the metadata from web UI to see the change."

http://git-wip-us.apache.org/repos/asf/kylin/blob/2ba0478b/examples/sample_cube/create_sample_tables.sql
----------------------------------------------------------------------
diff --git a/examples/sample_cube/create_sample_tables.sql b/examples/sample_cube/create_sample_tables.sql
index 943c0fa..d83566c 100644
--- a/examples/sample_cube/create_sample_tables.sql
+++ b/examples/sample_cube/create_sample_tables.sql
@@ -185,6 +185,6 @@ TRANS_ID bigint
 ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH 'DEFAULT.KYLIN_SALES.csv' OVERWRITE INTO TABLE DEFAULT.KYLIN_SALES;
-LOAD DATA LOCAL INPATH 'DEFAULT.KYLIN_CAL_DT.csv' OVERWRITE INTO TABLE DEFAULT.KYLIN_CAL_DT;
-LOAD DATA LOCAL INPATH 'DEFAULT.KYLIN_CATEGORY_GROUPINGS.csv' OVERWRITE INTO TABLE DEFAULT.KYLIN_CATEGORY_GROUPINGS;
\ No newline at end of file
+LOAD DATA INPATH '/tmp/kylin/sample_cube/data/DEFAULT.KYLIN_SALES.csv' OVERWRITE INTO TABLE DEFAULT.KYLIN_SALES;
+LOAD DATA INPATH '/tmp/kylin/sample_cube/data/DEFAULT.KYLIN_CAL_DT.csv' OVERWRITE INTO TABLE DEFAULT.KYLIN_CAL_DT;
+LOAD DATA INPATH '/tmp/kylin/sample_cube/data/DEFAULT.KYLIN_CATEGORY_GROUPINGS.csv' OVERWRITE INTO TABLE DEFAULT.KYLIN_CATEGORY_GROUPINGS;
\ No newline at end of file


[08/55] [abbrv] kylin git commit: KYLIN-1858 remove all ii related code

Posted by sh...@apache.org.
KYLIN-1858 remove all ii related code


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 2cc0b9c43f77624dfaa09b6410a818ac020fb475
Parents: cd5e603
Author: Hongbin Ma <ma...@apache.org>
Authored: Fri Jul 8 15:14:37 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Fri Jul 8 15:14:37 2016 +0800

----------------------------------------------------------------------
 assembly/pom.xml                                |    6 -
 .../kylin/job/hadoop/invertedindex/IICLI.java   |  106 -
 .../kylin/job/hadoop/invertedindex/IITest.java  |  266 --
 .../apache/kylin/common/KylinConfigBase.java    |    2 +-
 .../kylin/common/persistence/ResourceStore.java |    2 -
 .../org/apache/kylin/common/util/ClassUtil.java |    1 -
 .../common/util/AbstractKylinTestCase.java      |    2 -
 engine-mr/pom.xml                               |    5 -
 .../org/apache/kylin/engine/mr/IMROutput.java   |   26 +-
 .../kylin/engine/mr/JobBuilderSupport.java      |    9 -
 .../java/org/apache/kylin/engine/mr/MRUtil.java |    5 -
 .../engine/mr/common/AbstractHadoopJob.java     |   18 -
 .../mr/invertedindex/BatchIIJobBuilder.java     |   84 -
 .../kylin/engine/mr/invertedindex/IIJob.java    |   71 -
 .../mr/invertedindex/InvertedIndexJob.java      |  135 -
 .../mr/invertedindex/InvertedIndexMapper.java   |   77 -
 .../invertedindex/InvertedIndexPartitioner.java |   73 -
 .../mr/invertedindex/InvertedIndexReducer.java  |  128 -
 .../UpdateIIInfoAfterBuildStep.java             |   83 -
 engine-streaming/pom.xml                        |    5 -
 .../invertedindex/test_kylin_ii_inner_join.json |   16 -
 .../invertedindex/test_kylin_ii_left_join.json  |   15 -
 .../invertedindex/test_streaming_table_ii.json  |   15 -
 .../test_kylin_ii_inner_join_desc.json          |   69 -
 .../test_kylin_ii_left_join_desc.json           |   69 -
 .../test_streaming_table_ii_desc.json           |   26 -
 .../localmeta/project/default.json              |   10 -
 .../.settings/org.eclipse.core.resources.prefs  |    6 -
 .../.settings/org.eclipse.jdt.core.prefs        |  386 --
 .../.settings/org.eclipse.jdt.ui.prefs          |    7 -
 invertedindex/pom.xml                           |  103 -
 .../invertedindex/IICapabilityChecker.java      |  182 -
 .../kylin/invertedindex/IIDescManager.java      |  251 --
 .../apache/kylin/invertedindex/IIInstance.java  |  369 --
 .../apache/kylin/invertedindex/IIManager.java   |  272 --
 .../apache/kylin/invertedindex/IISegment.java   |  279 --
 .../invertedindex/index/BatchSliceMaker.java    |   95 -
 .../invertedindex/index/BitMapContainer.java    |  227 --
 .../index/ColumnValueContainer.java             |   43 -
 .../index/CompressedValueContainer.java         |  184 -
 .../index/IncrementalSliceMaker.java            |  123 -
 .../invertedindex/index/RawTableRecord.java     |  133 -
 .../index/ShardingSliceBuilder.java             |   53 -
 .../apache/kylin/invertedindex/index/Slice.java |  215 -
 .../kylin/invertedindex/index/SliceBuilder.java |   78 -
 .../kylin/invertedindex/index/TableRecord.java  |  193 -
 .../invertedindex/index/TableRecordInfo.java    |  147 -
 .../index/TableRecordInfoDigest.java            |  175 -
 .../invertedindex/measure/FixedHLLCodec.java    |   78 -
 .../measure/FixedLenMeasureCodec.java           |   50 -
 .../measure/FixedPointLongCodec.java            |  117 -
 .../kylin/invertedindex/model/IIDesc.java       |  418 --
 .../model/IIJoinedFlatTableDesc.java            |   68 -
 .../invertedindex/model/IIKeyValueCodec.java    |  235 --
 .../model/IIKeyValueCodecWithState.java         |  100 -
 .../apache/kylin/invertedindex/model/IIRow.java |   90 -
 .../invertedindex/model/KeyValueCodec.java      |   48 -
 .../invertedindex/util/IIDictionaryBuilder.java |   86 -
 .../kylin/invertedindex/IIDescManagerTest.java  |  103 -
 .../apache/kylin/invertedindex/IIDescTest.java  |   66 -
 .../kylin/invertedindex/IIInstanceTest.java     |   70 -
 .../invertedindex/InvertedIndexLocalTest.java   |  264 --
 .../org/apache/kylin/invertedindex/LZFTest.java |   49 -
 .../measure/FixedPointLongCodecTest.java        |   62 -
 kylin-it/pom.xml                                |    6 -
 .../apache/kylin/query/ITCombinationTest.java   |    1 -
 .../org/apache/kylin/query/ITIIQueryTest.java   |   92 -
 .../apache/kylin/query/ITKylinQueryTest.java    |    1 -
 .../hbase/ii/ITInvertedIndexHBaseTest.java      |  115 -
 pom.xml                                         |    1 -
 query/pom.xml                                   |    6 -
 .../apache/kylin/query/routing/Candidate.java   |    1 -
 .../apache/kylin/rest/service/BasicService.java |   10 -
 .../apache/kylin/rest/service/CacheService.java |   22 -
 .../apache/kylin/rest/service/ModelService.java |    9 -
 .../kylin/rest/service/ServiceTestBase.java     |    4 -
 storage-hbase/pom.xml                           |    6 -
 .../kylin/storage/hbase/HBaseStorage.java       |   17 +-
 .../coprocessor/CoprocessorProjector.java       |   16 +-
 .../common/coprocessor/CoprocessorRowType.java  |   12 -
 .../hbase/cube/v1/HBaseClientKVIterator.java    |   94 -
 .../kylin/storage/hbase/ii/IIBulkLoadJob.java   |   65 -
 .../storage/hbase/ii/IICreateHFileJob.java      |   88 -
 .../storage/hbase/ii/IICreateHFileMapper.java   |   71 -
 .../storage/hbase/ii/IICreateHTableJob.java     |  158 -
 .../hbase/ii/InvertedIndexStorageQuery.java     |   67 -
 .../endpoint/BitMapFilterEvaluator.java         |  249 --
 .../endpoint/ClearTextDictionary.java           |   66 -
 .../endpoint/EndpointAggregationCache.java      |   46 -
 .../endpoint/EndpointAggregators.java           |  287 --
 .../coprocessor/endpoint/EndpointEnabler.java   |   53 -
 .../endpoint/EndpointTupleConverter.java        |   94 -
 .../endpoint/EndpointTupleIterator.java         |  364 --
 .../endpoint/HbaseServerKVIterator.java         |  104 -
 .../ii/coprocessor/endpoint/IIEndpoint.java     |  336 --
 .../coprocessor/endpoint/IIResponseAdapter.java |   24 -
 .../coprocessor/endpoint/LocalDictionary.java   |   74 -
 .../endpoint/SliceBitMapProvider.java           |   55 -
 .../endpoint/generated/IIProtos.java            | 3773 ------------------
 .../ii/coprocessor/endpoint/protobuf/II.proto   |   65 -
 .../storage/hbase/steps/HBaseMROutput.java      |   18 -
 .../kylin/storage/hbase/steps/HBaseMRSteps.java |   78 -
 .../hbase/util/DeployCoprocessorCLI.java        |   35 +-
 .../hbase/util/IIDeployCoprocessorCLI.java      |  157 -
 .../storage/hbase/util/StorageCleanupJob.java   |   16 -
 .../storage/hbase/util/UpdateHTableHostCLI.java |   15 -
 .../hbase/common/TsConditionEraserTest.java     |  160 -
 .../endpoint/BitMapFilterEvaluatorTest.java     |  228 --
 .../endpoint/EndpointAggregationTest.java       |  177 -
 .../endpoint/TableRecordInfoTest.java           |   63 -
 .../endpoint/TsConditionExtractorTest.java      |  214 -
 111 files changed, 11 insertions(+), 14351 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 3904225..fc1ef63 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -56,12 +56,6 @@
             <version>${project.parent.version}</version>
         </dependency>
 
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-invertedindex</artifactId>
-            <version>${project.parent.version}</version>
-        </dependency>
-
         <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.kylin</groupId>

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IICLI.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IICLI.java b/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IICLI.java
deleted file mode 100644
index 7e7be34..0000000
--- a/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IICLI.java
+++ /dev/null
@@ -1,106 +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.job.hadoop.invertedindex;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-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;
-
-/**
- * @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/kylin/blob/2cc0b9c4/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IITest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IITest.java b/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IITest.java
deleted file mode 100644
index 3390586..0000000
--- a/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IITest.java
+++ /dev/null
@@ -1,266 +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.job.hadoop.invertedindex;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Queue;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-import org.apache.commons.lang.NotImplementedException;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.kylin.common.util.FIFOIterable;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.util.StreamingBatch;
-import org.apache.kylin.common.util.StreamingMessage;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.invertedindex.index.SliceBuilder;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodecWithState;
-import org.apache.kylin.invertedindex.model.IIRow;
-import org.apache.kylin.invertedindex.model.KeyValueCodec;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.kafka.StreamingParser;
-import org.apache.kylin.source.kafka.StringStreamingParser;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorFilter;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorProjector;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorRowType;
-import org.apache.kylin.storage.hbase.common.coprocessor.FilterDecorator;
-import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.ClearTextDictionary;
-import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.EndpointAggregators;
-import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.IIEndpoint;
-import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
-import kafka.message.Message;
-import kafka.message.MessageAndOffset;
-
-/**
- */
-public class IITest extends LocalFileMetadataTestCase {
-
-    String iiName = "test_kylin_ii_inner_join";
-    IIInstance ii;
-    IIDesc iiDesc;
-
-    List<IIRow> iiRows;
-
-    final String[] inputData = new String[] { //
-            "FP-non GTC,0,15,145970,0,28,Toys,2008-10-08 07:18:40,USER_Y,Toys & Hobbies,Models & Kits,Automotive,0,Ebay,USER_S,15,Professional-Other,2012-08-16,2012-08-11,0,2012-08-16,145970,10000329,26.8551,0", //
-            "ABIN,0,-99,43479,0,21,Photo,2012-09-11 20:26:04,USER_Y,Cameras & Photo,Film Photography,Other,0,Ebay,USER_S,-99,Not Applicable,2012-08-16,2012-08-11,0,2012-08-16,43479,10000807,26.2474,0", //
-            "ABIN,0,16,80053,0,12,Computers,2012-06-19 21:15:09,USER_Y,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,0,Ebay,USER_S,16,Consumer-Other,2012-08-16,2012-08-11,0,2012-08-16,80053,10000261,94.2273,0" };
-
-    @Before
-    public void setUp() throws Exception {
-        this.createTestMetadata();
-        this.ii = IIManager.getInstance(getTestConfig()).getII(iiName);
-        this.iiDesc = ii.getDescriptor();
-
-        List<MessageAndOffset> messages = Lists.transform(Arrays.asList(inputData), new Function<String, MessageAndOffset>() {
-            @Nullable
-            @Override
-            public MessageAndOffset apply(String input) {
-                return new MessageAndOffset(new Message(input.getBytes()), System.currentTimeMillis());
-            }
-        });
-
-        final StreamingParser parser = StringStreamingParser.instance;
-        final List<StreamingMessage> streamingMessages = Lists.transform(messages, new Function<MessageAndOffset, StreamingMessage>() {
-            @Nullable
-            @Override
-            public StreamingMessage apply(@Nullable MessageAndOffset input) {
-                return parser.parse(input);
-            }
-        });
-        StreamingBatch batch = new StreamingBatch(streamingMessages, Pair.newPair(0L, System.currentTimeMillis()));
-
-        iiRows = Lists.newArrayList();
-        final Slice slice = new SliceBuilder(iiDesc, (short) 0).buildSlice((batch));
-        IIKeyValueCodec codec = new IIKeyValueCodec(slice.getInfo());
-        for (IIRow iiRow : codec.encodeKeyValue(slice)) {
-            iiRows.add(iiRow);
-        }
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-    }
-
-    /**
-     * simulate stream building into slices, and encode the slice into IIRows.
-     * Then reconstruct the IIRows to slice.
-     */
-    @Test
-    public void basicTest() {
-        Queue<IIRow> buffer = Lists.newLinkedList();
-        FIFOIterable bufferIterable = new FIFOIterable(buffer);
-        TableRecordInfo info = new TableRecordInfo(iiDesc);
-        TableRecordInfoDigest digest = info.getDigest();
-        KeyValueCodec codec = new IIKeyValueCodecWithState(digest);
-        Iterator<Slice> slices = codec.decodeKeyValue(bufferIterable).iterator();
-
-        Assert.assertTrue(!slices.hasNext());
-        Assert.assertEquals(iiRows.size(), digest.getColumnCount());
-
-        for (int i = 0; i < digest.getColumnCount(); ++i) {
-            buffer.add(iiRows.get(i));
-
-            if (i != digest.getColumnCount() - 1) {
-                Assert.assertTrue(!slices.hasNext());
-            } else {
-                Assert.assertTrue(slices.hasNext());
-            }
-        }
-
-        Slice newSlice = slices.next();
-        Assert.assertEquals(newSlice.getLocalDictionaries()[0].getSize(), 2);
-    }
-
-    @Test
-    public void IIEndpointTest() {
-        TableRecordInfo info = new TableRecordInfo(ii.getDescriptor());
-        if (ii.getFirstSegment() == null) {
-            IISegment segment = IIManager.getInstance(getTestConfig()).buildSegment(ii, 0, System.currentTimeMillis());
-            ii.getSegments().add(segment);
-        }
-        CoprocessorRowType type = CoprocessorRowType.fromTableRecordInfo(info, ii.getFirstSegment().getColumns());
-        CoprocessorProjector projector = CoprocessorProjector.makeForEndpoint(info, Collections.singletonList(ii.getDescriptor().findColumnRef("default.test_kylin_fact", "lstg_format_name")));
-
-        FunctionDesc f1 = new FunctionDesc();
-        f1.setExpression("SUM");
-        ParameterDesc p1 = new ParameterDesc();
-        p1.setType("column");
-        p1.setValue("PRICE");
-        f1.setParameter(p1);
-        f1.setReturnType("decimal(19,4)");
-
-        TblColRef column = ii.getDescriptor().findColumnRef("default.test_kylin_fact", "cal_dt");
-        CompareTupleFilter compareFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GTE);
-        ColumnTupleFilter columnFilter = new ColumnTupleFilter(column);
-        compareFilter.addChild(columnFilter);
-        ConstantTupleFilter constantFilter = null;
-        constantFilter = new ConstantTupleFilter(("2012-08-16"));
-        compareFilter.addChild(constantFilter);
-
-        EndpointAggregators aggregators = EndpointAggregators.fromFunctions(info, Collections.singletonList(f1));
-        CoprocessorFilter filter = CoprocessorFilter.fromFilter(new ClearTextDictionary(info), compareFilter, FilterDecorator.FilterConstantsTreatment.AS_IT_IS);
-
-        final Iterator<IIRow> iiRowIterator = iiRows.iterator();
-
-        IIEndpoint endpoint = new IIEndpoint();
-        IIProtos.IIResponseInternal response = endpoint.getResponse(new RegionScanner() {
-            @Override
-            public HRegionInfo getRegionInfo() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public boolean isFilterDone() throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public boolean reseek(byte[] row) throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public long getMaxResultSize() {
-                throw new NotImplementedException();
-
-            }
-
-            @Override
-            public long getMvccReadPoint() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public boolean nextRaw(List<Cell> result) throws IOException {
-                if (iiRowIterator.hasNext()) {
-                    IIRow iiRow = iiRowIterator.next();
-                    result.addAll(iiRow.makeCells());
-                    return true;
-                } else {
-                    return false;
-                }
-            }
-
-            @Override
-            public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public boolean next(List<Cell> results) throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public boolean next(List<Cell> result, int limit) throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public void close() throws IOException {
-                throw new NotImplementedException();
-            }
-        }, type, projector, aggregators, filter);
-
-        Assert.assertEquals(2, response.getRowsList().size());
-        System.out.println(response.getRowsList().size());
-        Set<String> answers = Sets.newHashSet("120.4747", "26.8551");
-        for (IIProtos.IIResponseInternal.IIRow responseRow : response.getRowsList()) {
-            ByteBuffer bf = responseRow.getMeasures().asReadOnlyByteBuffer();
-            List<Object> metrics = aggregators.deserializeMetricValues(bf);
-            Assert.assertTrue(answers.contains(metrics.get(0)));
-            answers.remove(metrics.get(0));
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 84c4db9..bfbaaa4 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -203,7 +203,7 @@ abstract public class KylinConfigBase implements Serializable {
 
     public String[] getRealizationProviders() {
         return getOptionalStringArray("kylin.realization.providers", //
-                new String[] { "org.apache.kylin.cube.CubeManager", "org.apache.kylin.storage.hybrid.HybridManager", "org.apache.kylin.invertedindex.IIManager" });
+                new String[] { "org.apache.kylin.cube.CubeManager", "org.apache.kylin.storage.hybrid.HybridManager" });
     }
 
     public CliCommandExecutor getCliCommandExecutor() throws IOException {

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/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 cf00c05..99a9695 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
@@ -42,9 +42,7 @@ abstract public class ResourceStore {
     private static final Logger logger = LoggerFactory.getLogger(ResourceStore.class);
 
     public static final String CUBE_RESOURCE_ROOT = "/cube";
-    public static final String II_RESOURCE_ROOT = "/invertedindex";
     public static final String CUBE_DESC_RESOURCE_ROOT = "/cube_desc";
-    public static final String II_DESC_RESOURCE_ROOT = "/invertedindex_desc";
     public static final String DATA_MODEL_DESC_RESOURCE_ROOT = "/model_desc";
     public static final String DICT_RESOURCE_ROOT = "/dict";
     public static final String PROJECT_RESOURCE_ROOT = "/project";

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java
index ef89754..3c63c41 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java
@@ -54,7 +54,6 @@ public class ClassUtil {
         classRenameMap.put("org.apache.kylin.job.common.HadoopShellExecutable", "org.apache.kylin.engine.mr.common.HadoopShellExecutable");
         classRenameMap.put("org.apache.kylin.job.common.MapReduceExecutable", "org.apache.kylin.engine.mr.common.MapReduceExecutable");
         classRenameMap.put("org.apache.kylin.job.cube.CubingJob", "org.apache.kylin.engine.mr.CubingJob");
-        classRenameMap.put("org.apache.kylin.job.invertedindex.IIJob", "org.apache.kylin.engine.mr.invertedindex.IIJob");
         classRenameMap.put("org.apache.kylin.job.cube.GarbageCollectionStep", "org.apache.kylin.storage.hbase.steps.DeprecatedGCStep");
         classRenameMap.put("org.apache.kylin.job.cube.MergeDictionaryStep", "org.apache.kylin.engine.mr.steps.MergeDictionaryStep");
         classRenameMap.put("org.apache.kylin.job.cube.UpdateCubeInfoAfterBuildStep", "org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterBuildStep");

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java b/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
index 684d828..556019f 100644
--- a/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
+++ b/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
@@ -31,8 +31,6 @@ public abstract class AbstractKylinTestCase {
     public static final String[] SERVICES_WITH_CACHE = { //
             "org.apache.kylin.cube.CubeManager", //
             "org.apache.kylin.cube.CubeDescManager", //
-            "org.apache.kylin.invertedindex.IIDescManager", //
-            "org.apache.kylin.invertedindex.IIManager", //
             "org.apache.kylin.storage.hybrid.HybridManager", //
             "org.apache.kylin.metadata.realization.RealizationRegistry", //
             "org.apache.kylin.metadata.project.ProjectManager", //

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/pom.xml
----------------------------------------------------------------------
diff --git a/engine-mr/pom.xml b/engine-mr/pom.xml
index 363e459..12a6b73 100644
--- a/engine-mr/pom.xml
+++ b/engine-mr/pom.xml
@@ -50,11 +50,6 @@
         </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-core-job</artifactId>
             <version>${project.parent.version}</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput.java
index 276af65..beebb4e 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.engine.mr;
 
 import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 
 public interface IMROutput {
@@ -27,11 +26,8 @@ public interface IMROutput {
     /** Return a helper to participate in batch cubing job flow. */
     public IMRBatchCubingOutputSide getBatchCubingOutputSide(CubeSegment seg);
 
-    /** Return a helper to participate in batch cubing job flow. */
-    public IMRBatchInvertedIndexingOutputSide getBatchInvertedIndexingOutputSide(IISegment seg);
-
     /**
-     * Participate the batch cubing flow as the output side. Responsible for saving
+     * Participate the batch cubing flow as the output side. Responsible for savg
      * the cuboid output to storage (Phase 3).
      * 
      * - Phase 1: Create Flat Table
@@ -80,24 +76,4 @@ public interface IMROutput {
         public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow);
     }
 
-    /**
-     * Participate the batch inverted indexing flow as the output side. Responsible for saving
-     * the output to storage (Phase 3).
-     *
-     * - Phase 1: Create Flat Table
-     * - Phase 2: Build Dictionary
-     * - Phase 3: Build II
-     * - Phase 4: Update Metadata & Cleanup
-     */
-    public interface IMRBatchInvertedIndexingOutputSide {
-
-        /**
-         * Add step that saves II output from HDFS to storage.
-         *
-         */
-        public void addStepPhase3_BuildII(DefaultChainedExecutable jobFlow, String rootPath);
-
-        /** Add step that does any necessary clean up. */
-        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow);
-    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
index 5228088..5a82fe5 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
@@ -25,7 +25,6 @@ import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.common.BatchConstants;
 import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
 import org.apache.kylin.engine.mr.common.MapReduceExecutable;
-import org.apache.kylin.engine.mr.invertedindex.UpdateIIInfoAfterBuildStep;
 import org.apache.kylin.engine.mr.steps.CreateDictionaryJob;
 import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
 import org.apache.kylin.engine.mr.steps.FactDistinctColumnsJob;
@@ -130,14 +129,6 @@ public class JobBuilderSupport {
         return result;
     }
 
-    public UpdateIIInfoAfterBuildStep createUpdateIIInfoAfterBuildStep(String jobId) {
-        final UpdateIIInfoAfterBuildStep updateIIInfoStep = new UpdateIIInfoAfterBuildStep();
-        updateIIInfoStep.setName(ExecutableConstants.STEP_NAME_UPDATE_II_INFO);
-        updateIIInfoStep.setInvertedIndexName(seg.getRealization().getName());
-        updateIIInfoStep.setJobId(jobId);
-        return updateIIInfoStep;
-    }
-
     // ============================================================================
 
     public String getJobWorkingDir(String jobId) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
index b67dc38..abe05d7 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
@@ -29,7 +29,6 @@ import org.apache.kylin.engine.mr.IMROutput.IMRBatchCubingOutputSide;
 import org.apache.kylin.engine.mr.IMROutput.IMRBatchMergeOutputSide;
 import org.apache.kylin.engine.mr.IMROutput2.IMRBatchCubingOutputSide2;
 import org.apache.kylin.engine.mr.IMROutput2.IMRBatchMergeOutputSide2;
-import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.realization.IRealizationSegment;
@@ -70,10 +69,6 @@ public class MRUtil {
         return StorageFactory.createEngineAdapter(seg, IMROutput2.class).getBatchMergeOutputSide(seg);
     }
 
-    public static IMROutput.IMRBatchInvertedIndexingOutputSide getBatchInvertedIndexingOutputSide(IISegment seg) {
-        return StorageFactory.createEngineAdapter(seg, IMROutput.class).getBatchInvertedIndexingOutputSide(seg);
-    }
-
     // use this method instead of ToolRunner.run() because ToolRunner.run() is not thread-sale
     // Refer to: http://stackoverflow.com/questions/22462665/is-hadoops-toorunner-thread-safe
     public static int runMRJob(Tool tool, String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index 9b5ed67..bafe65b 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -63,7 +63,6 @@ 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.job.JobInstance;
 import org.apache.kylin.job.exception.JobException;
 import org.apache.kylin.metadata.MetadataManager;
@@ -441,23 +440,6 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
         attachKylinPropsAndMetadata(dumpList, cube.getConfig(), conf);
     }
 
-    protected void attachKylinPropsAndMetadata(IIInstance ii, Configuration conf) throws IOException {
-        MetadataManager metaMgr = MetadataManager.getInstance(ii.getConfig());
-
-        // write II / model_desc / II_desc / dict / table
-        ArrayList<String> dumpList = new ArrayList<String>();
-        dumpList.add(ii.getResourcePath());
-        dumpList.add(ii.getDescriptor().getModel().getResourcePath());
-        dumpList.add(ii.getDescriptor().getResourcePath());
-
-        for (String tableName : ii.getDescriptor().getModel().getAllTables()) {
-            TableDesc table = metaMgr.getTableDesc(tableName);
-            dumpList.add(table.getResourcePath());
-        }
-
-        attachKylinPropsAndMetadata(dumpList, ii.getConfig(), conf);
-    }
-
     protected void attachKylinPropsAndMetadata(ArrayList<String> dumpList, KylinConfig kylinConfig, Configuration conf) throws IOException {
         File tmp = File.createTempFile("kylin_job_meta", "");
         FileUtils.forceDelete(tmp); // we need a directory, so delete the file first

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/BatchIIJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/BatchIIJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/BatchIIJobBuilder.java
deleted file mode 100644
index e582fa0..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/BatchIIJobBuilder.java
+++ /dev/null
@@ -1,84 +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.engine.mr.invertedindex;
-
-import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
-import org.apache.kylin.engine.mr.IMROutput;
-import org.apache.kylin.engine.mr.JobBuilderSupport;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.engine.mr.common.MapReduceExecutable;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class BatchIIJobBuilder extends JobBuilderSupport {
-
-    private static final Logger logger = LoggerFactory.getLogger(BatchIIJobBuilder.class);
-
-    private final IMRBatchCubingInputSide inputSide;
-    private final IMROutput.IMRBatchInvertedIndexingOutputSide outputSide;
-
-    public BatchIIJobBuilder(IISegment newSegment, String submitter) {
-        super(newSegment, submitter);
-        this.inputSide = MRUtil.getBatchCubingInputSide(newSegment);
-        this.outputSide = MRUtil.getBatchInvertedIndexingOutputSide(newSegment);
-    }
-
-    public IIJob build() {
-        logger.info("MR new job to BUILD segment " + seg);
-
-        final IIJob result = IIJob.createBuildJob((IISegment) seg, submitter, config);
-        final String jobId = result.getId();
-
-        final String iiRootPath = getRealizationRootPath(jobId) + "/";
-        // Phase 1: Create Flat Table
-        inputSide.addStepPhase1_CreateFlatTable(result);
-
-        // Phase 2: Build Inverted Index
-        result.addTask(createInvertedIndexStep((IISegment) seg, iiRootPath));
-        outputSide.addStepPhase3_BuildII(result, iiRootPath);
-
-        // Phase 3: Update Metadata & Cleanup
-        result.addTask(createUpdateIIInfoAfterBuildStep(jobId));
-        inputSide.addStepPhase4_Cleanup(result);
-        outputSide.addStepPhase4_Cleanup(result);
-
-        return result;
-    }
-
-    private MapReduceExecutable createInvertedIndexStep(IISegment seg, String iiOutputTempPath) {
-        MapReduceExecutable buildIIStep = new MapReduceExecutable();
-
-        StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd);
-
-        buildIIStep.setName(ExecutableConstants.STEP_NAME_BUILD_II);
-
-        appendExecCmdParameters(cmd, BatchConstants.ARG_II_NAME, seg.getRealization().getName());
-        appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, iiOutputTempPath);
-        appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, ExecutableConstants.STEP_NAME_BUILD_II);
-
-        buildIIStep.setMapReduceParams(cmd.toString());
-        buildIIStep.setMapReduceJobClass(InvertedIndexJob.class);
-        return buildIIStep;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJob.java
deleted file mode 100644
index 7c0748a..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJob.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine.mr.invertedindex;
-
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.TimeZone;
-
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-
-/**
- */
-public class IIJob extends DefaultChainedExecutable {
-
-    public IIJob() {
-        super();
-    }
-
-    private static final String II_INSTANCE_NAME = "iiName";
-    private static final String SEGMENT_ID = "segmentId";
-
-    void setIIName(String name) {
-        setParam(II_INSTANCE_NAME, name);
-    }
-
-    public String getIIName() {
-        return getParam(II_INSTANCE_NAME);
-    }
-
-    void setSegmentId(String segmentId) {
-        setParam(SEGMENT_ID, segmentId);
-    }
-
-    public String getSegmentId() {
-        return getParam(SEGMENT_ID);
-    }
-
-    public static IIJob createBuildJob(IISegment seg, String submitter, JobEngineConfig config) {
-        return initialJob(seg, "BUILD", submitter, config);
-    }
-
-    private static IIJob initialJob(IISegment seg, String type, String submitter, JobEngineConfig config) {
-        IIJob result = new IIJob();
-        SimpleDateFormat format = new SimpleDateFormat("z yyyy-MM-dd HH:mm:ss");
-        format.setTimeZone(TimeZone.getTimeZone(config.getTimeZone()));
-        result.setIIName(seg.getIIInstance().getName());
-        result.setSegmentId(seg.getUuid());
-        result.setName(seg.getIIInstance().getName() + " - " + seg.getName() + " - " + type + " - " + format.format(new Date(System.currentTimeMillis())));
-        result.setSubmitter(submitter);
-        return result;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexJob.java
deleted file mode 100644
index f7adf6b..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexJob.java
+++ /dev/null
@@ -1,135 +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.engine.mr.invertedindex;
-
-import java.io.IOException;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.engine.mr.IMRInput;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-public class InvertedIndexJob extends AbstractHadoopJob {
-    protected static final Logger logger = LoggerFactory.getLogger(InvertedIndexJob.class);
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_II_NAME);
-            options.addOption(OPTION_OUTPUT_PATH);
-            parseOptions(options, args);
-
-            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
-            String iiname = getOptionValue(OPTION_II_NAME);
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-
-            // ----------------------------------------------------------------------------
-
-            logger.info("Starting: " + job.getJobName());
-
-            IIInstance ii = getII(iiname);
-            short sharding = ii.getDescriptor().getSharding();
-
-            setJobClasspath(job, ii.getConfig());
-
-            setupMapper(ii.getFirstSegment());
-            setupReducer(output, sharding);
-            attachMetadata(ii);
-
-            return waitForCompletion(job);
-
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        } finally {
-            if (job != null)
-                cleanupTempConfFile(job.getConfiguration());
-        }
-
-    }
-
-    private IIInstance getII(String iiName) {
-        IIManager mgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
-        IIInstance ii = mgr.getII(iiName);
-        if (ii == null)
-            throw new IllegalArgumentException("No Inverted Index found by name " + iiName);
-        return ii;
-    }
-
-    private void attachMetadata(IIInstance ii) throws IOException {
-
-        Configuration conf = job.getConfiguration();
-        attachKylinPropsAndMetadata(ii, conf);
-
-        IISegment seg = ii.getFirstSegment();
-        conf.set(BatchConstants.CFG_II_NAME, ii.getName());
-        conf.set(BatchConstants.CFG_II_SEGMENT_NAME, seg.getName());
-    }
-
-    private void setupMapper(IISegment segment) throws IOException {
-
-        IMRInput.IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(segment).getFlatTableInputFormat();
-        flatTableInputFormat.configureJob(job);
-
-        job.setMapperClass(InvertedIndexMapper.class);
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setPartitionerClass(InvertedIndexPartitioner.class);
-    }
-
-    private void setupReducer(Path output, short sharding) throws IOException {
-        job.setReducerClass(InvertedIndexReducer.class);
-        job.setOutputFormatClass(SequenceFileOutputFormat.class);
-        job.setOutputKeyClass(ImmutableBytesWritable.class);
-        job.setOutputValueClass(ImmutableBytesWritable.class);
-
-        job.setNumReduceTasks(sharding);
-
-        FileOutputFormat.setOutputPath(job, output);
-
-        job.getConfiguration().set(BatchConstants.CFG_OUTPUT_PATH, output.toString());
-
-        deletePath(job.getConfiguration(), output);
-    }
-
-    public static void main(String[] args) throws Exception {
-        InvertedIndexJob job = new InvertedIndexJob();
-        int exitCode = ToolRunner.run(job, args);
-        System.exit(exitCode);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexMapper.java
deleted file mode 100644
index 27e2470..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexMapper.java
+++ /dev/null
@@ -1,77 +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.engine.mr.invertedindex;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.engine.mr.IMRInput;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-
-/**
- * @author yangli9
- */
-public class InvertedIndexMapper<KEYIN> extends KylinMapper<KEYIN, Object, LongWritable, Writable> {
-
-    private TableRecordInfo info;
-
-    private LongWritable outputKey;
-    private IMRInput.IMRTableInputFormat flatTableInputFormat;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        Configuration conf = context.getConfiguration();
-
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-        IIManager mgr = IIManager.getInstance(config);
-        IIInstance ii = mgr.getII(conf.get(BatchConstants.CFG_II_NAME));
-        IISegment seg = ii.getSegment(conf.get(BatchConstants.CFG_II_SEGMENT_NAME), SegmentStatusEnum.NEW);
-        this.info = new TableRecordInfo(seg);
-
-        outputKey = new LongWritable();
-
-        flatTableInputFormat = MRUtil.getBatchCubingInputSide(ii.getFirstSegment()).getFlatTableInputFormat();
-    }
-
-    @Override
-    public void map(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
-
-        Writable writableRecord = (Writable) record;
-        String[] row = flatTableInputFormat.parseMapperInput(writableRecord);
-        String timestampString = row[info.getTimestampColumn()];
-
-        outputKey.set(DateFormat.stringToMillis(timestampString));
-        //
-        context.write(outputKey, writableRecord);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexPartitioner.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexPartitioner.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexPartitioner.java
deleted file mode 100644
index dcf707f..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexPartitioner.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.engine.mr.invertedindex;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Partitioner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.TableRecord;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-
-/**
- * @author yangli9
- */
-public class InvertedIndexPartitioner extends Partitioner<LongWritable, ImmutableBytesWritable> implements Configurable {
-
-    private Configuration conf;
-    private TableRecordInfo info;
-    private TableRecord rec;
-
-    @Override
-    public int getPartition(LongWritable key, ImmutableBytesWritable value, int numPartitions) {
-        rec.setBytes(value.get(), value.getOffset(), value.getLength());
-        return rec.getShard();
-    }
-
-    @Override
-    public void setConf(Configuration conf) {
-        this.conf = conf;
-        try {
-            KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-            IIManager mgr = IIManager.getInstance(config);
-            IIInstance ii = mgr.getII(conf.get(BatchConstants.CFG_II_NAME));
-            IISegment seg = ii.getSegment(conf.get(BatchConstants.CFG_II_SEGMENT_NAME), SegmentStatusEnum.NEW);
-            this.info = new TableRecordInfo(seg);
-            this.rec = this.info.createTableRecord();
-        } catch (IOException e) {
-            throw new RuntimeException("", e);
-        }
-    }
-
-    @Override
-    public Configuration getConf() {
-        return conf;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexReducer.java
deleted file mode 100644
index e7e760a..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexReducer.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.
-*/
-
-package org.apache.kylin.engine.mr.invertedindex;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-
-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.util.Pair;
-import org.apache.kylin.common.util.StreamingBatch;
-import org.apache.kylin.common.util.StreamingMessage;
-import org.apache.kylin.engine.mr.IMRInput;
-import org.apache.kylin.engine.mr.KylinReducer;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.invertedindex.index.SliceBuilder;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.apache.kylin.invertedindex.model.IIRow;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-public class InvertedIndexReducer extends KylinReducer<LongWritable, Object, ImmutableBytesWritable, ImmutableBytesWritable> {
-
-    private TableRecordInfo info;
-    private IIKeyValueCodec kv;
-    private IMRInput.IMRTableInputFormat flatTableInputFormat;
-    private SliceBuilder sliceBuilder;
-    private ArrayList<StreamingMessage> messages;
-    private int sliceSize;
-    private ImmutableBytesWritable immutableBytesWritable;
-    private ByteBuffer valueBuf;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        Configuration conf = context.getConfiguration();
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-        IIManager mgr = IIManager.getInstance(config);
-        IIInstance ii = mgr.getII(conf.get(BatchConstants.CFG_II_NAME));
-        IISegment seg = ii.getFirstSegment();
-        info = new TableRecordInfo(seg);
-        kv = new IIKeyValueCodec(info.getDigest());
-        flatTableInputFormat = MRUtil.getBatchCubingInputSide(ii.getFirstSegment()).getFlatTableInputFormat();
-        sliceSize = ii.getDescriptor().getSliceSize();
-        short shard = (short) context.getTaskAttemptID().getTaskID().getId();
-        System.out.println("Generating to shard - " + shard);
-        sliceBuilder = new SliceBuilder(seg.getIIDesc(), shard);
-        messages = Lists.newArrayListWithCapacity(sliceSize);
-        immutableBytesWritable = new ImmutableBytesWritable();
-        valueBuf = ByteBuffer.allocate(1024 * 1024); // 1MB
-    }
-
-    @Override
-    public void reduce(LongWritable key, Iterable<Object> values, Context context) //
-            throws IOException, InterruptedException {
-        for (Object v : values) {
-            String[] row = flatTableInputFormat.parseMapperInput(v);
-            messages.add((parse(row)));
-            if (messages.size() >= sliceSize) {
-                buildAndOutput(new StreamingBatch(messages, Pair.newPair(System.currentTimeMillis(), System.currentTimeMillis())), context);
-                messages = Lists.newArrayList();
-            }
-        }
-    }
-
-    private StreamingMessage parse(String[] row) {
-        return new StreamingMessage(Lists.newArrayList(row), System.currentTimeMillis(), System.currentTimeMillis(), Collections.<String, Object> emptyMap());
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-        if (!messages.isEmpty()) {
-            buildAndOutput(new StreamingBatch(messages, Pair.newPair(System.currentTimeMillis(), System.currentTimeMillis())), context);
-            messages.clear();
-        }
-
-    }
-
-    private void buildAndOutput(StreamingBatch streamingBatch, Context context) throws IOException, InterruptedException {
-        final Slice slice = sliceBuilder.buildSlice(streamingBatch);
-        ImmutableBytesWritable value, dictionary;
-        for (IIRow pair : kv.encodeKeyValue(slice)) {
-            value = pair.getValue();
-            dictionary = pair.getDictionary();
-            int newLength = 4 + value.getLength() + dictionary.getLength();
-            if (newLength > valueBuf.limit()) {
-                valueBuf = ByteBuffer.allocate(newLength);
-            }
-            valueBuf.clear();
-            valueBuf.putInt(value.getLength());
-            valueBuf.put(value.get(), value.getOffset(), value.getLength());
-            valueBuf.put(dictionary.get(), dictionary.getOffset(), dictionary.getLength());
-            immutableBytesWritable.set(valueBuf.array(), 0, newLength);
-            context.write(pair.getKey(), immutableBytesWritable);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/UpdateIIInfoAfterBuildStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/UpdateIIInfoAfterBuildStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/UpdateIIInfoAfterBuildStep.java
deleted file mode 100644
index a509c1f..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/UpdateIIInfoAfterBuildStep.java
+++ /dev/null
@@ -1,83 +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.engine.mr.invertedindex;
-
-import java.io.IOException;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-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.metadata.model.SegmentStatusEnum;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-public class UpdateIIInfoAfterBuildStep extends AbstractExecutable {
-    private static final Logger logger = LoggerFactory.getLogger(UpdateIIInfoAfterBuildStep.class);
-
-    private static final String II_NAME = "iiName";
-    private static final String JOB_ID = "jobId";
-
-    public UpdateIIInfoAfterBuildStep() {
-        super();
-    }
-
-    public void setInvertedIndexName(String cubeName) {
-        this.setParam(II_NAME, cubeName);
-    }
-
-    private String getInvertedIndexName() {
-        return getParam(II_NAME);
-    }
-
-    public void setJobId(String id) {
-        setParam(JOB_ID, id);
-    }
-
-    private String getJobId() {
-        return getParam(JOB_ID);
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-
-        IIManager mgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
-        IIInstance ii = mgr.getII(getInvertedIndexName());
-        IISegment segment = ii.getFirstSegment();
-        segment.setStatus(SegmentStatusEnum.READY);
-
-        segment.setLastBuildJobID(getJobId());
-        segment.setLastBuildTime(System.currentTimeMillis());
-
-        try {
-            mgr.updateII(ii);
-            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
-        } catch (IOException e) {
-            logger.error("fail to update inverted index after build", e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/engine-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/engine-streaming/pom.xml b/engine-streaming/pom.xml
index 54955a6..7110377 100644
--- a/engine-streaming/pom.xml
+++ b/engine-streaming/pom.xml
@@ -43,11 +43,6 @@
         </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-core-storage</artifactId>
             <version>${project.parent.version}</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/examples/test_case_data/localmeta/invertedindex/test_kylin_ii_inner_join.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/invertedindex/test_kylin_ii_inner_join.json b/examples/test_case_data/localmeta/invertedindex/test_kylin_ii_inner_join.json
deleted file mode 100644
index 13fc282..0000000
--- a/examples/test_case_data/localmeta/invertedindex/test_kylin_ii_inner_join.json
+++ /dev/null
@@ -1,16 +0,0 @@
-{
- 
-  "uuid": "dk853e80-41be-49a5-89ca-9fb7294db186",
-  "name": "test_kylin_ii_inner_join",
-  "owner": null,
-  "version": null,
-  "cost": 10,
-  "status": "DISABLED",
-  "segments": [],
-  "last_modified": 1420016227424,
-  "descriptor": "test_kylin_ii_inner_join_desc",
-  "create_time": null,
-  "input_records_size": 0,
-  "size_kb": 0,
-  "source_records_count": 0
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/examples/test_case_data/localmeta/invertedindex/test_kylin_ii_left_join.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/invertedindex/test_kylin_ii_left_join.json b/examples/test_case_data/localmeta/invertedindex/test_kylin_ii_left_join.json
deleted file mode 100644
index 28b23ef..0000000
--- a/examples/test_case_data/localmeta/invertedindex/test_kylin_ii_left_join.json
+++ /dev/null
@@ -1,15 +0,0 @@
-{
- 
-  "uuid": "daa53e80-41be-49a5-89ca-9fb7294db186",
-  "name": "test_kylin_ii_left_join",
-  "owner": null,
-  "cost": 10,
-  "status": "DISABLED",
-  "segments": [],
-  "last_modified": 1420016227424,
-  "descriptor": "test_kylin_ii_left_join_desc",
-  "create_time": null,
-  "input_records_size": 0,
-  "size_kb": 0,
-  "source_records_count": 0
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/examples/test_case_data/localmeta/invertedindex/test_streaming_table_ii.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/invertedindex/test_streaming_table_ii.json b/examples/test_case_data/localmeta/invertedindex/test_streaming_table_ii.json
deleted file mode 100644
index 4ec2711..0000000
--- a/examples/test_case_data/localmeta/invertedindex/test_streaming_table_ii.json
+++ /dev/null
@@ -1,15 +0,0 @@
-{
- 
-  "uuid": "daa53e80-41be-49a5-89ca-9fb729999812",
-  "name": "test_streaming_table_ii",
-  "owner": null,
-  "cost": 10,
-  "status": "DISABLED",
-  "segments": [],
-  "last_modified": 0,
-  "descriptor": "test_streaming_table_ii_desc",
-  "create_time": null,
-  "input_records_size": 0,
-  "size_kb": 0,
-  "source_records_count": 0
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_ii_inner_join_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_ii_inner_join_desc.json b/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_ii_inner_join_desc.json
deleted file mode 100644
index eb383fb..0000000
--- a/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_ii_inner_join_desc.json
+++ /dev/null
@@ -1,69 +0,0 @@
-{
- 
-  "uuid": "88bf87b5-c7b5-4420-a12a-07f6b37b3187",
-  "last_modified": 0,
-  "name": "test_kylin_ii_inner_join_desc",
-  "model_name": "test_kylin_inner_join_model_desc",
-  "timestamp_dimension": "cal_dt",
-  "value_dimensions": [
-    {
-      "table": "default.test_kylin_fact",
-      "columns": [
-        "lstg_format_name",
-        "LSTG_SITE_ID",
-        "SLR_SEGMENT_CD"
-      ]
-    },
-    {
-      "table": "default.test_category_groupings",
-      "columns": [
-        "leaf_categ_id",
-        "site_id",
-        "USER_DEFINED_FIELD1",
-        "USER_DEFINED_FIELD3",
-        "UPD_DATE",
-        "UPD_USER",
-        "meta_categ_name",
-        "categ_lvl2_name",
-        "categ_lvl3_name"
-      ]
-    },
-    {
-      "table": "edw.test_sites",
-      "columns": [
-        "site_id",
-        "site_name",
-        "cre_user"
-      ]
-    },
-    {
-      "table": "edw.test_seller_type_dim",
-      "columns": [
-        "seller_type_cd",
-        "seller_type_desc"
-      ]
-    },
-    {
-      "table": "edw.test_cal_dt",
-      "columns": [
-        "cal_dt",
-        "week_beg_dt"
-      ]
-    },
-    {
-      "table": "default.test_kylin_fact",
-      "columns": [
-        "TRANS_ID",
-        "CAL_DT",
-        "LEAF_CATEG_ID",
-        "SELLER_ID"
-      ]
-    }
-  ],
-  "metrics": [
-    "PRICE",
-    "ITEM_COUNT"
-  ],
-  "sharding": 1,
-  "slice_size": 1000
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_ii_left_join_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_ii_left_join_desc.json b/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_ii_left_join_desc.json
deleted file mode 100644
index 2f152ad..0000000
--- a/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_ii_left_join_desc.json
+++ /dev/null
@@ -1,69 +0,0 @@
-{
- 
-  "uuid": "74bf87b5-c7b5-4420-a12a-07f6b37b3187",
-  "last_modified": 0,
-  "name": "test_kylin_ii_left_join_desc",
-  "model_name": "test_kylin_left_join_model_desc",
-  "timestamp_dimension": "cal_dt",
-  "value_dimensions": [
-    {
-      "table": "default.test_kylin_fact",
-      "columns": [
-        "lstg_format_name",
-        "LSTG_SITE_ID",
-        "SLR_SEGMENT_CD"
-      ]
-    },
-    {
-      "table": "default.test_category_groupings",
-      "columns": [
-        "leaf_categ_id",
-        "site_id",
-        "USER_DEFINED_FIELD1",
-        "USER_DEFINED_FIELD3",
-        "UPD_DATE",
-        "UPD_USER",
-        "meta_categ_name",
-        "categ_lvl2_name",
-        "categ_lvl3_name"
-      ]
-    },
-    {
-      "table": "edw.test_sites",
-      "columns": [
-        "site_id",
-        "site_name",
-        "cre_user"
-      ]
-    },
-    {
-      "table": "edw.test_seller_type_dim",
-      "columns": [
-        "seller_type_cd",
-        "seller_type_desc"
-      ]
-    },
-    {
-      "table": "edw.test_cal_dt",
-      "columns": [
-        "cal_dt",
-        "week_beg_dt"
-      ]
-    },
-    {
-      "table": "default.test_kylin_fact",
-      "columns": [
-        "TRANS_ID",
-        "CAL_DT",
-        "LEAF_CATEG_ID",
-        "SELLER_ID"
-      ]
-    }
-  ],
-  "metrics": [
-    "PRICE",
-    "ITEM_COUNT"
-  ],
-  "sharding": 1,
-  "slice_size": 1000
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/examples/test_case_data/localmeta/invertedindex_desc/test_streaming_table_ii_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/invertedindex_desc/test_streaming_table_ii_desc.json b/examples/test_case_data/localmeta/invertedindex_desc/test_streaming_table_ii_desc.json
deleted file mode 100644
index c39ae90..0000000
--- a/examples/test_case_data/localmeta/invertedindex_desc/test_streaming_table_ii_desc.json
+++ /dev/null
@@ -1,26 +0,0 @@
-{
- 
-  "uuid": "74bf87b5-c7b5-4420-a12a-07f6b3864789",
-  "last_modified": 0,
-  "name": "test_streaming_table_ii_desc",
-  "model_name": "test_streaming_table_model_desc",
-  "timestamp_dimension": "minute_start",
-  "value_dimensions": [
-    {
-      "table": "default.streaming_table",
-      "columns": [
-        "minute_start",
-        "hour_start",
-        "day_start",
-        "itm",
-        "site"
-      ]
-    }
-  ],
-  "metrics": [
-    "gmv",
-    "item_count"
-  ],
-  "sharding": 1,
-  "slice_size": 1000
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/examples/test_case_data/localmeta/project/default.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/project/default.json b/examples/test_case_data/localmeta/project/default.json
index 88538c9..3026ff8 100644
--- a/examples/test_case_data/localmeta/project/default.json
+++ b/examples/test_case_data/localmeta/project/default.json
@@ -24,16 +24,6 @@
       "realization": "test_kylin_cube_without_slr_left_join_empty"
     },
     {
-      "name": "test_kylin_ii_left_join",
-      "type": "INVERTED_INDEX",
-      "realization": "test_kylin_ii_left_join"
-    },
-    {
-      "name": "test_kylin_ii_inner_join",
-      "type": "INVERTED_INDEX",
-      "realization": "test_kylin_ii_inner_join"
-    },
-    {
       "name": "test_streaming_table_cube",
       "type": "CUBE",
       "realization": "test_streaming_table_cube"

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/invertedindex/.settings/org.eclipse.core.resources.prefs b/invertedindex/.settings/org.eclipse.core.resources.prefs
deleted file mode 100644
index 29abf99..0000000
--- a/invertedindex/.settings/org.eclipse.core.resources.prefs
+++ /dev/null
@@ -1,6 +0,0 @@
-eclipse.preferences.version=1
-encoding//src/main/java=UTF-8
-encoding//src/main/resources=UTF-8
-encoding//src/test/java=UTF-8
-encoding//src/test/resources=UTF-8
-encoding/<project>=UTF-8


[28/55] [abbrv] kylin git commit: Fix RAT warnings

Posted by sh...@apache.org.
Fix RAT warnings

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: c92f71cc137a8c21d0b6fe55b0540efd43aadecd
Parents: 761db1e
Author: shaofengshi <sh...@apache.org>
Authored: Tue Jul 19 09:55:57 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Jul 19 09:56:48 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/dict/CachedTreeMapTest.java   | 17 +++++++++++++++++
 pom.xml                                            |  1 +
 2 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/c92f71cc/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
index 3366bf6..df64f3f 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/CachedTreeMapTest.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
 package org.apache.kylin.dict;
 
 import org.apache.hadoop.io.Writable;

http://git-wip-us.apache.org/repos/asf/kylin/blob/c92f71cc/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7a211fc..9f35ed3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -783,6 +783,7 @@
                                 <exclude>**/*.iml</exclude>
                                 <exclude>**/.classpath</exclude>
                                 <exclude>**/.project</exclude>
+                                <exclude>**/.settings/**</exclude>
 
                                 <!-- image files constitute images required for documentation. .pptx contain the sources for images -->
                                 <exclude>**/*.png</exclude>


[13/55] [abbrv] kylin git commit: KYLIN-1656 set default ‘kylin.job.mapreduce.mapper.input.rows’ to 1million

Posted by sh...@apache.org.
KYLIN-1656 set default \u2018kylin.job.mapreduce.mapper.input.rows\u2019 to 1million

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: e0ffd22f871f0846d4e0573183dd74f598ea30e9
Parents: b45dc44
Author: shaofengshi <sh...@apache.org>
Authored: Sat Jul 9 10:26:04 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sat Jul 9 10:26:04 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfigBase.java   |  2 +-
 .../kylin/source/hive/CreateFlatHiveTableStep.java | 17 +++++++++++++++++
 2 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/e0ffd22f/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index bfbaaa4..b146031 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -320,7 +320,7 @@ abstract public class KylinConfigBase implements Serializable {
     }
 
     public int getHadoopJobMapperInputRows() {
-        return Integer.parseInt(getOptional("kylin.job.mapreduce.mapper.input.rows", "500000"));
+        return Integer.parseInt(getOptional("kylin.job.mapreduce.mapper.input.rows", "1000000"));
     }
 
     public boolean getRunAsRemoteCommand() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/e0ffd22f/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java b/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java
index d8e2754..0297bfa 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/CreateFlatHiveTableStep.java
@@ -1,3 +1,20 @@
+/*
+ * 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;


[45/55] [abbrv] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
index f8e2644..7e3d98e 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
@@ -29,12 +29,10 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.ColumnDesc;
@@ -230,15 +228,8 @@ public class AggregateRegionObserverTest {
             return nextRaw(results);
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.InternalScanner#next(java.util
-         * .List, int)
-         */
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -307,6 +298,11 @@ public class AggregateRegionObserverTest {
             return 0;
         }
 
+        @Override
+        public int getBatch() {
+            return 0;
+        }
+
         /*
          * (non-Javadoc)
          * 
@@ -323,16 +319,9 @@ public class AggregateRegionObserverTest {
             return i < input.size();
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#nextRaw(java.util
-         * .List, int)
-         */
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-            return nextRaw(result);
+        public boolean nextRaw(List<Cell> list, ScannerContext scannerContext) throws IOException {
+            return false;
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
index 1d85922..04e2e8b 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -136,7 +137,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
 
                         Put p = new Put(rk);
                         p.setDurability(Durability.SKIP_WAL);
-                        p.add(cf.getBytes(), cq, Bytes.toBytes(c));
+                        p.addColumn(cf.getBytes(), cq, Bytes.toBytes(c));
                         ht.put(p);
                     }
                 }
@@ -224,7 +225,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
         scan.addFamily(cf.getBytes());
         scan.setFilter(filter);
         List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(table.getBytes());
-        HRegion first = regions.get(0);
+        Region first = regions.get(0);
         first.getScanner(scan);
         RegionScanner scanner = first.getScanner(scan);
         List<Cell> results = new ArrayList<Cell>();


[37/55] [abbrv] kylin git commit: KYLIN-1846 minimize dependencies of JDBC driver

Posted by sh...@apache.org.
KYLIN-1846 minimize dependencies of JDBC driver

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


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: edd528dac94a2b68dbf4b71810a21bcdbdce8431
Parents: 2ba0478
Author: gaodayue <ga...@meituan.com>
Authored: Tue Jul 5 10:23:54 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Jul 22 12:37:49 2016 +0800

----------------------------------------------------------------------
 jdbc/pom.xml                                    | 69 +++++++++++++-------
 .../main/java/org/apache/kylin/jdbc/Driver.java |  2 -
 .../java/org/apache/kylin/jdbc/KylinMeta.java   |  6 +-
 .../apache/kylin/jdbc/util/Log4jConfigurer.java | 69 --------------------
 jdbc/src/main/resources/kylin-log4j.properties  | 28 --------
 pom.xml                                         |  5 ++
 6 files changed, 51 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/edd528da/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index 3977d85..deb3b49 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -33,15 +33,12 @@
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <shadeBase>org.apache.kylin.jdbc.shaded</shadeBase>
     </properties>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.calcite</groupId>
-            <artifactId>calcite-core</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.calcite</groupId>
             <artifactId>calcite-avatica</artifactId>
             <exclusions>
                 <exclusion>
@@ -55,22 +52,10 @@
             <artifactId>httpclient</artifactId>
         </dependency>
         <dependency>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>jcl-over-slf4j</artifactId>
-        </dependency>
-        <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-log4j12</artifactId>
-        </dependency>
-        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>
@@ -81,20 +66,54 @@
     <build>
         <plugins>
             <plugin>
-                <artifactId>maven-assembly-plugin</artifactId>
-                <configuration>
-                    <descriptorRefs>
-                        <descriptorRef>jar-with-dependencies</descriptorRef>
-                    </descriptorRefs>
-                    <appendAssemblyId>false</appendAssemblyId>
-                </configuration>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
                 <executions>
                     <execution>
-                        <id>make-assembly</id>
                         <phase>package</phase>
                         <goals>
-                            <goal>single</goal>
+                            <goal>shade</goal>
                         </goals>
+                        <configuration>
+                            <createDependencyReducedPom>false</createDependencyReducedPom>
+                            <artifactSet>
+                                <!-- jackson is already packaged into calcite-avatica.jar. To avoid including jackson
+                                twice, we include calcite-avatica which has jackson and exclude jackson. -->
+                                <excludes>
+                                    <exclude>com.fasterxml.jackson.core:*</exclude>
+                                </excludes>
+                            </artifactSet>
+                            <relocations>
+                                <relocation>
+                                    <pattern>org.apache.calcite</pattern>
+                                    <shadedPattern>${shadeBase}.org.apache.calcite</shadedPattern>
+                                </relocation>
+                                <relocation>
+                                    <pattern>com.fasterxml.jackson</pattern>
+                                    <shadedPattern>${shadeBase}.com.fasterxml.jackson</shadedPattern>
+                                </relocation>
+                                <relocation>
+                                    <pattern>org.apache.commons</pattern>
+                                    <shadedPattern>${shadeBase}.org.apache.commons</shadedPattern>
+                                </relocation>
+                            </relocations>
+                            <filters>
+                                <filter>
+                                    <artifact>*:*</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/*.SF</exclude>
+                                        <exclude>META-INF/*.DSA</exclude>
+                                        <exclude>META-INF/*.RSA</exclude>
+                                    </excludes>
+                                </filter>
+                                <filter>
+                                    <artifact>commons-httpclient:commons-httpclient</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/**</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                        </configuration>
                     </execution>
                 </executions>
             </plugin>

http://git-wip-us.apache.org/repos/asf/kylin/blob/edd528da/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java b/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java
index 7f3c407..f9d840a 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java
@@ -25,7 +25,6 @@ import org.apache.calcite.avatica.AvaticaConnection;
 import org.apache.calcite.avatica.DriverVersion;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.UnregisteredDriver;
-import org.apache.kylin.jdbc.util.Log4jConfigurer;
 
 /**
  * <p>
@@ -73,7 +72,6 @@ public class Driver extends UnregisteredDriver {
     public static final String CONNECT_STRING_PREFIX = "jdbc:kylin:";
     static {
         try {
-            Log4jConfigurer.initLogger();
             DriverManager.registerDriver(new Driver());
         } catch (SQLException e) {
             throw new RuntimeException("Error occurred while registering JDBC driver " + Driver.class.getName() + ": " + e.toString());

http://git-wip-us.apache.org/repos/asf/kylin/blob/edd528da/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
index fdcc91b..8059dd0 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
@@ -35,8 +35,6 @@ import org.apache.calcite.avatica.NoSuchStatementException;
 import org.apache.calcite.avatica.QueryState;
 import org.apache.calcite.avatica.remote.TypedValue;
 
-import com.google.common.collect.ImmutableList;
-
 /**
  * Implementation of Avatica interface
  */
@@ -64,7 +62,7 @@ public class KylinMeta extends MetaImpl {
     @Override
     public ExecuteResult execute(StatementHandle sh, List<TypedValue> parameterValues, long maxRowCount) throws NoSuchStatementException {
         final MetaResultSet metaResultSet = MetaResultSet.create(sh.connectionId, sh.id, false, sh.signature, null);
-        return new ExecuteResult(ImmutableList.of(metaResultSet));
+        return new ExecuteResult(Collections.singletonList(metaResultSet));
     }
 
     // mimic from CalciteMetaImpl, real execution happens via callback in KylinResultSet.execute()
@@ -78,7 +76,7 @@ public class KylinMeta extends MetaImpl {
             }
             callback.execute();
             final MetaResultSet metaResultSet = MetaResultSet.create(sh.connectionId, sh.id, false, sh.signature, null);
-            return new ExecuteResult(ImmutableList.of(metaResultSet));
+            return new ExecuteResult(Collections.singletonList(metaResultSet));
         } catch (SQLException e) {
             throw new RuntimeException(e);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/edd528da/jdbc/src/main/java/org/apache/kylin/jdbc/util/Log4jConfigurer.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/util/Log4jConfigurer.java b/jdbc/src/main/java/org/apache/kylin/jdbc/util/Log4jConfigurer.java
deleted file mode 100644
index d68a46f..0000000
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/util/Log4jConfigurer.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.jdbc.util;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Enumeration;
-
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.apache.log4j.PropertyConfigurator;
-
-/**
- * Created by dongli on 11/24/15.
- */
-public class Log4jConfigurer {
-    private static boolean INITIALIZED = false;
-
-    public static void initLogger() {
-        if (!INITIALIZED && !isConfigured()) {
-            InputStream is = Log4jConfigurer.class.getClassLoader().getResourceAsStream("kylin-log4j.properties");
-            PropertyConfigurator.configure(is);
-            closeQuietly(is);
-        }
-        INITIALIZED = true;
-    }
-
-    private static boolean isConfigured() {
-        if (LogManager.getRootLogger().getAllAppenders().hasMoreElements()) {
-            return true;
-        } else {
-            Enumeration<?> loggers = LogManager.getCurrentLoggers();
-            while (loggers.hasMoreElements()) {
-                Logger logger = (Logger) loggers.nextElement();
-                if (logger.getAllAppenders().hasMoreElements())
-                    return true;
-            }
-        }
-        return false;
-    }
-
-    public static void closeQuietly(Closeable closeable) {
-        try {
-            if (closeable != null) {
-                closeable.close();
-            }
-        } catch (IOException ioe) {
-            // ignore
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/edd528da/jdbc/src/main/resources/kylin-log4j.properties
----------------------------------------------------------------------
diff --git a/jdbc/src/main/resources/kylin-log4j.properties b/jdbc/src/main/resources/kylin-log4j.properties
deleted file mode 100644
index e7201b3..0000000
--- a/jdbc/src/main/resources/kylin-log4j.properties
+++ /dev/null
@@ -1,28 +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.
-#
-
-# enable this by -Dlog4j.configuration=kylin-log4j.properties
-
-log4j.rootLogger=INFO,stdout
-
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t %c{1}:%L]: %m%n
-
-#log4j.logger.org.apache.hadoop=ERROR
-log4j.logger.org.apache.kylin=DEBUG
-log4j.logger.org.springframework=WARN

http://git-wip-us.apache.org/repos/asf/kylin/blob/edd528da/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9f35ed3..94f4e33 100644
--- a/pom.xml
+++ b/pom.xml
@@ -533,6 +533,11 @@
                 </plugin>
                 <plugin>
                     <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-shade-plugin</artifactId>
+                    <version>2.4.3</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-war-plugin</artifactId>
                     <version>2.6</version>
                     <configuration>


[38/55] [abbrv] kylin git commit: KYLIN-1846 minor also shade httpcomponents in jdbc

Posted by sh...@apache.org.
KYLIN-1846 minor also shade httpcomponents in jdbc


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 658839d63ccccfd60518a0a4659e7ab96feae22b
Parents: edd528d
Author: gaodayue <ga...@meituan.com>
Authored: Mon Jul 11 10:58:06 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Jul 22 12:38:04 2016 +0800

----------------------------------------------------------------------
 core-common/pom.xml |  4 ----
 jdbc/pom.xml        | 10 ++++------
 2 files changed, 4 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/658839d6/core-common/pom.xml
----------------------------------------------------------------------
diff --git a/core-common/pom.xml b/core-common/pom.xml
index 5d5369f..ca4c7a9 100644
--- a/core-common/pom.xml
+++ b/core-common/pom.xml
@@ -65,10 +65,6 @@
             <artifactId>commons-email</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.httpcomponents</groupId>
-            <artifactId>httpclient</artifactId>
-        </dependency>
-        <dependency>
 	        <groupId>org.apache.httpcomponents</groupId>
 	        <artifactId>httpclient</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/658839d6/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index deb3b49..a663fb3 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -93,6 +93,10 @@
                                     <shadedPattern>${shadeBase}.com.fasterxml.jackson</shadedPattern>
                                 </relocation>
                                 <relocation>
+                                    <pattern>org.apache.http</pattern>
+                                    <shadedPattern>${shadeBase}.org.apache.http</shadedPattern>
+                                </relocation>
+                                <relocation>
                                     <pattern>org.apache.commons</pattern>
                                     <shadedPattern>${shadeBase}.org.apache.commons</shadedPattern>
                                 </relocation>
@@ -106,12 +110,6 @@
                                         <exclude>META-INF/*.RSA</exclude>
                                     </excludes>
                                 </filter>
-                                <filter>
-                                    <artifact>commons-httpclient:commons-httpclient</artifact>
-                                    <excludes>
-                                        <exclude>META-INF/**</exclude>
-                                    </excludes>
-                                </filter>
                             </filters>
                         </configuration>
                     </execution>


[34/55] [abbrv] kylin git commit: KYLIN-1912: Fix Beeline build intermediate statement issue

Posted by sh...@apache.org.
KYLIN-1912: Fix Beeline build intermediate statement issue

Signed-off-by: lidongsjtu <li...@apache.org>


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 1c257be08f39cf39453efe951ce6abca5f47b218
Parents: 5268c54
Author: Yiming Liu <li...@gmail.com>
Authored: Fri Jul 22 07:48:48 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Jul 22 12:28:22 2016 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java   | 4 ++++
 .../src/main/java/org/apache/kylin/source/hive/HiveMRInput.java  | 2 +-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/1c257be0/source-hive/src/main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java
index 6316b9c..5a5b4e0 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java
@@ -20,6 +20,7 @@ package org.apache.kylin.source.hive;
 
 import java.io.BufferedWriter;
 import java.io.File;
+import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -78,6 +79,9 @@ public class HiveCmdBuilder {
                 buf.append(tmpHql.getAbsolutePath());
 
                 logger.info("The statements to execute in beeline: \n" + hqlBuf);
+                if (logger.isDebugEnabled()) {
+                    logger.debug("THe SQL to execute in beeline: \n" + IOUtils.toString(new FileReader(tmpHql)));
+                }
             } catch (IOException e) {
                 throw new RuntimeException(e);
             } finally {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1c257be0/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index e4492da..5de9a98 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -186,7 +186,7 @@ public class HiveMRInput implements IMRInput {
             StringBuilder hiveInitBuf = new StringBuilder();
             hiveInitBuf.append(JoinedFlatTable.generateHiveSetStatements(conf));
 
-            final String useDatabaseHql = "USE " + conf.getConfig().getHiveDatabaseForIntermediateTable() + ";";
+            final String useDatabaseHql = "USE " + conf.getConfig().getHiveDatabaseForIntermediateTable() + ";\n";
             final String dropTableHql = JoinedFlatTable.generateDropTableStatement(flatTableDesc);
             final String createTableHql = JoinedFlatTable.generateCreateTableStatement(flatTableDesc, JobBuilderSupport.getJobWorkingDir(conf, jobId));
             String insertDataHqls;


[48/55] [abbrv] kylin git commit: KYLIN-1758 add the hive set statements

Posted by sh...@apache.org.
KYLIN-1758 add the hive set statements


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 76d3bd21a46549dd6da4db1ffac3bdc5dcbe12c4
Parents: 9bac838
Author: shaofengshi <sh...@apache.org>
Authored: Sat Jun 4 16:22:38 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 25 20:13:38 2016 +0800

----------------------------------------------------------------------
 jdbc/pom.xml                                                  | 4 ----
 server/pom.xml                                                | 4 ----
 .../main/java/org/apache/kylin/source/hive/HiveMRInput.java   | 7 ++++---
 3 files changed, 4 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/76d3bd21/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index ff83568..e86998e 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -27,11 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-<<<<<<< 571874049bdb9139afdc467474b53a1d63be818b
         <version>1.5.3</version>
-=======
-        <version>1.5.2.1</version>
->>>>>>> update version number
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/76d3bd21/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index e0a4110..bca91fb 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -28,11 +28,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-<<<<<<< 571874049bdb9139afdc467474b53a1d63be818b
         <version>1.5.3</version>
-=======
-        <version>1.5.2.1</version>
->>>>>>> update version number
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/76d3bd21/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index 5de9a98..0dfeb5a 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -132,7 +132,6 @@ public class HiveMRInput implements IMRInput {
             hiveCmdBuilder.addStatement(JoinedFlatTable.generateHiveSetStatements(conf));
             hiveCmdBuilder.addStatement("set hive.exec.compress.output=false;\n");
             hiveCmdBuilder.addStatement(JoinedFlatTable.generateCountDataStatement(flatTableDesc, rowCountOutputDir));
-
             step.setCmd(hiveCmdBuilder.build());
             step.setName(ExecutableConstants.STEP_NAME_COUNT_HIVE_TABLE);
 
@@ -161,10 +160,12 @@ public class HiveMRInput implements IMRInput {
             if (lookupViewsTables.size() == 0) {
                 return null;
             }
+            String setHql = "";
+            setHql = JoinedFlatTable.generateHiveSetStatements(conf);
             final String useDatabaseHql = "USE " + conf.getConfig().getHiveDatabaseForIntermediateTable() + ";";
             hiveCmdBuilder.addStatement(useDatabaseHql);
-            hiveCmdBuilder.addStatement(JoinedFlatTable.generateHiveSetStatements(conf));
-            for (TableDesc lookUpTableDesc : lookupViewsTables) {
+            hiveCmdBuilder.addStatement(setHql);
+            for(TableDesc lookUpTableDesc : lookupViewsTables) {
                 if (TableDesc.TABLE_TYPE_VIRTUAL_VIEW.equalsIgnoreCase(lookUpTableDesc.getTableType())) {
                     StringBuilder createIntermediateTableHql = new StringBuilder();
                     createIntermediateTableHql.append("DROP TABLE IF EXISTS " + lookUpTableDesc.getMaterializedName() + ";\n");


[51/55] [abbrv] kylin git commit: minor change on hbase conf

Posted by sh...@apache.org.
minor change on hbase conf


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 8e780c2c73cff5ac44f5040d10f8378f5b6facbb
Parents: 90a0c5b
Author: lidongsjtu <li...@apache.org>
Authored: Mon May 30 17:34:41 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Jul 26 09:29:15 2016 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/storage/hbase/HBaseConnection.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8e780c2c/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
index e7ee2f5..7dbc7f0 100644
--- 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
@@ -283,3 +283,4 @@ public class HBaseConnection {
     }
 
 }
+


[46/55] [abbrv] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
index 938145b..9b8a3de 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
@@ -24,8 +24,9 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -164,8 +165,8 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
         // primary key (also the 0th column block) is always selected
         final ImmutableBitSet selectedColBlocks = scanRequest.getSelectedColBlocks().set(0);
         // globally shared connection, does not require close
-        HConnection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
-        final HTableInterface hbaseTable = hbaseConn.getTable(cubeSeg.getStorageLocationIdentifier());
+        Connection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+        final Table hbaseTable = hbaseConn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()));
 
         List<RawScan> rawScans = preparedHBaseScans(scanRequest.getGTScanRanges(), selectedColBlocks);
         List<List<Integer>> hbaseColumnsToGT = getHBaseColumnsGTMapping(selectedColBlocks);

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index 6ded5f5..3dc2aed 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -141,7 +141,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         if (shardLength == 0) {
             return;
         }
-        byte[] regionStartKey = ArrayUtils.isEmpty(region.getStartKey()) ? new byte[shardLength] : region.getStartKey();
+        byte[] regionStartKey = ArrayUtils.isEmpty(region.getRegionInfo().getStartKey()) ? new byte[shardLength] : region.getRegionInfo().getStartKey();
         Bytes.putBytes(rawScan.startKey, 0, regionStartKey, 0, shardLength);
         Bytes.putBytes(rawScan.endKey, 0, regionStartKey, 0, shardLength);
     }
@@ -177,7 +177,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         try {
             this.serviceStartTime = System.currentTimeMillis();
 
-            region = env.getRegion();
+            region = (HRegion)env.getRegion();
             region.startRegionOperation();
 
             // if user change kylin.properties on kylin server, need to manually redeploy coprocessor jar to update KylinConfig of Env.

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
index fe65598..ae26508 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
@@ -25,7 +25,8 @@ 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.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -78,7 +79,8 @@ public class CubeHTableUtil {
         tableDesc.setValue(IRealizationConstants.HTableSegmentTag, cubeSegment.toString());
 
         Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+        Admin admin = conn.getAdmin();
 
         try {
             if (User.isHBaseSecurityEnabled(conf)) {
@@ -91,7 +93,7 @@ public class CubeHTableUtil {
                 tableDesc.addFamily(cf);
             }
 
-            if (admin.tableExists(tableName)) {
+            if (admin.tableExists(TableName.valueOf(tableName))) {
                 // admin.disableTable(tableName);
                 // admin.deleteTable(tableName);
                 throw new RuntimeException("HBase table " + tableName + " exists!");
@@ -100,7 +102,7 @@ public class CubeHTableUtil {
             DeployCoprocessorCLI.deployCoprocessor(tableDesc);
 
             admin.createTable(tableDesc, splitKeys);
-            Preconditions.checkArgument(admin.isTableAvailable(tableName), "table " + tableName + " created, but is not available due to some reasons");
+            Preconditions.checkArgument(admin.isTableAvailable(TableName.valueOf(tableName)), "table " + tableName + " created, but is not available due to some reasons");
             logger.info("create hbase table " + tableName + " done.");
         } finally {
             admin.close();
@@ -109,8 +111,7 @@ public class CubeHTableUtil {
     }
 
     public static void deleteHTable(TableName tableName) throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
         try {
             if (admin.tableExists(tableName)) {
                 logger.info("disabling hbase table " + tableName);
@@ -125,8 +126,7 @@ public class CubeHTableUtil {
 
     /** create a HTable that has the same performance settings as normal cube table, for benchmark purpose */
     public static void createBenchmarkHTable(TableName tableName, String cfName) throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
         try {
             if (admin.tableExists(tableName)) {
                 logger.info("disabling hbase table " + tableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
index 7aecd7e..9dc9715 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
@@ -28,9 +28,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;
@@ -99,19 +100,21 @@ public class DeprecatedGCStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            HBaseAdmin admin = null;
+            Admin admin = null;
             try {
-                admin = new HBaseAdmin(conf);
+
+                Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+                admin = conn.getAdmin();
+
                 for (String table : oldTables) {
-                    if (admin.tableExists(table)) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                    if (admin.tableExists(TableName.valueOf(table))) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf(table));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(table)) {
-                                admin.disableTable(table);
+                            if (admin.isTableEnabled(TableName.valueOf(table))) {
+                                admin.disableTable(TableName.valueOf(table));
                             }
-                            admin.deleteTable(table);
+                            admin.deleteTable(TableName.valueOf(table));
                             logger.debug("Dropped HBase table " + table);
                             output.append("Dropped HBase table " + table + " \n");
                         } else {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
index 4fe7748..16955dd 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
@@ -38,7 +38,7 @@ import java.util.List;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.CubeSegment;
@@ -64,7 +64,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
 
     private final List<KeyValueCreator> keyValueCreators;
     private final int nColumns;
-    private final HTableInterface hTable;
+    private final Table hTable;
     private final CubeDesc cubeDesc;
     private final CubeSegment cubeSegment;
     private final Object[] measureValues;
@@ -73,7 +73,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
     private AbstractRowKeyEncoder rowKeyEncoder;
     private byte[] keybuf;
 
-    public HBaseCuboidWriter(CubeSegment segment, HTableInterface hTable) {
+    public HBaseCuboidWriter(CubeSegment segment, Table hTable) {
         this.keyValueCreators = Lists.newArrayList();
         this.cubeSegment = segment;
         this.cubeDesc = cubeSegment.getCubeDesc();
@@ -132,11 +132,12 @@ public class HBaseCuboidWriter implements ICuboidWriter {
             long t = System.currentTimeMillis();
             if (hTable != null) {
                 hTable.put(puts);
-                hTable.flushCommits();
             }
             logger.info("commit total " + puts.size() + " puts, totally cost:" + (System.currentTimeMillis() - t) + "ms");
             puts.clear();
         }
+        logger.info("commit total " + puts.size() + " puts, totally cost:" + (System.currentTimeMillis() - t) + "ms");
+        puts.clear();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java
index 9adaf24..e1e2cba 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java
@@ -27,7 +27,8 @@ 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.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.cube.CubeSegment;
@@ -56,7 +57,7 @@ public class HBaseStreamingOutput implements IStreamingOutput {
         try {
             CubeSegment cubeSegment = (CubeSegment) buildable;
 
-            final HTableInterface hTable;
+            final Table hTable;
             hTable = createHTable(cubeSegment);
             List<ICuboidWriter> cuboidWriters = Lists.newArrayList();
             cuboidWriters.add(new HBaseCuboidWriter(cubeSegment, hTable));
@@ -88,10 +89,10 @@ public class HBaseStreamingOutput implements IStreamingOutput {
         }
     }
 
-    private HTableInterface createHTable(final CubeSegment cubeSegment) throws IOException {
+    private Table createHTable(final CubeSegment cubeSegment) throws IOException {
         final String hTableName = cubeSegment.getStorageLocationIdentifier();
         CubeHTableUtil.createHTable(cubeSegment, null);
-        final HTableInterface hTable = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getTable(hTableName);
+        final Table hTable = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getTable(TableName.valueOf(hTableName));
         logger.info("hTable:" + hTableName + " for segment:" + cubeSegment.getName() + " created!");
         return hTable;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
index 5b2441c..2f7e164 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
@@ -24,11 +24,11 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableContext;
@@ -69,19 +69,20 @@ public class MergeGCStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            HBaseAdmin admin = null;
+            Admin admin = null;
             try {
-                admin = new HBaseAdmin(conf);
+                Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+                admin = conn.getAdmin();
+
                 for (String table : oldTables) {
-                    if (admin.tableExists(table)) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                    if (admin.tableExists(TableName.valueOf(table))) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf((table)));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(table)) {
-                                admin.disableTable(table);
+                            if (admin.isTableEnabled(TableName.valueOf(table))) {
+                                admin.disableTable(TableName.valueOf(table));
                             }
-                            admin.deleteTable(table);
+                            admin.deleteTable(TableName.valueOf(table));
                             logger.debug("Dropped htable: " + table);
                             output.append("HBase table " + table + " is dropped. \n");
                         } else {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
index 9e30a06..8c19b9f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
@@ -21,10 +21,12 @@ package org.apache.kylin.storage.hbase.util;
 import java.io.IOException;
 
 import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.storage.hbase.HBaseConnection;
@@ -52,8 +54,8 @@ public class CleanHtableCLI extends AbstractHadoopJob {
     }
 
     private void clean() throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
 
         for (HTableDescriptor descriptor : hbaseAdmin.listTables()) {
             String name = descriptor.getNameAsString().toLowerCase();
@@ -64,7 +66,7 @@ public class CleanHtableCLI extends AbstractHadoopJob {
                 System.out.println();
 
                 descriptor.setValue(IRealizationConstants.HTableOwner, "DL-eBay-Kylin@ebay.com");
-                hbaseAdmin.modifyTable(descriptor.getNameAsString(), descriptor);
+                hbaseAdmin.modifyTable(TableName.valueOf(descriptor.getNameAsString()), descriptor);
             }
         }
         hbaseAdmin.close();

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index 9a83a3f..6b63e66 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -84,7 +84,7 @@ public class CubeMigrationCLI {
     private static ResourceStore srcStore;
     private static ResourceStore dstStore;
     private static FileSystem hdfsFS;
-    private static HBaseAdmin hbaseAdmin;
+    private static Admin hbaseAdmin;
 
     public static final String ACL_INFO_FAMILY = "i";
     private static final String ACL_TABLE_NAME = "_acl";
@@ -128,8 +128,8 @@ public class CubeMigrationCLI {
 
         checkAndGetHbaseUrl();
 
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(srcConfig.getStorageUrl());
+        hbaseAdmin = conn.getAdmin();
 
         hdfsFS = FileSystem.get(new Configuration());
 
@@ -154,6 +154,10 @@ public class CubeMigrationCLI {
         } else {
             showOpts();
         }
+        
+        checkMigrationSuccess(dstConfig, cubeName, true);
+
+        IOUtils.closeQuietly(hbaseAdmin);
     }
 
     public static void moveCube(String srcCfgUri, String dstCfgUri, String cubeName, String projectName, String copyAcl, String purgeAndDisable, String overwriteIfExists, String realExecute) throws IOException, InterruptedException {
@@ -320,171 +324,172 @@ public class CubeMigrationCLI {
         logger.info("Executing operation: " + opt.toString());
 
         switch (opt.type) {
-        case CHANGE_HTABLE_HOST: {
-            String tableName = (String) opt.params[0];
-            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-            hbaseAdmin.disableTable(tableName);
-            desc.setValue(IRealizationConstants.HTableTag, dstConfig.getMetadataUrlPrefix());
-            hbaseAdmin.modifyTable(tableName, desc);
-            hbaseAdmin.enableTable(tableName);
-            logger.info("CHANGE_HTABLE_HOST is completed");
-            break;
-        }
-        case COPY_FILE_IN_META: {
-            String item = (String) opt.params[0];
-            RawResource res = srcStore.getResource(item);
-            dstStore.putResource(item, res.inputStream, res.timestamp);
-            res.inputStream.close();
-            logger.info("Item " + item + " is copied");
-            break;
-        }
-        case COPY_DICT_OR_SNAPSHOT: {
-            String item = (String) opt.params[0];
-
-            if (item.toLowerCase().endsWith(".dict")) {
-                DictionaryManager dstDictMgr = DictionaryManager.getInstance(dstConfig);
-                DictionaryManager srcDicMgr = DictionaryManager.getInstance(srcConfig);
-                DictionaryInfo dictSrc = srcDicMgr.getDictionaryInfo(item);
-
-                long ts = dictSrc.getLastModified();
-                dictSrc.setLastModified(0);//to avoid resource store write conflict
-                Dictionary dictObj = dictSrc.getDictionaryObject().copyToAnotherMeta(srcConfig, dstConfig);
-                DictionaryInfo dictSaved = dstDictMgr.trySaveNewDict(dictObj, dictSrc);
-                dictSrc.setLastModified(ts);
-
-                if (dictSaved == dictSrc) {
-                    //no dup found, already saved to dest
-                    logger.info("Item " + item + " is copied");
-                } else {
-                    //dictSrc is rejected because of duplication
-                    //modify cube's dictionary path
-                    String cubeName = (String) opt.params[1];
-                    String cubeResPath = CubeInstance.concatResourcePath(cubeName);
-                    Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
-                    CubeInstance cube = dstStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
-                    for (CubeSegment segment : cube.getSegments()) {
-                        for (Map.Entry<String, String> entry : segment.getDictionaries().entrySet()) {
-                            if (entry.getValue().equalsIgnoreCase(item)) {
-                                entry.setValue(dictSaved.getResourcePath());
+            case CHANGE_HTABLE_HOST: {
+                String tableName = (String) opt.params[0];
+                HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+                hbaseAdmin.disableTable(TableName.valueOf(tableName));
+                desc.setValue(IRealizationConstants.HTableTag, dstConfig.getMetadataUrlPrefix());
+                hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+                hbaseAdmin.enableTable(TableName.valueOf(tableName));
+                logger.info("CHANGE_HTABLE_HOST is completed");
+                break;
+            }
+            case COPY_FILE_IN_META: {
+                String item = (String) opt.params[0];
+                RawResource res = srcStore.getResource(item);
+                dstStore.putResource(item, res.inputStream, res.timestamp);
+                res.inputStream.close();
+                logger.info("Item " + item + " is copied");
+                break;
+            }
+            case COPY_DICT_OR_SNAPSHOT: {
+                String item = (String) opt.params[0];
+
+                if (item.toLowerCase().endsWith(".dict")) {
+                    DictionaryManager dstDictMgr = DictionaryManager.getInstance(dstConfig);
+                    DictionaryManager srcDicMgr = DictionaryManager.getInstance(srcConfig);
+                    DictionaryInfo dictSrc = srcDicMgr.getDictionaryInfo(item);
+
+                    long ts = dictSrc.getLastModified();
+                    dictSrc.setLastModified(0);//to avoid resource store write conflict
+                    DictionaryInfo dictSaved = dstDictMgr.trySaveNewDict(dictSrc.getDictionaryObject(), dictSrc);
+                    dictSrc.setLastModified(ts);
+
+                    if (dictSaved == dictSrc) {
+                        //no dup found, already saved to dest
+                        logger.info("Item " + item + " is copied");
+                    } else {
+                        //dictSrc is rejected because of duplication
+                        //modify cube's dictionary path
+                        String cubeName = (String) opt.params[1];
+                        String cubeResPath = CubeInstance.concatResourcePath(cubeName);
+                        Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
+                        CubeInstance cube = dstStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
+                        for (CubeSegment segment : cube.getSegments()) {
+                            for (Map.Entry<String, String> entry : segment.getDictionaries().entrySet()) {
+                                if (entry.getValue().equalsIgnoreCase(item)) {
+                                    entry.setValue(dictSaved.getResourcePath());
+                                }
                             }
                         }
+                        dstStore.putResource(cubeResPath, cube, cubeSerializer);
+                        logger.info("Item " + item + " is dup, instead " + dictSaved.getResourcePath() + " is reused");
                     }
-                    dstStore.putResource(cubeResPath, cube, cubeSerializer);
-                    logger.info("Item " + item + " is dup, instead " + dictSaved.getResourcePath() + " is reused");
-                }
-
-            } else if (item.toLowerCase().endsWith(".snapshot")) {
-                SnapshotManager dstSnapMgr = SnapshotManager.getInstance(dstConfig);
-                SnapshotManager srcSnapMgr = SnapshotManager.getInstance(srcConfig);
-                SnapshotTable snapSrc = srcSnapMgr.getSnapshotTable(item);
-
-                long ts = snapSrc.getLastModified();
-                snapSrc.setLastModified(0);
-                SnapshotTable snapSaved = dstSnapMgr.trySaveNewSnapshot(snapSrc);
-                snapSrc.setLastModified(ts);
-
-                if (snapSaved == snapSrc) {
-                    //no dup found, already saved to dest
-                    logger.info("Item " + item + " is copied");
-
-                } else {
-                    String cubeName = (String) opt.params[1];
-                    String cubeResPath = CubeInstance.concatResourcePath(cubeName);
-                    Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
-                    CubeInstance cube = dstStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
-                    for (CubeSegment segment : cube.getSegments()) {
-                        for (Map.Entry<String, String> entry : segment.getSnapshots().entrySet()) {
-                            if (entry.getValue().equalsIgnoreCase(item)) {
-                                entry.setValue(snapSaved.getResourcePath());
+                } else if (item.toLowerCase().endsWith(".snapshot")) {
+                    SnapshotManager dstSnapMgr = SnapshotManager.getInstance(dstConfig);
+                    SnapshotManager srcSnapMgr = SnapshotManager.getInstance(srcConfig);
+                    SnapshotTable snapSrc = srcSnapMgr.getSnapshotTable(item);
+
+                    long ts = snapSrc.getLastModified();
+                    snapSrc.setLastModified(0);
+                    SnapshotTable snapSaved = dstSnapMgr.trySaveNewSnapshot(snapSrc);
+                    snapSrc.setLastModified(ts);
+
+                    if (snapSaved == snapSrc) {
+                        //no dup found, already saved to dest
+                        logger.info("Item " + item + " is copied");
+
+                    } else {
+                        String cubeName = (String) opt.params[1];
+                        String cubeResPath = CubeInstance.concatResourcePath(cubeName);
+                        Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
+                        CubeInstance cube = dstStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
+                        for (CubeSegment segment : cube.getSegments()) {
+                            for (Map.Entry<String, String> entry : segment.getSnapshots().entrySet()) {
+                                if (entry.getValue().equalsIgnoreCase(item)) {
+                                    entry.setValue(snapSaved.getResourcePath());
+                                }
                             }
                         }
+                        dstStore.putResource(cubeResPath, cube, cubeSerializer);
+                        logger.info("Item " + item + " is dup, instead " + snapSaved.getResourcePath() + " is reused");
+
                     }
-                    dstStore.putResource(cubeResPath, cube, cubeSerializer);
-                    logger.info("Item " + item + " is dup, instead " + snapSaved.getResourcePath() + " is reused");
 
+                } else {
+                    logger.error("unknown item found: " + item);
+                    logger.info("ignore it");
                 }
-
-            } else {
-                logger.error("unknown item found: " + item);
-                logger.info("ignore it");
+                break;
             }
 
-            break;
-        }
-        case RENAME_FOLDER_IN_HDFS: {
-            String srcPath = (String) opt.params[0];
-            String dstPath = (String) opt.params[1];
-            hdfsFS.rename(new Path(srcPath), new Path(dstPath));
-            logger.info("HDFS Folder renamed from " + srcPath + " to " + dstPath);
-            break;
-        }
-        case ADD_INTO_PROJECT: {
-            String cubeName = (String) opt.params[0];
-            String projectName = (String) opt.params[1];
-            String projectResPath = ProjectInstance.concatResourcePath(projectName);
-            Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
-            ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
-            project.removeRealization(RealizationType.CUBE, cubeName);
-            project.addRealizationEntry(RealizationType.CUBE, cubeName);
-            dstStore.putResource(projectResPath, project, projectSerializer);
-            logger.info("Project instance for " + projectName + " is corrected");
-            break;
-        }
-        case COPY_ACL: {
-            String cubeId = (String) opt.params[0];
-            String modelId = (String) opt.params[1];
-            String projectName = (String) opt.params[2];
-            String projectResPath = ProjectInstance.concatResourcePath(projectName);
-            Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
-            ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
-            String projUUID = project.getUuid();
-            HTableInterface srcAclHtable = null;
-            HTableInterface destAclHtable = null;
-            try {
-                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
-
-                // cube acl
-                Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
-                if (result.listCells() != null) {
-                    for (Cell cell : result.listCells()) {
-                        byte[] family = CellUtil.cloneFamily(cell);
-                        byte[] column = CellUtil.cloneQualifier(cell);
-                        byte[] value = CellUtil.cloneValue(cell);
-
-                        // use the target project uuid as the parent
-                        if (Bytes.toString(family).equals(ACL_INFO_FAMILY) && Bytes.toString(column).equals(ACL_INFO_FAMILY_PARENT_COLUMN)) {
-                            String valueString = "{\"id\":\"" + projUUID + "\",\"type\":\"org.apache.kylin.metadata.project.ProjectInstance\"}";
-                            value = Bytes.toBytes(valueString);
+            case RENAME_FOLDER_IN_HDFS: {
+                String srcPath = (String) opt.params[0];
+                String dstPath = (String) opt.params[1];
+                hdfsFS.rename(new Path(srcPath), new Path(dstPath));
+                logger.info("HDFS Folder renamed from " + srcPath + " to " + dstPath);
+                break;
+            }
+            case ADD_INTO_PROJECT: {
+                String cubeName = (String) opt.params[0];
+                String projectName = (String) opt.params[1];
+                String projectResPath = ProjectInstance.concatResourcePath(projectName);
+                Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
+                ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
+                project.removeRealization(RealizationType.CUBE, cubeName);
+                project.addRealizationEntry(RealizationType.CUBE, cubeName);
+                dstStore.putResource(projectResPath, project, projectSerializer);
+                logger.info("Project instance for " + projectName + " is corrected");
+                break;
+            }
+            case COPY_ACL: {
+                String cubeId = (String) opt.params[0];
+                String modelId = (String) opt.params[1];
+                String projectName = (String) opt.params[2];
+                String projectResPath = ProjectInstance.concatResourcePath(projectName);
+                Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
+                ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
+                String projUUID = project.getUuid();
+                Table srcAclHtable = null;
+                Table destAclHtable = null;
+                try {
+                    srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(TableName.valueOf(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+                    destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+
+                    // cube acl
+                    Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
+                    if (result.listCells() != null) {
+                        for (Cell cell : result.listCells()) {
+                            byte[] family = CellUtil.cloneFamily(cell);
+                            byte[] column = CellUtil.cloneQualifier(cell);
+                            byte[] value = CellUtil.cloneValue(cell);
+
+                            // use the target project uuid as the parent
+                            if (Bytes.toString(family).equals(ACL_INFO_FAMILY) && Bytes.toString(column).equals(ACL_INFO_FAMILY_PARENT_COLUMN)) {
+                                String valueString = "{\"id\":\"" + projUUID + "\",\"type\":\"org.apache.kylin.metadata.project.ProjectInstance\"}";
+                                value = Bytes.toBytes(valueString);
+                            }
+                            Put put = new Put(Bytes.toBytes(cubeId));
+                            put.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), CellUtil.cloneValue(cell));
+                            destAclHtable.put(put);
                         }
                         Put put = new Put(Bytes.toBytes(cubeId));
                         put.add(family, column, value);
                         destAclHtable.put(put);
                     }
+                    destAclHtable.flushCommits();
+                } finally {
+                    IOUtils.closeQuietly(srcAclHtable);
+                    IOUtils.closeQuietly(destAclHtable);
                 }
-                destAclHtable.flushCommits();
-            } finally {
-                IOUtils.closeQuietly(srcAclHtable);
-                IOUtils.closeQuietly(destAclHtable);
+                break;
+            }
+            case PURGE_AND_DISABLE: {
+                String cubeName = (String) opt.params[0];
+                String cubeResPath = CubeInstance.concatResourcePath(cubeName);
+                Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
+                CubeInstance cube = srcStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
+                cube.getSegments().clear();
+                cube.setStatus(RealizationStatusEnum.DISABLED);
+                srcStore.putResource(cubeResPath, cube, cubeSerializer);
+                logger.info("Cube " + cubeName + " is purged and disabled in " + srcConfig.getMetadataUrl());
+
+                break;
+            }
+            default: {
+                //do nothing
+                break;
             }
-            break;
-        }
-        case PURGE_AND_DISABLE: {
-            String cubeName = (String) opt.params[0];
-            String cubeResPath = CubeInstance.concatResourcePath(cubeName);
-            Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
-            CubeInstance cube = srcStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
-            cube.getSegments().clear();
-            cube.setStatus(RealizationStatusEnum.DISABLED);
-            srcStore.putResource(cubeResPath, cube, cubeSerializer);
-            logger.info("Cube " + cubeName + " is purged and disabled in " + srcConfig.getMetadataUrl());
-
-            break;
-        }
-        default: {
-            //do nothing
-            break;
-        }
         }
     }
 
@@ -492,35 +497,35 @@ public class CubeMigrationCLI {
         logger.info("Undo operation: " + opt.toString());
 
         switch (opt.type) {
-        case CHANGE_HTABLE_HOST: {
-            String tableName = (String) opt.params[0];
-            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-            hbaseAdmin.disableTable(tableName);
-            desc.setValue(IRealizationConstants.HTableTag, srcConfig.getMetadataUrlPrefix());
-            hbaseAdmin.modifyTable(tableName, desc);
-            hbaseAdmin.enableTable(tableName);
-            break;
-        }
-        case COPY_FILE_IN_META: {
-            // no harm
-            logger.info("Undo for COPY_FILE_IN_META is ignored");
-            break;
-        }
-        case COPY_DICT_OR_SNAPSHOT: {
-            // no harm
-            logger.info("Undo for COPY_DICT_OR_SNAPSHOT is ignored");
-            break;
-        }
-        case RENAME_FOLDER_IN_HDFS: {
-            String srcPath = (String) opt.params[1];
-            String dstPath = (String) opt.params[0];
+            case CHANGE_HTABLE_HOST: {
+                String tableName = (String) opt.params[0];
+                HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+                hbaseAdmin.disableTable(TableName.valueOf(tableName));
+                desc.setValue(IRealizationConstants.HTableTag, srcConfig.getMetadataUrlPrefix());
+                hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+                hbaseAdmin.enableTable(TableName.valueOf(tableName));
+                break;
+            }
+            case COPY_FILE_IN_META: {
+                // no harm
+                logger.info("Undo for COPY_FILE_IN_META is ignored");
+                break;
+            }
+            case COPY_DICT_OR_SNAPSHOT: {
+                // no harm
+                logger.info("Undo for COPY_DICT_OR_SNAPSHOT is ignored");
+                break;
+            }
+            case RENAME_FOLDER_IN_HDFS: {
+                String srcPath = (String) opt.params[1];
+                String dstPath = (String) opt.params[0];
 
             if (hdfsFS.exists(new Path(srcPath)) && !hdfsFS.exists(new Path(dstPath))) {
                 hdfsFS.rename(new Path(srcPath), new Path(dstPath));
                 logger.info("HDFS Folder renamed from " + srcPath + " to " + dstPath);
             }
             break;
-        }
+             }
         case ADD_INTO_PROJECT: {
             logger.info("Undo for ADD_INTO_PROJECT is ignored");
             break;

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
index 295750a..5c152f4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
@@ -29,7 +29,9 @@ import org.apache.commons.cli.ParseException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.cube.CubeInstance;
@@ -60,7 +62,7 @@ public class CubeMigrationCheckCLI {
     private static final Option OPTION_CUBE = OptionBuilder.withArgName("cube").hasArg().isRequired(false).withDescription("The name of cube migrated").create("cube");
 
     private KylinConfig dstCfg;
-    private HBaseAdmin hbaseAdmin;
+    private Admin hbaseAdmin;
 
     private List<String> issueExistHTables;
     private List<String> inconsistentHTables;
@@ -128,9 +130,8 @@ public class CubeMigrationCheckCLI {
         this.dstCfg = kylinConfig;
         this.ifFix = isFix;
 
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        hbaseAdmin = new HBaseAdmin(conf);
-
+        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+        hbaseAdmin = conn.getAdmin();
         issueExistHTables = Lists.newArrayList();
         inconsistentHTables = Lists.newArrayList();
     }
@@ -186,11 +187,11 @@ public class CubeMigrationCheckCLI {
             for (String segFullName : inconsistentHTables) {
                 String[] sepNameList = segFullName.split(",");
                 HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(sepNameList[0]));
-                logger.info("Change the host of htable " + sepNameList[0] + "belonging to cube " + sepNameList[1] + " from " + desc.getValue(IRealizationConstants.HTableTag) + " to " + dstCfg.getMetadataUrlPrefix());
-                hbaseAdmin.disableTable(sepNameList[0]);
+                logger.info("Change the host of htable "+sepNameList[0]+"belonging to cube "+sepNameList[1]+" from "+desc.getValue(IRealizationConstants.HTableTag)+" to "+dstCfg.getMetadataUrlPrefix());
+                hbaseAdmin.disableTable(TableName.valueOf(sepNameList[0]));
                 desc.setValue(IRealizationConstants.HTableTag, dstCfg.getMetadataUrlPrefix());
-                hbaseAdmin.modifyTable(sepNameList[0], desc);
-                hbaseAdmin.enableTable(sepNameList[0]);
+                hbaseAdmin.modifyTable(TableName.valueOf(sepNameList[0]), desc);
+                hbaseAdmin.enableTable(TableName.valueOf(sepNameList[0]));
             }
         } else {
             logger.info("------ Inconsistent HTables Needed To Be Fixed ------");

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
index 63e8ac1..069115e 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
@@ -30,7 +30,9 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.KylinVersion;
@@ -70,7 +72,8 @@ public class DeployCoprocessorCLI {
         KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
+        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
 
         String localCoprocessorJar;
         if ("default".equals(args[0])) {
@@ -153,10 +156,10 @@ public class DeployCoprocessorCLI {
     public static void deployCoprocessor(HTableDescriptor tableDesc) {
         try {
             initHTableCoprocessor(tableDesc);
-            logger.info("hbase table " + tableDesc.getName() + " deployed with coprocessor.");
+            logger.info("hbase table " + tableDesc.getTableName() + " deployed with coprocessor.");
 
         } catch (Exception ex) {
-            logger.error("Error deploying coprocessor on " + tableDesc.getName(), ex);
+            logger.error("Error deploying coprocessor on " + tableDesc.getTableName(), ex);
             logger.error("Will try creating the table without coprocessor.");
         }
     }
@@ -178,9 +181,9 @@ public class DeployCoprocessorCLI {
         desc.addCoprocessor(CubeObserverClass, hdfsCoprocessorJar, 1002, null);
     }
 
-    public static void resetCoprocessor(String tableName, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
+    public static void resetCoprocessor(String tableName, Admin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
         logger.info("Disable " + tableName);
-        hbaseAdmin.disableTable(tableName);
+        hbaseAdmin.disableTable(TableName.valueOf(tableName));
 
         logger.info("Unset coprocessor on " + tableName);
         HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
@@ -208,13 +211,13 @@ public class DeployCoprocessorCLI {
             desc.setValue(IRealizationConstants.HTableGitTag, commitInfo);
         }
 
-        hbaseAdmin.modifyTable(tableName, desc);
+        hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
 
         logger.info("Enable " + tableName);
-        hbaseAdmin.enableTable(tableName);
+        hbaseAdmin.enableTable(TableName.valueOf(tableName));
     }
 
-    private static List<String> resetCoprocessorOnHTables(HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
+    private static List<String> resetCoprocessorOnHTables(Admin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
         List<String> processed = new ArrayList<String>();
 
         for (String tableName : tableNames) {
@@ -325,7 +328,7 @@ public class DeployCoprocessorCLI {
         return coprocessorDir;
     }
 
-    private static Set<String> getCoprocessorJarPaths(HBaseAdmin hbaseAdmin, List<String> tableNames) throws IOException {
+    private static Set<String> getCoprocessorJarPaths(Admin hbaseAdmin, List<String> tableNames) throws IOException {
         HashSet<String> result = new HashSet<String>();
 
         for (String tableName : tableNames) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
index a5a85fa..d830276 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
@@ -26,8 +26,9 @@ import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.kylin.common.KylinConfig;
@@ -231,9 +232,9 @@ public class ExtendCubeToHybridCLI {
         Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
         ProjectInstance project = store.getResource(projectResPath, ProjectInstance.class, projectSerializer);
         String projUUID = project.getUuid();
-        HTableInterface aclHtable = null;
+        Table aclHtable = null;
         try {
-            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(kylinConfig.getMetadataUrlPrefix() + "_acl");
+            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(TableName.valueOf(kylinConfig.getMetadataUrlPrefix() + "_acl"));
 
             // cube acl
             Result result = aclHtable.get(new Get(Bytes.toBytes(origCubeId)));
@@ -253,7 +254,6 @@ public class ExtendCubeToHybridCLI {
                     aclHtable.put(put);
                 }
             }
-            aclHtable.flushCommits();
         } finally {
             IOUtils.closeQuietly(aclHtable);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
index 86ba22f..8387792 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
@@ -28,9 +28,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -75,7 +75,7 @@ public class GridTableHBaseBenchmark {
         System.out.println("Testing grid table scanning, hit ratio " + hitRatio + ", index ratio " + indexRatio);
         String hbaseUrl = "hbase"; // use hbase-site.xml on classpath
 
-        HConnection conn = HBaseConnection.get(hbaseUrl);
+        Connection conn = HBaseConnection.get(hbaseUrl);
         createHTableIfNeeded(conn, TEST_TABLE);
         prepareData(conn);
 
@@ -91,10 +91,10 @@ public class GridTableHBaseBenchmark {
 
     }
 
-    private static void testColumnScan(HConnection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
+    private static void testColumnScan(Connection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
         Stats stats = new Stats("COLUMN_SCAN");
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -122,20 +122,20 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void testRowScanNoIndexFullScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexFullScan(Connection conn, boolean[] hits) throws IOException {
         fullScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_FULL"));
     }
 
-    private static void testRowScanNoIndexSkipScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexSkipScan(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_SKIP"));
     }
 
-    private static void testRowScanWithIndex(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanWithIndex(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_IDX"));
     }
 
-    private static void fullScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void fullScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -156,11 +156,11 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void jumpScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
+    private static void jumpScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
 
         final int jumpThreshold = 6; // compensate for Scan() overhead, totally by experience
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
 
             stats.markStart();
@@ -204,8 +204,8 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void prepareData(HConnection conn) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void prepareData(Connection conn) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
 
         try {
             // check how many rows existing
@@ -258,8 +258,8 @@ public class GridTableHBaseBenchmark {
         return bytes;
     }
 
-    private static void createHTableIfNeeded(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    private static void createHTableIfNeeded(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
 
         try {
             boolean tableExist = false;

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
index b6958d6..ad4884e 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
@@ -27,8 +27,12 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.storage.hbase.HBaseConnection;
@@ -79,8 +83,8 @@ public class HBaseClean extends AbstractHadoopJob {
     private void cleanUp() {
         try {
             // get all kylin hbase tables
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            Admin hbaseAdmin = conn.getAdmin();
             String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
             HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
             List<String> allTablesNeedToBeDropped = Lists.newArrayList();
@@ -95,12 +99,12 @@ public class HBaseClean extends AbstractHadoopJob {
                 // drop tables
                 for (String htableName : allTablesNeedToBeDropped) {
                     logger.info("Deleting HBase table " + htableName);
-                    if (hbaseAdmin.tableExists(htableName)) {
-                        if (hbaseAdmin.isTableEnabled(htableName)) {
-                            hbaseAdmin.disableTable(htableName);
+                    if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+                        if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
+                            hbaseAdmin.disableTable(TableName.valueOf(htableName));
                         }
 
-                        hbaseAdmin.deleteTable(htableName);
+                        hbaseAdmin.deleteTable(TableName.valueOf(htableName));
                         logger.info("Deleted HBase table " + htableName);
                     } else {
                         logger.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
index 346c3a2..58aa8fd 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
@@ -20,22 +20,11 @@
 package org.apache.kylin.storage.hbase.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 java.util.*;
 
 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.*;
+import org.apache.hadoop.hbase.client.*;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.kylin.common.util.Pair;
 import org.slf4j.Logger;
@@ -57,30 +46,31 @@ public class HBaseRegionSizeCalculator {
     /**
      * Computes size of each region for table and given column families.
      * */
-    public HBaseRegionSizeCalculator(HTable table) throws IOException {
-        this(table, new HBaseAdmin(table.getConfiguration()));
-    }
-
-    /** Constructor for unit testing */
-    HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
+    public HBaseRegionSizeCalculator(String tableName, Connection hbaseConnection) throws IOException {
 
+        Table table = null;
+        Admin admin = null;
         try {
+            table = hbaseConnection.getTable(TableName.valueOf(tableName));
+            admin = hbaseConnection.getAdmin();
+
             if (!enabled(table.getConfiguration())) {
                 logger.info("Region size calculation disabled.");
                 return;
             }
 
-            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
+            logger.info("Calculating region sizes for table \"" + table.getName() + "\".");
 
             // Get regions for table.
-            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
+            RegionLocator regionLocator = hbaseConnection.getRegionLocator(table.getName());
+            List<HRegionLocation> regionLocationList = regionLocator.getAllRegionLocations();
             Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
 
-            for (HRegionInfo regionInfo : tableRegionInfos) {
-                tableRegions.add(regionInfo.getRegionName());
+             for (HRegionLocation hRegionLocation : regionLocationList) {
+                 tableRegions.add(hRegionLocation.getRegionInfo().getRegionName());
             }
 
-            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
+            ClusterStatus clusterStatus = admin.getClusterStatus();
             Collection<ServerName> servers = clusterStatus.getServers();
             final long megaByte = 1024L * 1024L;
 
@@ -104,7 +94,7 @@ public class HBaseRegionSizeCalculator {
                 }
             }
         } finally {
-            hBaseAdmin.close();
+            admin.close();
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
index 266f7e7..1351492 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
@@ -25,12 +25,16 @@ import java.util.Map;
 import org.apache.commons.lang.StringUtils;
 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.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 
 public class HBaseUsage {
 
@@ -42,8 +46,8 @@ public class HBaseUsage {
         Map<String, List<String>> envs = Maps.newHashMap();
 
         // get all kylin hbase tables
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         for (HTableDescriptor desc : tableDescriptors) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
index f30f2c9..1257932 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
@@ -31,15 +31,16 @@ import org.apache.hadoop.hbase.Cell;
 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.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
@@ -57,11 +58,11 @@ public class HbaseStreamingInput {
     private static final byte[] QN = "C".getBytes();
 
     public static void createTable(String tableName) throws IOException {
-        HConnection conn = getConnection();
-        HBaseAdmin hadmin = new HBaseAdmin(conn);
+        Connection conn = getConnection();
+        Admin hadmin = conn.getAdmin();
 
         try {
-            boolean tableExist = hadmin.tableExists(tableName);
+            boolean tableExist = hadmin.tableExists(TableName.valueOf(tableName));
             if (tableExist) {
                 logger.info("HTable '" + tableName + "' already exists");
                 return;
@@ -118,8 +119,8 @@ public class HbaseStreamingInput {
                 e.printStackTrace();
             }
 
-            HConnection conn = getConnection();
-            HTableInterface table = conn.getTable(tableName);
+            Connection conn = getConnection();
+            Table table = conn.getTable(TableName.valueOf(tableName));
 
             byte[] key = new byte[8 + 4];//time + id
 
@@ -134,7 +135,7 @@ public class HbaseStreamingInput {
                 Bytes.putInt(key, 8, i);
                 Put put = new Put(key);
                 byte[] cell = randomBytes(CELL_SIZE);
-                put.add(CF, QN, cell);
+                put.addColumn(CF, QN, cell);
                 buffer.add(put);
             }
             table.put(buffer);
@@ -169,8 +170,8 @@ public class HbaseStreamingInput {
             }
 
             Random r = new Random();
-            HConnection conn = getConnection();
-            HTableInterface table = conn.getTable(tableName);
+            Connection conn = getConnection();
+            Table table = conn.getTable(TableName.valueOf(tableName));
 
             long leftBound = getFirstKeyTime(table);
             long rightBound = System.currentTimeMillis();
@@ -205,7 +206,7 @@ public class HbaseStreamingInput {
         }
     }
 
-    private static long getFirstKeyTime(HTableInterface table) throws IOException {
+    private static long getFirstKeyTime(Table table) throws IOException {
         long startTime = 0;
 
         Scan scan = new Scan();
@@ -223,8 +224,8 @@ public class HbaseStreamingInput {
 
     }
 
-    private static HConnection getConnection() throws IOException {
-        return HConnectionManager.createConnection(HBaseConnection.getCurrentHBaseConfiguration());
+    private static Connection getConnection() throws IOException {
+        return HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
     }
 
     private static String formatTime(long time) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
index 4db183b..7120c3f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
@@ -26,8 +26,11 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
@@ -70,8 +73,8 @@ public class HtableAlterMetadataCLI extends AbstractHadoopJob {
     }
 
     private void alter() throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         HTableDescriptor table = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
 
         hbaseAdmin.disableTable(table.getTableName());

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
index f0618c9..b7303a5 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
@@ -31,10 +31,15 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -86,7 +91,8 @@ public class OrphanHBaseCleanJob extends AbstractHadoopJob {
     private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
 
         // get all kylin hbase tables
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -105,12 +111,12 @@ public class OrphanHBaseCleanJob extends AbstractHadoopJob {
             // drop tables
             for (String htableName : allTablesNeedToBeDropped) {
                 logger.info("Deleting HBase table " + htableName);
-                if (hbaseAdmin.tableExists(htableName)) {
-                    if (hbaseAdmin.isTableEnabled(htableName)) {
-                        hbaseAdmin.disableTable(htableName);
+                if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+                    if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
+                        hbaseAdmin.disableTable(TableName.valueOf(htableName));
                     }
 
-                    hbaseAdmin.deleteTable(htableName);
+                    hbaseAdmin.deleteTable(TableName.valueOf(htableName));
                     logger.info("Deleted HBase table " + htableName);
                 } else {
                     logger.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
index 58ef7cb..b86b561 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
@@ -21,9 +21,10 @@ package org.apache.kylin.storage.hbase.util;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -57,12 +58,12 @@ public class PingHBaseCLI {
         Scan scan = new Scan();
         int limit = 20;
 
-        HConnection conn = null;
-        HTableInterface table = null;
+        Connection conn = null;
+        Table table = null;
         ResultScanner scanner = null;
         try {
-            conn = HConnectionManager.createConnection(hconf);
-            table = conn.getTable(hbaseTable);
+            conn = ConnectionFactory.createConnection(hconf);
+            table = conn.getTable(TableName.valueOf(hbaseTable));
             scanner = table.getScanner(scan);
             int count = 0;
             for (Result r : scanner) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
index 01edb1f..a854973 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
@@ -22,9 +22,10 @@ import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.kylin.common.util.Bytes;
@@ -70,8 +71,8 @@ public class RowCounterCLI {
 
         logger.info("My Scan " + scan.toString());
 
-        HConnection conn = HConnectionManager.createConnection(conf);
-        HTableInterface tableInterface = conn.getTable(htableName);
+        Connection conn = ConnectionFactory.createConnection(conf);
+        Table tableInterface = conn.getTable(TableName.valueOf(htableName));
 
         Iterator<Result> iterator = tableInterface.getScanner(scan).iterator();
         int counter = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
index 3f4a6d5..d7f49df 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
@@ -41,7 +41,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.CliCommandExecutor;
@@ -55,6 +57,7 @@ import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -106,7 +109,8 @@ public class StorageCleanupJob extends AbstractHadoopJob {
         CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
         long TIME_THREADSHOLD = KylinConfig.getInstanceFromEnv().getStorageCleanupTimeThreshold();
         // get all kylin hbase tables
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -173,12 +177,12 @@ public class StorageCleanupJob extends AbstractHadoopJob {
 
         public Object call() throws Exception {
             logger.info("Deleting HBase table " + htableName);
-            if (hbaseAdmin.tableExists(htableName)) {
-                if (hbaseAdmin.isTableEnabled(htableName)) {
-                    hbaseAdmin.disableTable(htableName);
+            if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+                if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
+                    hbaseAdmin.disableTable(TableName.valueOf(htableName));
                 }
 
-                hbaseAdmin.deleteTable(htableName);
+                hbaseAdmin.deleteTable(TableName.valueOf(htableName));
                 logger.info("Deleted HBase table " + htableName);
             } else {
                 logger.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
index b1f275f..9687e25 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
@@ -26,9 +26,12 @@ import java.util.List;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -36,7 +39,6 @@ import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
-import org.apache.kylin.storage.hbase.HBaseConnection;
 
 import com.google.common.collect.Lists;
 
@@ -49,14 +51,15 @@ public class UpdateHTableHostCLI {
     private List<String> errorMsgs = Lists.newArrayList();
 
     private List<String> htables;
-    private HBaseAdmin hbaseAdmin;
+    private Admin hbaseAdmin;
     private KylinConfig kylinConfig;
     private String oldHostValue;
 
     public UpdateHTableHostCLI(List<String> htables, String oldHostValue) throws IOException {
         this.htables = htables;
         this.oldHostValue = oldHostValue;
-        this.hbaseAdmin = new HBaseAdmin(HBaseConnection.getCurrentHBaseConfiguration());
+        Connection conn = ConnectionFactory.createConnection(HBaseConfiguration.create());
+        hbaseAdmin = conn.getAdmin();
         this.kylinConfig = KylinConfig.getInstanceFromEnv();
     }
 
@@ -166,9 +169,9 @@ public class UpdateHTableHostCLI {
         HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
         if (oldHostValue.equals(desc.getValue(IRealizationConstants.HTableTag))) {
             desc.setValue(IRealizationConstants.HTableTag, kylinConfig.getMetadataUrlPrefix());
-            hbaseAdmin.disableTable(tableName);
-            hbaseAdmin.modifyTable(tableName, desc);
-            hbaseAdmin.enableTable(tableName);
+            hbaseAdmin.disableTable(TableName.valueOf(tableName));
+            hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+            hbaseAdmin.enableTable(TableName.valueOf(tableName));
 
             updatedResources.add(tableName);
         }


[50/55] [abbrv] kylin git commit: update version number

Posted by sh...@apache.org.
update version number


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 9bac8386b7cbff36bae78d4e458a276fc7d0bf42
Parents: 4cdc07e
Author: shaofengshi <sh...@apache.org>
Authored: Sat Jun 4 08:24:38 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 25 20:13:38 2016 +0800

----------------------------------------------------------------------
 core-metadata/pom.xml | 2 +-
 jdbc/pom.xml          | 4 ++++
 server/pom.xml        | 4 ++++
 3 files changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/9bac8386/core-metadata/pom.xml
----------------------------------------------------------------------
diff --git a/core-metadata/pom.xml b/core-metadata/pom.xml
index 1a2c5e8..3c1ee0b 100644
--- a/core-metadata/pom.xml
+++ b/core-metadata/pom.xml
@@ -28,7 +28,7 @@
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
         <version>1.5.3</version>
-        
+
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/kylin/blob/9bac8386/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index e86998e..ff83568 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -27,7 +27,11 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
+<<<<<<< 571874049bdb9139afdc467474b53a1d63be818b
         <version>1.5.3</version>
+=======
+        <version>1.5.2.1</version>
+>>>>>>> update version number
 
     </parent>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/9bac8386/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index bca91fb..e0a4110 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -28,7 +28,11 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
+<<<<<<< 571874049bdb9139afdc467474b53a1d63be818b
         <version>1.5.3</version>
+=======
+        <version>1.5.2.1</version>
+>>>>>>> update version number
 
     </parent>
 


[12/55] [abbrv] kylin git commit: KYLIN-1862 'table not found' in 'Build Dimension Dictionary' step

Posted by sh...@apache.org.
KYLIN-1862 'table not found' in 'Build Dimension Dictionary' step


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: b45dc44cdb5dfedbce69a903e620726290c27a5c
Parents: b999eac
Author: shaofengshi <sh...@apache.org>
Authored: Sat Jul 9 09:50:29 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sat Jul 9 09:50:29 2016 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/dict/DictionaryManager.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b45dc44c/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 bd00f10..ff088b9 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
@@ -293,7 +293,7 @@ public class DictionaryManager {
                 TableDesc materializedTbl = new TableDesc();
                 materializedTbl.setDatabase(config.getHiveDatabaseForIntermediateTable());
                 materializedTbl.setName(tableDesc.getMaterializedName());
-                inpTable = SourceFactory.createReadableTable(tableDesc);
+                inpTable = SourceFactory.createReadableTable(materializedTbl);
             } else {
                 inpTable = SourceFactory.createReadableTable(tableDesc);
             }


[05/55] [abbrv] kylin git commit: KYLIN-1858 remove all ii related code

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
deleted file mode 100644
index dfaf106..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
+++ /dev/null
@@ -1,235 +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.model;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.Array;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.dict.DictionarySerializer;
-import org.apache.kylin.invertedindex.index.ColumnValueContainer;
-import org.apache.kylin.invertedindex.index.CompressedValueContainer;
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
-import org.apache.kylin.metadata.datatype.DataType;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-/**
- * @author yangli9
- */
-public class IIKeyValueCodec implements KeyValueCodec {
-
-    public static final int SHARD_LEN = 2;
-    public static final int TIMEPART_LEN = 8;
-    public static final int COLNO_LEN = 2;
-    protected final TableRecordInfoDigest digest;
-
-    public IIKeyValueCodec(TableRecordInfoDigest digest) {
-        this.digest = digest;
-    }
-
-    @Override
-    public Collection<IIRow> encodeKeyValue(Slice slice) {
-        ArrayList<IIRow> result = Lists.newArrayList();
-        ColumnValueContainer[] containers = slice.getColumnValueContainers();
-        for (int col = 0; col < containers.length; col++) {
-            if (containers[col] instanceof CompressedValueContainer) {
-                final IIRow row = collectKeyValues(slice, col, (CompressedValueContainer) containers[col]);
-                result.add(row);
-            } else {
-                throw new IllegalArgumentException("Unknown container class " + containers[col].getClass());
-            }
-        }
-        return result;
-    }
-
-    private IIRow collectKeyValues(Slice slice, int col, CompressedValueContainer container) {
-        ImmutableBytesWritable key = encodeKey(slice.getShard(), slice.getTimestamp(), col);
-        ImmutableBytesWritable value = container.toBytes();
-        final Dictionary<?> dictionary = slice.getLocalDictionaries() != null ? slice.getLocalDictionaries()[col] : null;
-        if (dictionary == null) {
-            return new IIRow(key, value, new ImmutableBytesWritable(BytesUtil.EMPTY_BYTE_ARRAY));
-        } else {
-            ByteArray bytes = DictionarySerializer.serialize(dictionary);
-            return new IIRow(key, value, new ImmutableBytesWritable(bytes.array(), bytes.offset(), bytes.length()));
-        }
-    }
-
-    ImmutableBytesWritable encodeKey(short shard, long timestamp, int col) {
-        byte[] bytes = new byte[20];
-        int len = encodeKey(shard, timestamp, col, bytes, 0);
-        return new ImmutableBytesWritable(bytes, 0, len);
-    }
-
-    int encodeKey(short shard, long timestamp, int col, byte[] buf, int offset) {
-        int i = offset;
-
-        BytesUtil.writeUnsigned(shard, buf, i, SHARD_LEN);
-        i += SHARD_LEN;
-        BytesUtil.writeLong(timestamp, buf, i, TIMEPART_LEN);
-        i += TIMEPART_LEN;
-
-        BytesUtil.writeUnsigned(col, buf, i, COLNO_LEN);
-        i += COLNO_LEN;
-
-        return i - offset;
-    }
-
-    @Override
-    public Iterable<Slice> decodeKeyValue(Iterable<IIRow> kvs) {
-        return new IIRowDecoder(digest, kvs.iterator());
-        //return new Decoder(kvs, incompleteDigest);
-    }
-
-    private static TableRecordInfoDigest createDigest(int nColumns, boolean[] isMetric, String[] dataTypes, Dictionary<?>[] dictionaries) {
-        int[] dictMaxIds = new int[nColumns];
-        int[] lengths = new int[nColumns];
-        final boolean emptyDictionary = Array.isEmpty(dictionaries);
-        for (int i = 0; i < nColumns; ++i) {
-            if (isMetric[i]) {
-                final FixedLenMeasureCodec<?> fixedLenMeasureCodec = FixedLenMeasureCodec.get(DataType.getType(dataTypes[i]));
-                lengths[i] = fixedLenMeasureCodec.getLength();
-            } else {
-                if (emptyDictionary) {
-                    final DataType dataType = DataType.getType(dataTypes[i]);
-                    if (dataType.isNumberFamily()) {
-                        lengths[i] = 16;
-                    } else if (dataType.isStringFamily()) {
-                        lengths[i] = 256;
-                    } else if (dataType.isDateTimeFamily()) {
-                        lengths[i] = 19;
-                    } else {
-                        throw new RuntimeException("invalid data type:" + dataType);
-                    }
-                    dictMaxIds[i] = Integer.MAX_VALUE;
-                } else {
-                    final Dictionary<?> dictionary = dictionaries[i];
-                    lengths[i] = dictionary.getSizeOfId();
-                    dictMaxIds[i] = dictionary.getMaxId();
-                }
-            }
-        }
-        // offsets
-        int pos = 0;
-        int[] offsets = new int[nColumns];
-        for (int i = 0; i < nColumns; i++) {
-            offsets[i] = pos;
-            pos += lengths[i];
-        }
-
-        int byteFormLen = pos;
-
-        return new TableRecordInfoDigest(nColumns, byteFormLen, offsets, dictMaxIds, lengths, isMetric, dataTypes);
-    }
-
-    protected static class IIRowDecoder implements Iterable<Slice> {
-
-        protected final TableRecordInfoDigest incompleteDigest;
-        protected final Iterator<IIRow> iiRowIterator;
-        protected Iterator<IIRow> feedingIterator;//this is for extending
-
-        protected IIRowDecoder(TableRecordInfoDigest digest, Iterator<IIRow> iiRowIterator) {
-            this.incompleteDigest = digest;
-            this.iiRowIterator = iiRowIterator;
-            this.feedingIterator = this.iiRowIterator;
-        }
-
-        @Override
-        public Iterator<Slice> iterator() {
-            return new Iterator<Slice>() {
-                @Override
-                public boolean hasNext() {
-                    return iiRowIterator.hasNext();
-                }
-
-                @Override
-                public Slice next() {
-                    int columns = 0;
-                    ColumnValueContainer[] valueContainers = new ColumnValueContainer[incompleteDigest.getColumnCount()];
-                    Dictionary<?>[] localDictionaries = new Dictionary<?>[incompleteDigest.getColumnCount()];
-                    boolean firstTime = true;
-                    short curShard = 0;
-                    long curTimestamp = 0;
-                    short lastShard = 0;
-                    long lastTimestamp = 0;
-
-                    while (feedingIterator.hasNext() && columns < incompleteDigest.getColumnCount()) {
-                        final IIRow row = feedingIterator.next();
-                        final ImmutableBytesWritable key = row.getKey();
-                        int i = key.getOffset();
-                        curShard = (short) BytesUtil.readUnsigned(key.get(), i, SHARD_LEN);
-                        i += SHARD_LEN;
-                        curTimestamp = BytesUtil.readLong(key.get(), i, TIMEPART_LEN);
-                        i += TIMEPART_LEN;
-
-                        if (!firstTime) {
-                            Preconditions.checkArgument(curShard == lastShard, "shard should be equals in one slice, curShard is" + curShard + " lastShard is " + lastShard);
-                            Preconditions.checkArgument(curTimestamp == lastTimestamp, "timestamp should be equals in one slice, curTimestamp is" + curTimestamp + " lastTimestamp is " + lastTimestamp);
-                        }
-
-                        int curCol = BytesUtil.readUnsigned(key.get(), i, COLNO_LEN);
-                        if (incompleteDigest.isMetrics(curCol)) {
-                            CompressedValueContainer c = new CompressedValueContainer(incompleteDigest, curCol, 0);
-                            c.fromBytes(row.getValue());
-                            valueContainers[curCol] = c;
-                        } else {
-                            final ImmutableBytesWritable dictBytes = row.getDictionary();
-                            if (dictBytes.getLength() != 0) {
-                                final Dictionary<?> dictionary = DictionarySerializer.deserialize(new ByteArray(dictBytes.get(), dictBytes.getOffset(), dictBytes.getLength()));
-                                CompressedValueContainer c = new CompressedValueContainer(dictionary.getSizeOfId(), dictionary.getMaxId() - dictionary.getMinId() + 1, 0);
-                                c.fromBytes(row.getValue());
-                                valueContainers[curCol] = c;
-                                localDictionaries[curCol] = dictionary;
-                            } else {
-                                CompressedValueContainer c = new CompressedValueContainer(incompleteDigest.length(curCol), incompleteDigest.getMaxID(curCol) - 0 + 1, 0);
-                                c.fromBytes(row.getValue());
-                                valueContainers[curCol] = c;
-                            }
-                        }
-                        columns++;
-                        lastShard = curShard;
-                        lastTimestamp = curTimestamp;
-                        firstTime = false;
-                    }
-                    Preconditions.checkArgument(columns == incompleteDigest.getColumnCount(), "column count is " + columns + " should be equals to incompleteDigest.getColumnCount() " + incompleteDigest.getColumnCount());
-
-                    TableRecordInfoDigest digest = createDigest(columns, incompleteDigest.getIsMetric(), incompleteDigest.getMetricDataTypes(), localDictionaries);
-                    Slice slice = new Slice(digest, curShard, curTimestamp, valueContainers);
-                    slice.setLocalDictionaries(localDictionaries);
-                    return slice;
-                }
-
-                @Override
-                public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-            };
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodecWithState.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodecWithState.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodecWithState.java
deleted file mode 100644
index 8747916..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodecWithState.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.invertedindex.model;
-
-import java.util.Iterator;
-import java.util.LinkedList;
-
-import org.apache.kylin.common.util.FIFOIterable;
-import org.apache.kylin.common.util.FIFOIterator;
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-/**
- */
-public class IIKeyValueCodecWithState extends IIKeyValueCodec {
-
-    public IIKeyValueCodecWithState(TableRecordInfoDigest digest) {
-        super(digest);
-    }
-
-    /**
-     * 
-     * @param kvs kvs must be a {@link org.apache.kylin.common.util.FIFOIterable } to avoid {@link java.util.ConcurrentModificationException}.
-     * @return
-     */
-    @Override
-    public Iterable<Slice> decodeKeyValue(Iterable<IIRow> kvs) {
-        if (!(kvs instanceof FIFOIterable)) {
-            throw new IllegalArgumentException("kvs must be a {@link org.apache.kylin.common.util.FIFOIterable } to avoid {@link java.util.ConcurrentModificationException}.");
-        }
-        return new IIRowDecoderWithState(digest, kvs.iterator());
-    }
-
-    //TODO refactor this class, does not have to extend IIKeyValueCodec, composite might be enough, mhb
-    protected static class IIRowDecoderWithState extends IIRowDecoder {
-
-        final LinkedList<IIRow> buffer = Lists.newLinkedList();
-        private Iterator<Slice> superIterator = null;
-
-        private IIRowDecoderWithState(TableRecordInfoDigest digest, Iterator<IIRow> iiRowIterator) {
-            super(digest, iiRowIterator);
-            this.feedingIterator = new FIFOIterator<>(buffer);
-        }
-
-        private Iterator<Slice> getSuperIterator() {
-            if (superIterator == null) {
-                superIterator = super.iterator();
-            }
-            return superIterator;
-        }
-
-        @Override
-        public Iterator<Slice> iterator() {
-            return new Iterator<Slice>() {
-                @Override
-                public boolean hasNext() {
-                    while (buffer.size() < incompleteDigest.getColumnCount() && iiRowIterator.hasNext()) {
-                        buffer.add(iiRowIterator.next());
-                    }
-                    return buffer.size() == incompleteDigest.getColumnCount();
-                }
-
-                @Override
-                public Slice next() {
-                    while (buffer.size() < incompleteDigest.getColumnCount() && iiRowIterator.hasNext()) {
-                        buffer.add(iiRowIterator.next());
-                    }
-                    Preconditions.checkArgument(buffer.size() == incompleteDigest.getColumnCount(), "not enough IIRows!");
-                    Slice ret = IIRowDecoderWithState.this.getSuperIterator().next();
-                    buffer.clear();
-                    return ret;
-                }
-
-                @Override
-                public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-            };
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIRow.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIRow.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIRow.java
deleted file mode 100644
index 9d13487..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIRow.java
+++ /dev/null
@@ -1,90 +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.model;
-
-import java.util.List;
-
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.BytesUtil;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-public final class IIRow {
-
-    private final ImmutableBytesWritable key;
-    private final ImmutableBytesWritable value;
-    private final ImmutableBytesWritable dictionary;
-
-    public IIRow(ImmutableBytesWritable key, ImmutableBytesWritable value, ImmutableBytesWritable dictionary) {
-        this.key = key;
-        this.value = value;
-        this.dictionary = dictionary;
-    }
-
-    public IIRow() {
-        this(new ImmutableBytesWritable(), new ImmutableBytesWritable(), new ImmutableBytesWritable());
-    }
-
-    public ImmutableBytesWritable getKey() {
-        return key;
-    }
-
-    public ImmutableBytesWritable getValue() {
-        return value;
-    }
-
-    public ImmutableBytesWritable getDictionary() {
-        return dictionary;
-    }
-
-    public void updateWith(Cell c) {
-        if (BytesUtil.compareBytes(IIDesc.HBASE_QUALIFIER_BYTES, 0, c.getQualifierArray(), c.getQualifierOffset(), IIDesc.HBASE_QUALIFIER_BYTES.length) == 0) {
-            this.getKey().set(c.getRowArray(), c.getRowOffset(), c.getRowLength());
-            this.getValue().set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-        } else if (BytesUtil.compareBytes(IIDesc.HBASE_DICTIONARY_BYTES, 0, c.getQualifierArray(), c.getQualifierOffset(), IIDesc.HBASE_DICTIONARY_BYTES.length) == 0) {
-            this.getDictionary().set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-        }
-    }
-
-    public List<Cell> makeCells() {
-        Cell a = new KeyValue(this.getKey().copyBytes(), IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES, this.getValue().copyBytes());
-        Cell b = new KeyValue(this.getKey().copyBytes(), IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_DICTIONARY_BYTES, this.getDictionary().copyBytes());
-        return Lists.newArrayList(a, b);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/KeyValueCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/KeyValueCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/KeyValueCodec.java
deleted file mode 100644
index 7843ee7..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/KeyValueCodec.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.invertedindex.model;
-
-import java.util.Collection;
-
-import org.apache.kylin.invertedindex.index.Slice;
-
-/**
- */
-public interface KeyValueCodec {
-
-    Collection<IIRow> encodeKeyValue(Slice slice);
-
-    Iterable<Slice> decodeKeyValue(Iterable<IIRow> kvs);
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/util/IIDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/util/IIDictionaryBuilder.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/util/IIDictionaryBuilder.java
deleted file mode 100644
index 36a8781..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/util/IIDictionaryBuilder.java
+++ /dev/null
@@ -1,86 +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.util;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-
-import javax.annotation.Nullable;
-
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.dict.DictionaryGenerator;
-import org.apache.kylin.dict.IterableDictionaryValueEnumerator;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.HashMultimap;
-
-/**
- */
-public final class IIDictionaryBuilder {
-
-    private IIDictionaryBuilder() {
-    }
-
-    public static Dictionary<?>[] buildDictionary(List<List<String>> table, IIDesc desc) throws IOException {
-        HashMultimap<TblColRef, String> valueMap = HashMultimap.create();
-        final List<TblColRef> allColumns = desc.listAllColumns();
-        for (List<String> row : table) {
-            for (int i = 0; i < row.size(); i++) {
-                String cell = row.get(i);
-                if (!desc.isMetricsCol(i)) {
-                    valueMap.put(allColumns.get(i), cell);
-                }
-            }
-        }
-
-        Dictionary<?>[] result = new Dictionary<?>[allColumns.size()];
-        for (TblColRef tblColRef : valueMap.keySet()) {
-            final Collection<byte[]> bytes = Collections2.transform(valueMap.get(tblColRef), new Function<String, byte[]>() {
-                @Nullable
-                @Override
-                public byte[] apply(String input) {
-                    return input == null ? null : input.getBytes();
-                }
-            });
-            final Dictionary<?> dict = DictionaryGenerator.buildDictionary(tblColRef.getType(), new IterableDictionaryValueEnumerator(bytes));
-            result[desc.findColumn(tblColRef)] = dict;
-        }
-        return result;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescManagerTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescManagerTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescManagerTest.java
deleted file mode 100644
index 96dcbfc..0000000
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescManagerTest.java
+++ /dev/null
@@ -1,103 +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;
-
-import java.io.IOException;
-import java.util.UUID;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- */
-public class IIDescManagerTest extends LocalFileMetadataTestCase {
-
-    public static final String TEST_II_DESC_NAME = "test_kylin_ii_left_join_desc";
-
-    @Before
-    public void setup() {
-        createTestMetadata();
-    }
-
-    @After
-    public void clean() {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void testCRUD() throws IOException {
-        IIDescManager mgr = IIDescManager.getInstance(getTestConfig());
-
-        String newDescName = "Copy of " + TEST_II_DESC_NAME;
-
-        try {
-            IIDesc testRecord = mgr.getIIDesc(newDescName);
-            if (testRecord != null)
-                mgr.removeIIDesc(testRecord);
-        } catch (IOException e) {
-            // just ensure the old one is removed
-        }
-
-        Assert.assertNull(mgr.getIIDesc(newDescName));
-        IIDesc desc = mgr.getIIDesc(TEST_II_DESC_NAME);
-
-        desc.setName(newDescName);
-        desc.setLastModified(0);
-
-        mgr.createIIDesc(desc);
-
-        desc = mgr.getIIDesc(newDescName);
-
-        Assert.assertNotNull(desc);
-
-        mgr.updateIIDesc(desc); // this will trigger cache wipe; please ignore the HTTP error in logs.
-
-        mgr.removeIIDesc(desc);
-
-        Assert.assertNull(mgr.getIIDesc(newDescName));
-
-    }
-
-    @Test
-    public void testReload() throws IOException {
-        IIDescManager mgr = IIDescManager.getInstance(getTestConfig());
-
-        IIDesc desc = mgr.getIIDesc(TEST_II_DESC_NAME);
-
-        // do some modification
-        desc.setUuid(UUID.randomUUID().toString());
-
-        IIDesc newDesc = mgr.getIIDesc(TEST_II_DESC_NAME);
-
-        Assert.assertEquals(desc, newDesc);
-
-        // reload the cache
-        mgr.reloadIIDescLocal(TEST_II_DESC_NAME);
-
-        newDesc = mgr.getIIDesc(TEST_II_DESC_NAME);
-
-        Assert.assertNotEquals(desc, newDesc);
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescTest.java
deleted file mode 100644
index 4b5517f..0000000
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescTest.java
+++ /dev/null
@@ -1,66 +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;
-
-import java.io.IOException;
-
-import org.apache.kylin.common.util.JsonUtil;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.model.DataModelDesc;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- */
-public class IIDescTest extends LocalFileMetadataTestCase {
-
-    @Before
-    public void setup() {
-        this.createTestMetadata();
-
-    }
-
-    @After
-    public void clear() {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void testGetIIDesc() {
-
-        IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_left_join_desc");
-        DataModelDesc model = iiDesc.getModel();
-        Assert.assertNotNull(iiDesc);
-        Assert.assertNotNull(model);
-
-    }
-
-    @Test
-    public void testSerialization() throws IOException {
-        IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_left_join_desc");
-        String str = JsonUtil.writeValueAsIndentString(iiDesc);
-        System.out.println(str);
-        IIDesc desc2 = JsonUtil.readValue(str, IIDesc.class);
-
-        Assert.assertEquals(iiDesc, desc2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
deleted file mode 100644
index 16bbdf0..0000000
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
+++ /dev/null
@@ -1,70 +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;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- */
-public class IIInstanceTest extends LocalFileMetadataTestCase {
-    @Before
-    public void setup() {
-        createTestMetadata();
-    }
-
-    @After
-    public void clean() {
-        cleanupTestMetadata();
-    }
-
-    @Test
-    public void testGetIIsByDesc() throws IOException {
-        IIManager mgr = IIManager.getInstance(getTestConfig());
-
-        List<IIInstance> iiInstances = mgr.getIIsByDesc("test_kylin_ii_left_join_desc");
-
-        Assert.assertTrue(iiInstances.size() > 0);
-
-    }
-
-    @Test
-    public void testCreateIIInstance() throws IOException {
-
-        IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_left_join_desc");
-
-        IIInstance ii = IIInstance.create("new ii", "default", iiDesc);
-
-        IIManager iiMgr = IIManager.getInstance(getTestConfig());
-
-        List<IIInstance> allIIList = iiMgr.listAllIIs();
-
-        iiMgr.createII(ii);
-
-        Assert.assertNotNull(iiMgr.getII("new ii"));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
deleted file mode 100644
index 7b6a688..0000000
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
+++ /dev/null
@@ -1,264 +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;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.dict.DictionaryGenerator;
-import org.apache.kylin.dict.IterableDictionaryValueEnumerator;
-import org.apache.kylin.invertedindex.index.CompressedValueContainer;
-import org.apache.kylin.invertedindex.index.RawTableRecord;
-import org.apache.kylin.invertedindex.index.ShardingSliceBuilder;
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.invertedindex.index.TableRecord;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.apache.kylin.invertedindex.model.IIRow;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.SetMultimap;
-import com.google.common.collect.Sets;
-
-public class InvertedIndexLocalTest extends LocalFileMetadataTestCase {
-
-    IIInstance ii;
-    TableRecordInfo info;
-    List<String> lines;
-    private Dictionary<?>[] dictionaryMap;
-
-    @Before
-    public void setUp() throws Exception {
-        this.createTestMetadata();
-        this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii_left_join");
-
-        File file = new File(LOCALMETA_TEST_DATA, "data/flatten_data_for_ii.csv");
-        FileInputStream in = new FileInputStream(file);
-        this.lines = IOUtils.readLines(in, "UTF-8");
-        in.close();
-
-        dictionaryMap = buildDictionary(Lists.transform(lines, new Function<String, List<String>>() {
-            @Nullable
-            @Override
-            public List<String> apply(@Nullable String input) {
-                return Lists.newArrayList(input.split(","));
-            }
-        }), ii.getDescriptor());
-        this.info = new TableRecordInfo(ii.getDescriptor(), dictionaryMap);
-    }
-
-    @After
-    public void after() throws Exception {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void testCompressedValueContainer() {
-        // create container
-        CompressedValueContainer container = new CompressedValueContainer(info.getDigest(), 0, 500);
-        Dictionary<String> dict = info.dict(0);
-
-        byte[] buf = new byte[dict.getSizeOfId()];
-        ImmutableBytesWritable bytes = new ImmutableBytesWritable(buf);
-
-        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
-            BytesUtil.writeUnsigned(v, buf, 0, dict.getSizeOfId());
-            container.append(bytes);
-        }
-        BytesUtil.writeUnsigned(Dictionary.NULL_ID[dict.getSizeOfId()], buf, 0, dict.getSizeOfId());
-        container.append(bytes);
-        container.closeForChange();
-
-        // copy by serialization
-        ImmutableBytesWritable copy = container.toBytes();
-        CompressedValueContainer container2 = new CompressedValueContainer(info.getDigest(), 0, 500);
-        container2.fromBytes(copy);
-
-        // check the copy
-        int i = 0;
-        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
-            container2.getValueAt(i++, bytes);
-            int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength());
-            assertEquals(v, value);
-        }
-        container2.getValueAt(i++, bytes);
-        int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength());
-        assertEquals(Dictionary.NULL_ID[dict.getSizeOfId()], value);
-        assertEquals(container, container2);
-    }
-
-    @Test
-    public void testCodec() throws IOException {
-        List<TableRecord> records = loadRecordsSorted();
-        System.out.println(records.size() + " records");
-        List<Slice> slices = buildTimeSlices(records);
-        System.out.println(slices.size() + " slices");
-
-        IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest());
-        List<IIRow> kvs = encodeKVs(codec, slices);
-        System.out.println(kvs.size() + " KV pairs");
-
-        List<Slice> slicesCopy = decodeKVs(codec, kvs);
-        assertEquals(slices.size(), slicesCopy.size());
-        for (int i = 0; i < slices.size(); i++) {
-            assertEquals(slices.get(i), slicesCopy.get(i));
-        }
-
-        List<TableRecord> recordsCopy = iterateRecords(slicesCopy);
-        assertEquals(new HashSet<TableRecord>(records), new HashSet<TableRecord>(recordsCopy));
-        dump(recordsCopy);
-    }
-
-    private Dictionary<?>[] buildDictionary(List<List<String>> table, IIDesc desc) throws IOException {
-        SetMultimap<TblColRef, String> valueMap = HashMultimap.create();
-        Set<TblColRef> dimensionColumns = Sets.newHashSet();
-        for (int i = 0; i < desc.listAllColumns().size(); i++) {
-            if (!desc.isMetricsCol(i)) {
-                dimensionColumns.add(desc.listAllColumns().get(i));
-            }
-        }
-        for (List<String> row : table) {
-            for (int i = 0; i < row.size(); i++) {
-                String cell = row.get(i);
-                valueMap.put(desc.listAllColumns().get(i), cell);
-            }
-        }
-        Dictionary<?>[] result = new Dictionary<?>[desc.listAllColumns().size()];
-        for (TblColRef tblColRef : valueMap.keys()) {
-            result[desc.findColumn(tblColRef)] = DictionaryGenerator.buildDictionary(tblColRef.getType(), new IterableDictionaryValueEnumerator(Collections2.transform(valueMap.get(tblColRef), new Function<String, byte[]>() {
-                @Nullable
-                @Override
-                public byte[] apply(String input) {
-                    try {
-                        return input.getBytes("UTF-8");
-                    } catch (UnsupportedEncodingException e) {
-                        throw new RuntimeException(e);
-                    }
-                }
-            })));
-        }
-        return result;
-    }
-
-    private List<TableRecord> loadRecordsSorted() throws IOException {
-        List<TableRecord> records = Lists.newArrayList();
-        for (String line : lines) {
-            String[] fields = line.split(",");
-            TableRecord rec = info.createTableRecord();
-            for (int col = 0; col < fields.length; col++) {
-                rec.setValueString(col, fields[col]);
-            }
-            records.add(rec);
-        }
-
-        Collections.sort(records, new Comparator<TableRecord>() {
-            @Override
-            public int compare(TableRecord a, TableRecord b) {
-                long x = a.getTimestamp() - b.getTimestamp();
-                if (x > 0)
-                    return 1;
-                else if (x == 0)
-                    return 0;
-                else
-                    return -1;
-            }
-        });
-
-        return records;
-    }
-
-    private List<Slice> buildTimeSlices(List<TableRecord> records) throws IOException {
-        ShardingSliceBuilder builder = new ShardingSliceBuilder(info);
-        List<Slice> slices = Lists.newArrayList();
-        for (TableRecord rec : records) {
-            //here assume there less records than slice size for each shard
-            Slice slice = builder.append(rec);
-            if (slice != null) {
-                slice.setLocalDictionaries(dictionaryMap);
-                slices.add(slice);
-            }
-        }
-        List<Slice> finals = builder.close();
-        for (Slice slice : finals) {
-            slice.setLocalDictionaries(dictionaryMap);
-        }
-        slices.addAll(finals);
-
-        Collections.sort(slices);
-        return slices;
-    }
-
-    private List<IIRow> encodeKVs(IIKeyValueCodec codec, List<Slice> slices) {
-
-        List<IIRow> kvs = Lists.newArrayList();
-        for (Slice slice : slices) {
-            kvs.addAll(codec.encodeKeyValue(slice));
-        }
-        return kvs;
-    }
-
-    private List<Slice> decodeKVs(IIKeyValueCodec codec, List<IIRow> kvs) {
-        List<Slice> slices = Lists.newArrayList();
-        for (Slice slice : codec.decodeKeyValue(kvs)) {
-            slices.add(slice);
-        }
-        return slices;
-    }
-
-    private List<TableRecord> iterateRecords(List<Slice> slices) {
-        List<TableRecord> records = Lists.newArrayList();
-        for (Slice slice : slices) {
-            for (RawTableRecord rec : slice) {
-                records.add(new TableRecord((RawTableRecord) rec.clone(), info));
-            }
-        }
-        return records;
-    }
-
-    private void dump(Iterable<TableRecord> records) {
-        for (TableRecord rec : records) {
-            System.out.println(rec.toString());
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/test/java/org/apache/kylin/invertedindex/LZFTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/LZFTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/LZFTest.java
deleted file mode 100644
index 943e76c..0000000
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/LZFTest.java
+++ /dev/null
@@ -1,49 +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;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.junit.Test;
-
-import com.ning.compress.lzf.LZFDecoder;
-import com.ning.compress.lzf.LZFEncoder;
-
-/**
- */
-public class LZFTest {
-    @Test
-    public void test() throws IOException {
-
-        byte[] raw = new byte[] { 1, 2, 3, 3, 2, 23 };
-        byte[] data = LZFEncoder.encode(raw);
-
-        byte[] data2 = new byte[data.length * 2];
-        java.lang.System.arraycopy(data, 0, data2, 0, data.length);
-        ImmutableBytesWritable bytes = new ImmutableBytesWritable();
-        bytes.set(data2, 0, data.length);
-
-        try {
-            byte[] uncompressed = LZFDecoder.decode(bytes.get(), bytes.getOffset(), bytes.getLength());
-        } catch (IOException e) {
-            throw new RuntimeException("LZF decode failure", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
deleted file mode 100644
index 6b21282..0000000
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
+++ /dev/null
@@ -1,62 +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.measure;
-
-import org.apache.kylin.metadata.datatype.DataType;
-import org.junit.Test;
-
-/**
- */
-public class FixedPointLongCodecTest {
-
-    @Test
-    public void testEncode1() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
-        long x = codec.getValueIgnoringDecimalPoint("12.12345");
-        org.junit.Assert.assertEquals(1212345, x);
-    }
-
-    @Test
-    public void testEncode2() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
-        long x = codec.getValueIgnoringDecimalPoint("12.1234");
-        org.junit.Assert.assertEquals(1212340, x);
-    }
-
-    @Test
-    public void testEncode3() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
-        long x = codec.getValueIgnoringDecimalPoint("12.123456");
-        org.junit.Assert.assertEquals(1212345, x);
-    }
-
-    @Test
-    public void testEncode4() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
-        long x = codec.getValueIgnoringDecimalPoint("12");
-        org.junit.Assert.assertEquals(1200000, x);
-    }
-
-    @Test
-    public void testDecode1() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
-        String x = codec.restoreDecimalPoint(1212345);
-        org.junit.Assert.assertEquals("12.12345", x);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/kylin-it/pom.xml
----------------------------------------------------------------------
diff --git a/kylin-it/pom.xml b/kylin-it/pom.xml
index 271bae8..a96dcd9 100644
--- a/kylin-it/pom.xml
+++ b/kylin-it/pom.xml
@@ -74,12 +74,6 @@
             <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.calcite</groupId>
             <artifactId>calcite-linq4j</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/kylin-it/src/test/java/org/apache/kylin/query/ITCombinationTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITCombinationTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITCombinationTest.java
index f0a3178..cf18b20 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/ITCombinationTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/ITCombinationTest.java
@@ -42,7 +42,6 @@ public class ITCombinationTest extends ITKylinQueryTest {
     @BeforeClass
     public static void setUp() throws SQLException {
         Map<RealizationType, Integer> priorities = Maps.newHashMap();
-        priorities.put(RealizationType.INVERTED_INDEX, 2);
         priorities.put(RealizationType.HYBRID, 0);
         priorities.put(RealizationType.CUBE, 0);
         Candidate.setPriorities(priorities);

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/kylin-it/src/test/java/org/apache/kylin/query/ITIIQueryTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITIIQueryTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITIIQueryTest.java
deleted file mode 100644
index fd74dc1..0000000
--- a/kylin-it/src/test/java/org/apache/kylin/query/ITIIQueryTest.java
+++ /dev/null
@@ -1,92 +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.query;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.kylin.metadata.realization.RealizationType;
-import org.apache.kylin.query.routing.Candidate;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runners.Parameterized;
-
-import com.google.common.collect.Maps;
-
-/**
- */
-@Ignore("II query is no longer maintained")
-//@RunWith(Parameterized.class)
-public class ITIIQueryTest extends ITKylinQueryTest {
-    @BeforeClass
-    public static void setUp() throws Exception {
-
-        // give II higher priority than other realizations
-        Map<RealizationType, Integer> priorities = Maps.newHashMap();
-        priorities.put(RealizationType.INVERTED_INDEX, 0);
-        priorities.put(RealizationType.HYBRID, 1);
-        priorities.put(RealizationType.CUBE, 2);
-        Candidate.setPriorities(priorities);
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        ITKylinQueryTest.tearDown();//invoke super class
-        Candidate.restorePriorities();
-    }
-
-    @Parameterized.Parameters
-    public static Collection<Object[]> configs() {
-        return Arrays.asList(new Object[][] { { "inner" }, { "left" } });
-    }
-
-    public ITIIQueryTest(String joinType) throws Exception {
-
-        ITKylinQueryTest.clean();
-
-        ITKylinQueryTest.joinType = joinType;
-        ITKylinQueryTest.setupAll();
-
-    }
-
-    @Test
-    public void testSingleRunQuery() throws Exception {
-        super.testSingleRunQuery();
-    }
-
-    @Test
-    public void testDetailedQuery() throws Exception {
-        execAndCompQuery("src/test/resources/query/sql_ii", null, true);
-    }
-
-    @Override
-    @Test
-    @Ignore("Skip Precisely Distinct Count Queries for II")
-    public void testPreciselyDistinctCountQuery() {
-    }
-
-    @Override
-    @Test
-    @Ignore("Skip Dimension Distinct Count Queries for II")
-    public void testDimDistinctCountQuery() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
index d909a99..a53c624 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
@@ -51,7 +51,6 @@ public class ITKylinQueryTest extends KylinTestBase {
     @BeforeClass
     public static void setUp() throws Exception {
         Map<RealizationType, Integer> priorities = Maps.newHashMap();
-        priorities.put(RealizationType.INVERTED_INDEX, 2);
         priorities.put(RealizationType.HYBRID, 0);
         priorities.put(RealizationType.CUBE, 0);
         Candidate.setPriorities(priorities);

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

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2def6d2..7a211fc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -677,7 +677,6 @@
         <module>server-base</module>
         <module>server</module>
         <module>jdbc</module>
-        <module>invertedindex</module>
         <module>assembly</module>
         <module>tool</module>
         <module>kylin-it</module>

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/query/pom.xml
----------------------------------------------------------------------
diff --git a/query/pom.xml b/query/pom.xml
index 87a4eba..a0b8d94 100644
--- a/query/pom.xml
+++ b/query/pom.xml
@@ -48,12 +48,6 @@
             <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.calcite</groupId>
             <artifactId>calcite-linq4j</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/query/src/main/java/org/apache/kylin/query/routing/Candidate.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/routing/Candidate.java b/query/src/main/java/org/apache/kylin/query/routing/Candidate.java
index 28c60a8..c7c99ab 100644
--- a/query/src/main/java/org/apache/kylin/query/routing/Candidate.java
+++ b/query/src/main/java/org/apache/kylin/query/routing/Candidate.java
@@ -35,7 +35,6 @@ public class Candidate implements Comparable<Candidate> {
     static {
         DEFAULT_PRIORITIES.put(RealizationType.HYBRID, 0);
         DEFAULT_PRIORITIES.put(RealizationType.CUBE, 1);
-        DEFAULT_PRIORITIES.put(RealizationType.INVERTED_INDEX, 2);
     }
 
     /** for test only */

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java b/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
index 7197f03..5f07adf 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
@@ -30,8 +30,6 @@ import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.engine.mr.CubingJob;
 import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
 import org.apache.kylin.engine.streaming.StreamingManager;
-import org.apache.kylin.invertedindex.IIDescManager;
-import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.execution.Output;
@@ -94,14 +92,6 @@ public abstract class BasicService {
         return ExecutableManager.getInstance(getConfig());
     }
 
-    public IIDescManager getIIDescManager() {
-        return IIDescManager.getInstance(getConfig());
-    }
-
-    public IIManager getIIManager() {
-        return IIManager.getInstance(getConfig());
-    }
-
     public BadQueryHistoryManager getBadQueryHistoryManager() {
         return BadQueryHistoryManager.getInstance(getConfig());
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
index 9185544..02efe70 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
@@ -38,8 +38,6 @@ import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.dict.DictionaryManager;
 import org.apache.kylin.engine.streaming.StreamingManager;
-import org.apache.kylin.invertedindex.IIDescManager;
-import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.project.ProjectManager;
@@ -186,28 +184,16 @@ public class CacheService extends BasicService {
             case PROJECT:
                 reloadProjectCache(cacheKey);
                 break;
-            case INVERTED_INDEX:
-                //II update does not need to update storage cache because it is dynamic already
-                getIIManager().reloadIILocal(cacheKey);
-                getHybridManager().reloadHybridInstanceByChild(RealizationType.INVERTED_INDEX, cacheKey);
-                getProjectManager().clearL2Cache();
-                break;
-            case INVERTED_INDEX_DESC:
-                getIIDescManager().reloadIIDescLocal(cacheKey);
-                break;
             case TABLE:
                 getMetadataManager().reloadTableCache(cacheKey);
-                IIDescManager.clearCache();
                 CubeDescManager.clearCache();
                 break;
             case EXTERNAL_FILTER:
                 getMetadataManager().reloadExtFilter(cacheKey);
-                IIDescManager.clearCache();
                 CubeDescManager.clearCache();
                 break;
             case DATA_MODEL:
                 getMetadataManager().reloadDataModelDesc(cacheKey);
-                IIDescManager.clearCache();
                 CubeDescManager.clearCache();
                 break;
             case ALL:
@@ -215,8 +201,6 @@ public class CacheService extends BasicService {
                 MetadataManager.clearCache();
                 CubeDescManager.clearCache();
                 CubeManager.clearCache();
-                IIDescManager.clearCache();
-                IIManager.clearCache();
                 HybridManager.clearCache();
                 RealizationRegistry.clearCache();
                 ProjectManager.clearCache();
@@ -259,12 +243,6 @@ public class CacheService extends BasicService {
             case PROJECT:
                 ProjectManager.clearCache();
                 break;
-            case INVERTED_INDEX:
-                getIIManager().removeIILocal(cacheKey);
-                break;
-            case INVERTED_INDEX_DESC:
-                getIIDescManager().removeIIDescLocal(cacheKey);
-                break;
             case TABLE:
                 throw new UnsupportedOperationException(log);
             case EXTERNAL_FILTER:

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
index 9d8ccfb..4cfa209 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
@@ -24,7 +24,6 @@ import java.util.List;
 
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.invertedindex.model.IIDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.rest.constant.Constant;
@@ -117,14 +116,6 @@ public class ModelService extends BasicService {
             }
         }
 
-        //check II desc exist
-        List<IIDesc> iiDescs = getIIDescManager().listAllDesc();
-        for (IIDesc iidesc : iiDescs) {
-            if (iidesc.getModelName().equals(desc.getName())) {
-                throw new InternalErrorException("Model referenced by IIDesc.");
-            }
-        }
-
         getMetadataManager().dropModel(desc);
 
         accessService.clean(desc, true);

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java b/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java
index f8dc945..ae4c089 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java
@@ -21,8 +21,6 @@ package org.apache.kylin.rest.service;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.CubeDescManager;
 import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.invertedindex.IIDescManager;
-import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.project.ProjectManager;
 import org.apache.kylin.metadata.realization.RealizationRegistry;
@@ -65,8 +63,6 @@ public class ServiceTestBase extends LocalFileMetadataTestCase {
         MetadataManager.clearCache();
         CubeDescManager.clearCache();
         CubeManager.clearCache();
-        IIDescManager.clearCache();
-        IIManager.clearCache();
         RealizationRegistry.clearCache();
         ProjectManager.clearCache();
         CacheService.removeAllOLAPDataSources();

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/storage-hbase/pom.xml b/storage-hbase/pom.xml
index 943b2d4..0cb42db 100644
--- a/storage-hbase/pom.xml
+++ b/storage-hbase/pom.xml
@@ -46,11 +46,6 @@
             <artifactId>kylin-engine-streaming</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-invertedindex</artifactId>
-            <version>${project.parent.version}</version>
-        </dependency>
 
         <!-- Env & Test -->
         <dependency>
@@ -135,7 +130,6 @@
                                     <include>org.apache.kylin:kylin-core-metadata</include>
                                     <include>org.apache.kylin:kylin-core-dictionary</include>
                                     <include>org.apache.kylin:kylin-core-cube</include>
-                                    <include>org.apache.kylin:kylin-invertedindex</include>
                                     <include>com.ning:compress-lzf</include>
                                     <include>org.roaringbitmap:RoaringBitmap</include>
                                     <!-- below for inverted index only -->

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
index 20bc229..e7a322c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
@@ -18,12 +18,12 @@
 
 package org.apache.kylin.storage.hbase;
 
+import com.google.common.base.Preconditions;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.engine.mr.IMROutput;
 import org.apache.kylin.engine.mr.IMROutput2;
-import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.IStorageAware;
@@ -36,8 +36,6 @@ import org.apache.kylin.storage.IStorageQuery;
 import org.apache.kylin.storage.hbase.steps.HBaseMROutput;
 import org.apache.kylin.storage.hbase.steps.HBaseMROutput2Transition;
 
-import com.google.common.base.Preconditions;
-
 @SuppressWarnings("unused")
 //used by reflection
 public class HBaseStorage implements IStorage {
@@ -46,21 +44,10 @@ public class HBaseStorage implements IStorage {
     public final static String v1CubeStorageQuery = "org.apache.kylin.storage.hbase.cube.v1.CubeStorageQuery";
     public static String overwriteStorageQuery = null;//for test case
 
-    private final static String defaultIIStorageQuery = "org.apache.kylin.storage.hbase.ii.InvertedIndexStorageQuery";
-
     @Override
     public IStorageQuery createQuery(IRealization realization) {
 
-        if (realization.getType() == RealizationType.INVERTED_INDEX) {
-            IStorageQuery ret;
-            try {
-                ret = (IStorageQuery) Class.forName(defaultIIStorageQuery).getConstructor(IIInstance.class).newInstance((IIInstance) realization);
-            } catch (Exception e) {
-                throw new RuntimeException("Failed to initialize storage query for " + defaultIIStorageQuery, e);
-            }
-            return ret;
-
-        } else if (realization.getType() == RealizationType.CUBE) {
+        if (realization.getType() == RealizationType.CUBE) {
 
             CubeInstance cubeInstance = (CubeInstance) realization;
             String cubeStorageQuery;

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorProjector.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorProjector.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorProjector.java
index e142536..65c5f92 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorProjector.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorProjector.java
@@ -29,7 +29,6 @@ import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.RowKeyEncoder;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
 import org.apache.kylin.metadata.model.TblColRef;
 
 /**
@@ -55,20 +54,7 @@ public class CoprocessorProjector {
         byte[] mask = rowKeyMaskEncoder.encode(new byte[cuboid.getColumns().size()][]);
         return new CoprocessorProjector(mask, dimensionColumns.size() != 0);
     }
-
-    public static CoprocessorProjector makeForEndpoint(final TableRecordInfo tableInfo, final Collection<TblColRef> groupby) {
-        byte[] mask = new byte[tableInfo.getDigest().getByteFormLen()];
-        int maskIdx = 0;
-        for (int i = 0; i < tableInfo.getDigest().getColumnCount(); ++i) {
-            TblColRef tblColRef = tableInfo.getColumns().get(i);
-            int length = tableInfo.getDigest().length(i);
-            byte bits = groupby.contains(tblColRef) ? (byte) 0xff : 0x00;
-            for (int j = 0; j < length; ++j) {
-                mask[maskIdx++] = bits;
-            }
-        }
-        return new CoprocessorProjector(mask, groupby.size() != 0);
-    }
+  
 
     public static byte[] serialize(CoprocessorProjector o) {
         ByteBuffer buf = ByteBuffer.allocate(BytesSerializer.SERIALIZE_BUFFER_SIZE);

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorRowType.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorRowType.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorRowType.java
index a6e3073..d47bf61 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorRowType.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/CoprocessorRowType.java
@@ -27,7 +27,6 @@ import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.RowKeyColumnIO;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -39,17 +38,6 @@ import com.google.common.collect.Maps;
  */
 public class CoprocessorRowType {
 
-    //for endpoint
-    public static CoprocessorRowType fromTableRecordInfo(TableRecordInfo tableRecordInfo, List<TblColRef> cols) {
-
-        int[] colSizes = new int[cols.size()];
-        for (int i = 0; i < cols.size(); i++) {
-            colSizes[i] = tableRecordInfo.getDigest().length(i);
-        }
-
-        //TODO:check0
-        return new CoprocessorRowType(cols.toArray(new TblColRef[cols.size()]), colSizes, 0);
-    }
 
     //for observer
     public static CoprocessorRowType fromCuboid(CubeSegment seg, Cuboid cuboid) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/HBaseClientKVIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/HBaseClientKVIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/HBaseClientKVIterator.java
deleted file mode 100644
index 8aace22..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/HBaseClientKVIterator.java
+++ /dev/null
@@ -1,94 +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.storage.hbase.cube.v1;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.invertedindex.model.IIRow;
-
-/**
- * @author yangli9
- * 
- */
-public class HBaseClientKVIterator implements Iterable<IIRow>, Closeable {
-
-    byte[] family;
-
-    HTableInterface table;
-    ResultScanner scanner;
-    Iterator<Result> iterator;
-
-    public HBaseClientKVIterator(HConnection hconn, String tableName, byte[] family) throws IOException {
-        this.family = family;
-
-        this.table = hconn.getTable(tableName);
-        this.scanner = table.getScanner(family);
-        this.iterator = scanner.iterator();
-    }
-
-    @Override
-    public void close() {
-        IOUtils.closeQuietly(scanner);
-        IOUtils.closeQuietly(table);
-    }
-
-    @Override
-    public Iterator<IIRow> iterator() {
-        return new MyIterator();
-    }
-
-    private class MyIterator implements Iterator<IIRow> {
-
-        ImmutableBytesWritable key = new ImmutableBytesWritable();
-        ImmutableBytesWritable value = new ImmutableBytesWritable();
-        ImmutableBytesWritable dict = new ImmutableBytesWritable();
-        IIRow pair = new IIRow(key, value, dict);
-
-        @Override
-        public boolean hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public IIRow next() {
-            Result r = iterator.next();
-            Cell c = r.getColumnLatestCell(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
-            key.set(c.getRowArray(), c.getRowOffset(), c.getRowLength());
-            value.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-            c = r.getColumnLatestCell(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_DICTIONARY_BYTES);
-            dict.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-            return pair;
-        }
-
-        public void remove() {
-            throw new UnsupportedOperationException();
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IIBulkLoadJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IIBulkLoadJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IIBulkLoadJob.java
deleted file mode 100644
index 22c8ec6..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IIBulkLoadJob.java
+++ /dev/null
@@ -1,65 +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.storage.hbase.ii;
-
-import java.io.IOException;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.storage.hbase.HBaseConnection;
-
-/**
- */
-public class IIBulkLoadJob extends AbstractHadoopJob {
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_INPUT_PATH);
-            options.addOption(OPTION_HTABLE_NAME);
-            options.addOption(OPTION_II_NAME);
-            parseOptions(options, args);
-
-            String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
-            String input = getOptionValue(OPTION_INPUT_PATH);
-
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            FsShell shell = new FsShell(conf);
-            try {
-                shell.run(new String[] { "-chmod", "-R", "777", input });
-            } catch (Exception e) {
-                logger.error("Couldn't change the file permissions ", e);
-                throw new IOException(e);
-            }
-
-            return ToolRunner.run(new LoadIncrementalHFiles(conf), new String[] { input, tableName });
-
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileJob.java
deleted file mode 100644
index 30dca8e..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileJob.java
+++ /dev/null
@@ -1,88 +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.storage.hbase.ii;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author yangli9
- * 
- */
-public class IICreateHFileJob extends AbstractHadoopJob {
-
-    protected static final Logger logger = LoggerFactory.getLogger(IICreateHFileJob.class);
-
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_II_NAME);
-            options.addOption(OPTION_INPUT_PATH);
-            options.addOption(OPTION_OUTPUT_PATH);
-            options.addOption(OPTION_HTABLE_NAME);
-            parseOptions(options, args);
-
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-
-            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
-
-            String iiName = getOptionValue(OPTION_II_NAME);
-            IIManager mgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
-            IIInstance ii = mgr.getII(iiName);
-
-            setJobClasspath(job, ii.getConfig());
-
-            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
-            FileOutputFormat.setOutputPath(job, output);
-
-            job.setInputFormatClass(SequenceFileInputFormat.class);
-            job.setMapperClass(IICreateHFileMapper.class);
-            job.setMapOutputKeyClass(ImmutableBytesWritable.class);
-            job.setMapOutputValueClass(KeyValue.class);
-
-            String tableName = getOptionValue(OPTION_HTABLE_NAME);
-            HTable htable = new HTable(HBaseConfiguration.create(getConf()), tableName);
-            HFileOutputFormat.configureIncrementalLoad(job, htable);
-
-            this.deletePath(job.getConfiguration(), output);
-
-            return waitForCompletion(job);
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-}


[10/55] [abbrv] kylin git commit: KYLIN-1862 "table not found" in "Build Dimension Dictionary" step

Posted by sh...@apache.org.
KYLIN-1862 "table not found" in "Build Dimension Dictionary" step

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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 687d5933a91139d900c8544fcf74abf6c6dae0ee
Parents: 854ec37
Author: shaofengshi <sh...@apache.org>
Authored: Fri Jul 8 22:30:52 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Jul 8 22:30:52 2016 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/dict/DictionaryManager.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/687d5933/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 74120dc..bd00f10 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
@@ -290,9 +290,9 @@ public class DictionaryManager {
             MetadataManager metadataManager = MetadataManager.getInstance(config);
             TableDesc tableDesc = new TableDesc(metadataManager.getTableDesc(srcTable));
             if (TableDesc.TABLE_TYPE_VIRTUAL_VIEW.equalsIgnoreCase(tableDesc.getTableType())) {
-                tableDesc.setDatabase(config.getHiveDatabaseForIntermediateTable());
-                String tableName = tableDesc.getMaterializedName();
-                tableDesc.setName(tableName);
+                TableDesc materializedTbl = new TableDesc();
+                materializedTbl.setDatabase(config.getHiveDatabaseForIntermediateTable());
+                materializedTbl.setName(tableDesc.getMaterializedName());
                 inpTable = SourceFactory.createReadableTable(tableDesc);
             } else {
                 inpTable = SourceFactory.createReadableTable(tableDesc);


[07/55] [abbrv] kylin git commit: KYLIN-1858 remove all ii related code

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/invertedindex/.settings/org.eclipse.jdt.core.prefs b/invertedindex/.settings/org.eclipse.jdt.core.prefs
deleted file mode 100644
index 5aaaf1e..0000000
--- a/invertedindex/.settings/org.eclipse.jdt.core.prefs
+++ /dev/null
@@ -1,386 +0,0 @@
-eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
-org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
-org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
-org.eclipse.jdt.core.compiler.annotation.nonnull.secondary=
-org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
-org.eclipse.jdt.core.compiler.annotation.nonnullbydefault.secondary=
-org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
-org.eclipse.jdt.core.compiler.annotation.nullable.secondary=
-org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
-org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
-org.eclipse.jdt.core.compiler.codegen.methodParameters=do not generate
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
-org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
-org.eclipse.jdt.core.compiler.compliance=1.7
-org.eclipse.jdt.core.compiler.debug.lineNumber=generate
-org.eclipse.jdt.core.compiler.debug.localVariable=generate
-org.eclipse.jdt.core.compiler.debug.sourceFile=generate
-org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
-org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
-org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
-org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
-org.eclipse.jdt.core.compiler.problem.deadCode=warning
-org.eclipse.jdt.core.compiler.problem.deprecation=warning
-org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
-org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
-org.eclipse.jdt.core.compiler.problem.discouragedReference=ignore
-org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
-org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
-org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
-org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
-org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
-org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
-org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
-org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
-org.eclipse.jdt.core.compiler.problem.forbiddenReference=ignore
-org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
-org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
-org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
-org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
-org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
-org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
-org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
-org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
-org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
-org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
-org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
-org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
-org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
-org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
-org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
-org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
-org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
-org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
-org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
-org.eclipse.jdt.core.compiler.problem.nonnullTypeVariableFromLegacyInvocation=warning
-org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
-org.eclipse.jdt.core.compiler.problem.nullReference=warning
-org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
-org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
-org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
-org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
-org.eclipse.jdt.core.compiler.problem.pessimisticNullAnalysisForFreeTypeVariables=warning
-org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
-org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
-org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
-org.eclipse.jdt.core.compiler.problem.rawTypeReference=ignore
-org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
-org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
-org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
-org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
-org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
-org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
-org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
-org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
-org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
-org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
-org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
-org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
-org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
-org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
-org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=ignore
-org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
-org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
-org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
-org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
-org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
-org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
-org.eclipse.jdt.core.compiler.problem.unusedExceptionParameter=ignore
-org.eclipse.jdt.core.compiler.problem.unusedImport=warning
-org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
-org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
-org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
-org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
-org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
-org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
-org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
-org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
-org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
-org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
-org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
-org.eclipse.jdt.core.compiler.source=1.7
-org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
-org.eclipse.jdt.core.formatter.alignment_for_assignment=0
-org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
-org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
-org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
-org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
-org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
-org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
-org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
-org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
-org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
-org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
-org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
-org.eclipse.jdt.core.formatter.blank_lines_after_package=1
-org.eclipse.jdt.core.formatter.blank_lines_before_field=0
-org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
-org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
-org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
-org.eclipse.jdt.core.formatter.blank_lines_before_method=1
-org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
-org.eclipse.jdt.core.formatter.blank_lines_before_package=0
-org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
-org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
-org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
-org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
-org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
-org.eclipse.jdt.core.formatter.comment.format_block_comments=false
-org.eclipse.jdt.core.formatter.comment.format_header=false
-org.eclipse.jdt.core.formatter.comment.format_html=true
-org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
-org.eclipse.jdt.core.formatter.comment.format_line_comments=false
-org.eclipse.jdt.core.formatter.comment.format_source_code=true
-org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
-org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
-org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
-org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
-org.eclipse.jdt.core.formatter.comment.line_length=80
-org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
-org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
-org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
-org.eclipse.jdt.core.formatter.compact_else_if=true
-org.eclipse.jdt.core.formatter.continuation_indentation=2
-org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
-org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
-org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
-org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
-org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
-org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
-org.eclipse.jdt.core.formatter.indent_empty_lines=false
-org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
-org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
-org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
-org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
-org.eclipse.jdt.core.formatter.indentation.size=4
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
-org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
-org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
-org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
-org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
-org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
-org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.join_lines_in_comments=true
-org.eclipse.jdt.core.formatter.join_wrapped_lines=true
-org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
-org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
-org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
-org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
-org.eclipse.jdt.core.formatter.lineSplit=999
-org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
-org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
-org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
-org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
-org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
-org.eclipse.jdt.core.formatter.tabulation.char=space
-org.eclipse.jdt.core.formatter.tabulation.size=4
-org.eclipse.jdt.core.formatter.use_on_off_tags=false
-org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
-org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
-org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
-org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/invertedindex/.settings/org.eclipse.jdt.ui.prefs b/invertedindex/.settings/org.eclipse.jdt.ui.prefs
deleted file mode 100644
index d521bab..0000000
--- a/invertedindex/.settings/org.eclipse.jdt.ui.prefs
+++ /dev/null
@@ -1,7 +0,0 @@
-eclipse.preferences.version=1
-formatter_profile=_Space Indent & Long Lines
-formatter_settings_version=12
-org.eclipse.jdt.ui.ignorelowercasenames=true
-org.eclipse.jdt.ui.importorder=java;javax;org;com;
-org.eclipse.jdt.ui.ondemandthreshold=99
-org.eclipse.jdt.ui.staticondemandthreshold=99

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/pom.xml
----------------------------------------------------------------------
diff --git a/invertedindex/pom.xml b/invertedindex/pom.xml
deleted file mode 100644
index 11fa401..0000000
--- a/invertedindex/pom.xml
+++ /dev/null
@@ -1,103 +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/xsd/maven-4.0.0.xsd">
-    <parent>
-        <artifactId>kylin</artifactId>
-        <groupId>org.apache.kylin</groupId>
-        <version>1.5.3-SNAPSHOT</version>
-
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-    <artifactId>kylin-invertedindex</artifactId>
-    <packaging>jar</packaging>
-    <name>Kylin:InvertedIndex</name>
-
-    <dependencies>
-
-        <!--Kylin Jar -->
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-common</artifactId>
-            <version>${project.parent.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-metadata</artifactId>
-            <version>${project.parent.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-dictionary</artifactId>
-            <version>${project.parent.version}</version>
-        </dependency>
-        
-        <dependency>
-            <groupId>com.n3twork.druid</groupId>
-            <artifactId>extendedset</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.ning</groupId>
-            <artifactId>compress-lzf</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-mapreduce-client-jobclient</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-hadoop2-compat</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.mrunit</groupId>
-            <artifactId>mrunit</artifactId>
-            <classifier>hadoop2</classifier>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-
-</project>

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/IICapabilityChecker.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IICapabilityChecker.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IICapabilityChecker.java
deleted file mode 100644
index 4ee8f50..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IICapabilityChecker.java
+++ /dev/null
@@ -1,182 +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;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.JoinDesc;
-import org.apache.kylin.metadata.model.LookupDesc;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.realization.CapabilityResult;
-import org.apache.kylin.metadata.realization.SQLDigest;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Sets;
-
-/**
- */
-public class IICapabilityChecker {
-    private static final Logger logger = LoggerFactory.getLogger(IICapabilityChecker.class);
-
-    public static CapabilityResult check(IIInstance ii, SQLDigest digest) {
-        CapabilityResult result = new CapabilityResult();
-        result.capable = false;
-
-        // match fact table
-        if (!digest.factTable.equalsIgnoreCase(ii.getFactTable())) {
-            logger.info("Exclude II " + ii.getName() + " because fact table unmatch");
-            return result;
-        }
-
-        // match joins
-        boolean matchJoin = isMatchedWithJoins(digest.joinDescs, ii);
-        if (!matchJoin) {
-            logger.info("Exclude II " + ii.getName() + " because unmatched joins");
-            return result;
-        }
-
-        // dimensions & measures
-        Collection<TblColRef> dimensionColumns = getDimensionColumns(digest);
-        Collection<FunctionDesc> aggrFunctions = digest.aggregations;
-        Collection<TblColRef> unmatchedDimensions = unmatchedDimensions(dimensionColumns, ii);
-        Collection<FunctionDesc> unmatchedAggregations = unmatchedAggregations(aggrFunctions, ii);
-
-        // try dimension-as-measure
-        if (!unmatchedAggregations.isEmpty()) {
-            tryDimensionAsMeasures(unmatchedAggregations, digest, ii, result);
-        }
-
-        if (!unmatchedDimensions.isEmpty()) {
-            logger.info("Exclude ii " + ii.getName() + " because unmatched dimensions");
-            return result;
-        }
-
-        if (!unmatchedAggregations.isEmpty()) {
-            logger.info("Exclude ii " + ii.getName() + " because unmatched aggregations");
-            return result;
-        }
-
-        // cost will be minded by caller
-        result.capable = true;
-        return result;
-    }
-
-    private static boolean isMatchedWithJoins(Collection<JoinDesc> joins, IIInstance iiInstance) {
-        IIDesc iiDesc = iiInstance.getDescriptor();
-        List<TableDesc> tables = iiDesc.listTables();
-
-        List<JoinDesc> cubeJoins = new ArrayList<JoinDesc>(tables.size());
-        for (TableDesc tableDesc : tables) {
-            JoinDesc join = null;
-            for (LookupDesc lookup : iiDesc.getModel().getLookups()) {
-                if (lookup.getTable().equalsIgnoreCase(tableDesc.getIdentity())) {
-                    join = lookup.getJoin();
-                    cubeJoins.add(join);
-                    break;
-                }
-            }
-        }
-
-        for (JoinDesc j : joins) {
-            // optiq engine can't decide which one is fk or pk
-            String pTable = j.getPrimaryKeyColumns()[0].getTable();
-            String factTable = iiDesc.getModel().getFactTable();
-            if (factTable.equals(pTable)) {
-                j.swapPKFK();
-            }
-
-            // check primary key, all PK column should refer to same tale, the Fact Table of iiInstance.
-            // Using first column's table name to check.
-            String fTable = j.getForeignKeyColumns()[0].getTable();
-            if (!factTable.equals(fTable)) {
-                logger.info("Fact Table" + factTable + " not matched in join: " + j + " on ii " + iiInstance.getName());
-                return false;
-            }
-
-            // The hashcode() function of JoinDesc has been overwritten,
-            // which takes into consideration: pk,fk,jointype
-            if (!cubeJoins.contains(j)) {
-                logger.info("Query joins don't match on ii " + iiInstance.getName());
-                return false;
-            }
-        }
-        return true;
-    }
-
-    private static Collection<TblColRef> getDimensionColumns(SQLDigest sqlDigest) {
-        Collection<TblColRef> groupByColumns = sqlDigest.groupbyColumns;
-        Collection<TblColRef> filterColumns = sqlDigest.filterColumns;
-
-        Collection<TblColRef> dimensionColumns = new HashSet<TblColRef>();
-        dimensionColumns.addAll(groupByColumns);
-        dimensionColumns.addAll(filterColumns);
-        return dimensionColumns;
-    }
-
-    private static Set<TblColRef> unmatchedDimensions(Collection<TblColRef> dimensionColumns, IIInstance ii) {
-        HashSet<TblColRef> result = Sets.newHashSet(dimensionColumns);
-        result.removeAll(ii.getDescriptor().listAllDimensions());
-        return result;
-    }
-
-    private static Set<FunctionDesc> unmatchedAggregations(Collection<FunctionDesc> aggregations, IIInstance ii) {
-        HashSet<FunctionDesc> result = Sets.newHashSet(aggregations);
-        result.removeAll(ii.getDescriptor().listAllFunctions());
-        return result;
-    }
-
-    private static void tryDimensionAsMeasures(Collection<FunctionDesc> unmatchedAggregations, SQLDigest digest, IIInstance ii, CapabilityResult result) {
-        IIDesc iiDesc = ii.getDescriptor();
-        Collection<FunctionDesc> iiFuncs = iiDesc.listAllFunctions();
-
-        Iterator<FunctionDesc> it = unmatchedAggregations.iterator();
-        while (it.hasNext()) {
-            FunctionDesc functionDesc = it.next();
-
-            if (iiFuncs.contains(functionDesc)) {
-                it.remove();
-                continue;
-            }
-
-            // let calcite handle count
-            if (functionDesc.isCount()) {
-                it.remove();
-                continue;
-            }
-
-            // calcite can do aggregation from columns on-the-fly
-            List<TblColRef> neededCols = functionDesc.getParameter().getColRefs();
-            if (neededCols.size() > 0 && iiDesc.listAllDimensions().containsAll(neededCols)) {
-                result.influences.add(new CapabilityResult.DimensionAsMeasure(functionDesc));
-                it.remove();
-                continue;
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIDescManager.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIDescManager.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIDescManager.java
deleted file mode 100644
index 917fe46..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIDescManager.java
+++ /dev/null
@@ -1,251 +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;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.JsonSerializer;
-import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.MetadataConstants;
-import org.apache.kylin.metadata.MetadataManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * copied from CubeDescManager
- * 
- * @author honma
- */
-public class IIDescManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(IIDescManager.class);
-
-    private static final Serializer<IIDesc> II_DESC_SERIALIZER = new JsonSerializer<IIDesc>(IIDesc.class);
-
-    // static cached instances
-    private static final ConcurrentHashMap<KylinConfig, IIDescManager> CACHE = new ConcurrentHashMap<KylinConfig, IIDescManager>();
-
-    // ============================================================================
-
-    private KylinConfig config;
-    // name ==> IIDesc
-    private CaseInsensitiveStringCache<IIDesc> iiDescMap;
-
-    public static IIDescManager getInstance(KylinConfig config) {
-        IIDescManager r = CACHE.get(config);
-        if (r != null) {
-            return r;
-        }
-
-        synchronized (IIDescManager.class) {
-            r = CACHE.get(config);
-            if (r != null) {
-                return r;
-            }
-            try {
-                r = new IIDescManager(config);
-                CACHE.put(config, r);
-                if (CACHE.size() > 1) {
-                    logger.warn("More than one singleton exist");
-                }
-                return r;
-            } catch (IOException e) {
-                throw new IllegalStateException("Failed to init IIDescManager from " + config, e);
-            }
-        }
-    }
-
-    public static void clearCache() {
-        CACHE.clear();
-    }
-
-    private IIDescManager(KylinConfig config) throws IOException {
-        logger.info("Initializing IIDescManager with config " + config);
-        this.config = config;
-        this.iiDescMap = new CaseInsensitiveStringCache<IIDesc>(config, Broadcaster.TYPE.INVERTED_INDEX_DESC);
-        reloadAllIIDesc();
-    }
-
-    public List<IIDesc> listAllDesc() {
-        return new ArrayList<IIDesc>(iiDescMap.values());
-    }
-
-    public IIDesc getIIDesc(String name) {
-        return iiDescMap.get(name);
-    }
-
-    /**
-     * Reload IIDesc from resource store It will be triggered by an desc update
-     * event.
-     * 
-     * @param name
-     * @throws IOException
-     */
-    public IIDesc reloadIIDescLocal(String name) throws IOException {
-
-        // Save Source
-        String path = IIDesc.getIIDescResourcePath(name);
-
-        // Reload the IIDesc
-        IIDesc ndesc = loadIIDesc(path);
-
-        // Here replace the old one
-        iiDescMap.putLocal(ndesc.getName(), ndesc);
-        return ndesc;
-    }
-
-    private IIDesc loadIIDesc(String path) throws IOException {
-        ResourceStore store = getStore();
-        logger.info("Loading IIDesc " + store.getReadableResourcePath(path));
-
-        IIDesc ndesc = store.getResource(path, IIDesc.class, II_DESC_SERIALIZER);
-
-        if (StringUtils.isBlank(ndesc.getName())) {
-            throw new IllegalStateException("IIDesc name must not be blank");
-        }
-
-        ndesc.init(getMetadataManager());
-
-        return ndesc;
-    }
-
-    /**
-     * Create a new IIDesc
-     * 
-     * @param iiDesc
-     * @return
-     * @throws IOException
-     */
-    public IIDesc createIIDesc(IIDesc iiDesc) throws IOException {
-        if (iiDesc.getUuid() == null || iiDesc.getName() == null)
-            throw new IllegalArgumentException();
-
-        if (iiDescMap.containsKey(iiDesc.getName()))
-            throw new IllegalArgumentException("IIDesc '" + iiDesc.getName() + "' already exists");
-
-        iiDesc.init(getMetadataManager());
-
-        // Check base validation
-        // Semantic validation
-        // TODO
-
-        iiDesc.setSignature(iiDesc.calculateSignature());
-
-        String path = iiDesc.getResourcePath();
-        getStore().putResource(path, iiDesc, II_DESC_SERIALIZER);
-        iiDescMap.put(iiDesc.getName(), iiDesc);
-
-        return iiDesc;
-    }
-
-    // remove iiDesc
-    public void removeIIDesc(IIDesc iiDesc) throws IOException {
-        String path = iiDesc.getResourcePath();
-        getStore().deleteResource(path);
-        iiDescMap.remove(iiDesc.getName());
-    }
-
-    public void removeIIDescLocal(String name) throws IOException {
-        iiDescMap.removeLocal(name);
-    }
-
-    private void reloadAllIIDesc() throws IOException {
-        ResourceStore store = getStore();
-        logger.info("Reloading all II desc from folder " + store.getReadableResourcePath(ResourceStore.II_DESC_RESOURCE_ROOT));
-
-        iiDescMap.clear();
-
-        List<String> paths = store.collectResourceRecursively(ResourceStore.II_DESC_RESOURCE_ROOT, MetadataConstants.FILE_SURFIX);
-        for (String path : paths) {
-            logger.info("loading II Desc from path" + path);
-            IIDesc desc;
-            try {
-                desc = loadIIDesc(path);
-            } catch (Exception e) {
-                logger.error("Error loading II desc " + path, e);
-                continue;
-            }
-            if (!path.equals(desc.getResourcePath())) {
-                logger.error("Skip suspicious desc at " + path + ", " + desc + " should be at " + desc.getResourcePath());
-                continue;
-            }
-            if (iiDescMap.containsKey(desc.getName())) {
-                logger.error("Dup IIDesc name '" + desc.getName() + "' on path " + path);
-                continue;
-            }
-
-            iiDescMap.putLocal(desc.getName(), desc);
-        }
-
-        logger.debug("Loaded " + iiDescMap.size() + " II desc(s)");
-    }
-
-    /**
-     * Update IIDesc with the input. Broadcast the event into cluster
-     * 
-     * @param desc
-     * @return
-     * @throws IOException
-     */
-    public IIDesc updateIIDesc(IIDesc desc) throws IOException {
-        // Validate IIDesc
-        if (desc.getUuid() == null || desc.getName() == null) {
-            throw new IllegalArgumentException();
-        }
-        String name = desc.getName();
-        if (!iiDescMap.containsKey(name)) {
-            throw new IllegalArgumentException("IIDesc '" + name + "' does not exist.");
-        }
-
-        desc.init(getMetadataManager());
-
-        // TODO: Semantic validation
-
-        desc.setSignature(desc.calculateSignature());
-
-        // Save Source
-        String path = desc.getResourcePath();
-        getStore().putResource(path, desc, II_DESC_SERIALIZER);
-
-        // Reload the IIDesc
-        IIDesc ndesc = loadIIDesc(path);
-        // Here replace the old one
-        iiDescMap.put(ndesc.getName(), desc);
-
-        return ndesc;
-    }
-
-    private MetadataManager getMetadataManager() {
-        return MetadataManager.getInstance(config);
-    }
-
-    private ResourceStore getStore() {
-        return ResourceStore.getStore(this.config);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/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
deleted file mode 100644
index 9b56c88..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
+++ /dev/null
@@ -1,369 +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;
-
-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.DataModelDesc;
-import org.apache.kylin.metadata.model.IBuildable;
-import org.apache.kylin.metadata.model.IStorageAware;
-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.CapabilityResult;
-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 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
- */
-@SuppressWarnings("serial")
-@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class IIInstance extends RootPersistentEntity implements IRealization, IBuildable {
-
-    public static IIInstance create(String iiName, String projectName, IIDesc iiDesc) {
-        IIInstance iii = new IIInstance();
-
-        iii.setConfig(iiDesc.getConfig());
-        iii.setName(iiName);
-        iii.setDescName(iiDesc.getName());
-        iii.setCreateTimeUTC(System.currentTimeMillis());
-        iii.setStatus(RealizationStatusEnum.DISABLED);
-        iii.updateRandomUuid();
-
-        return iii;
-    }
-
-    @JsonIgnore
-    private KylinConfig config;
-    @JsonProperty("name")
-    private String name;
-    @JsonProperty("owner")
-    private String owner;
-    @JsonProperty("descriptor")
-    private String descName;
-    // Mark cube priority for query
-    @JsonProperty("cost")
-    private int cost = 50;
-    @JsonProperty("status")
-    private RealizationStatusEnum status;
-
-    @JsonManagedReference
-    @JsonProperty("segments")
-    private List<IISegment> segments = new ArrayList<IISegment>();
-
-    @JsonProperty("create_time_utc")
-    private long createTimeUTC;
-
-    private static final int COST_WEIGHT_LOOKUP_TABLE = 1;
-    private static final int COST_WEIGHT_INNER_JOIN = 2;
-
-    public long getAllocatedEndDate() {
-        if (null == segments || segments.size() == 0) {
-            return 0;
-        }
-
-        Collections.sort(segments);
-
-        return segments.get(segments.size() - 1).getDateRangeEnd();
-    }
-
-    public long getAllocatedStartDate() {
-        if (null == segments || segments.size() == 0) {
-            return 0;
-        }
-
-        Collections.sort(segments);
-
-        return segments.get(0).getDateRangeStart();
-    }
-
-    public IIDesc getDescriptor() {
-        return IIDescManager.getInstance(config).getIIDesc(descName);
-    }
-
-    @Override
-    public DataModelDesc getDataModelDesc() {
-        return this.getDescriptor().getModel();
-    }
-
-    public boolean isReady() {
-        return getStatus() == RealizationStatusEnum.READY;
-    }
-
-    public String getResourcePath() {
-        return concatResourcePath(name);
-    }
-
-    public static String concatResourcePath(String cubeName) {
-        return ResourceStore.II_RESOURCE_ROOT + "/" + cubeName + ".json";
-    }
-
-    @Override
-    public String toString() {
-        return getCanonicalName();
-    }
-
-    // ============================================================================
-
-    @JsonProperty("size_kb")
-    public long getSizeKB() {
-        long sizeKb = 0L;
-
-        for (IISegment cubeSegment : this.getSegments(SegmentStatusEnum.READY)) {
-            sizeKb += cubeSegment.getSizeKB();
-        }
-
-        return sizeKb;
-    }
-
-    @JsonProperty("input_records_count")
-    public long getInputRecordCount() {
-        long sizeRecordCount = 0L;
-
-        for (IISegment cubeSegment : this.getSegments(SegmentStatusEnum.READY)) {
-            sizeRecordCount += cubeSegment.getInputRecords();
-        }
-
-        return sizeRecordCount;
-    }
-
-    @JsonProperty("input_records_size")
-    public long getInputRecordSize() {
-        long sizeRecordSize = 0L;
-
-        for (IISegment cubeSegment : this.getSegments(SegmentStatusEnum.READY)) {
-            sizeRecordSize += cubeSegment.getInputRecordsSize();
-        }
-
-        return sizeRecordSize;
-    }
-
-    public KylinConfig getConfig() {
-        return config;
-    }
-
-    public void setConfig(KylinConfig config) {
-        this.config = config;
-    }
-
-    @Override
-    public String getCanonicalName() {
-        return getType() + "[name=" + name + "]";
-    }
-
-    @Override
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getOwner() {
-        return owner;
-    }
-
-    public void setOwner(String owner) {
-        this.owner = owner;
-    }
-
-    public String getDescName() {
-        return descName;
-    }
-
-    public void setDescName(String descName) {
-        this.descName = descName;
-    }
-
-    public int getCost() {
-        return cost;
-    }
-
-    public RealizationStatusEnum getStatus() {
-        return status;
-    }
-
-    public void setStatus(RealizationStatusEnum status) {
-        this.status = status;
-    }
-
-    public IISegment getFirstSegment() {
-        if (this.segments == null || this.segments.size() == 0) {
-            return null;
-        } else {
-            return this.segments.get(0);
-        }
-    }
-
-    public List<IISegment> getSegments() {
-        return segments;
-    }
-
-    public List<IISegment> getSegments(SegmentStatusEnum status) {
-        List<IISegment> result = new ArrayList<IISegment>();
-
-        for (IISegment segment : segments) {
-            if (segment.getStatus() == status) {
-                result.add(segment);
-            }
-        }
-
-        return result;
-    }
-
-    public IISegment getSegment(String name, SegmentStatusEnum status) {
-        for (IISegment segment : segments) {
-            if ((null != segment.getName() && segment.getName().equals(name)) && segment.getStatus() == status) {
-                return segment;
-            }
-        }
-
-        return null;
-    }
-
-    public void setSegments(List<IISegment> segments) {
-        this.segments = segments;
-    }
-
-    public long getCreateTimeUTC() {
-        return createTimeUTC;
-    }
-
-    public void setCreateTimeUTC(long createTimeUTC) {
-        this.createTimeUTC = createTimeUTC;
-    }
-
-    @Override
-    public CapabilityResult isCapable(SQLDigest digest) {
-        CapabilityResult result = IICapabilityChecker.check(this, digest);
-        if (result.capable) {
-            result.cost = getCost(digest);
-        } else {
-            result.cost = -1;
-        }
-        return result;
-    }
-
-    private int getCost(SQLDigest digest) {
-
-        int calculatedCost = cost;
-        for (LookupDesc lookupDesc : this.getDescriptor().getModel().getLookups()) {
-            // more tables, more cost
-            calculatedCost += COST_WEIGHT_LOOKUP_TABLE;
-            if ("inner".equals(lookupDesc.getJoin().getType())) {
-                // inner join cost is bigger than left join, as it will filter some records
-                calculatedCost += COST_WEIGHT_INNER_JOIN;
-            }
-        }
-        return calculatedCost;
-    }
-
-    @Override
-    public RealizationType getType() {
-        return RealizationType.INVERTED_INDEX;
-    }
-
-    @Override
-    public List<TblColRef> getAllColumns() {
-        return getDescriptor().listAllColumns();
-    }
-
-    @Override
-    public String getFactTable() {
-        return getDescriptor().getFactTableName();
-    }
-
-    @Override
-    public List<MeasureDesc> getMeasures() {
-        return getDescriptor().getMeasures();
-    }
-
-    @Override
-    public long getDateRangeStart() {
-        List<IISegment> readySegs = getSegments(SegmentStatusEnum.READY);
-
-        long startTime = Long.MAX_VALUE;
-        for (IISegment seg : readySegs) {
-            if (seg.getDateRangeStart() < startTime)
-                startTime = seg.getDateRangeStart();
-        }
-
-        return startTime;
-    }
-
-    @Override
-    public long getDateRangeEnd() {
-
-        List<IISegment> readySegs = getSegments(SegmentStatusEnum.READY);
-
-        long endTime = 0;
-        for (IISegment seg : readySegs) {
-            if (seg.getDateRangeEnd() > endTime)
-                endTime = seg.getDateRangeEnd();
-        }
-
-        return endTime;
-    }
-
-    @Override
-    public String getModelName() {
-        return this.getDescriptor().getModelName();
-    }
-
-    @Override
-    public List<TblColRef> getAllDimensions() {
-        return getDescriptor().listAllDimensions();
-    }
-
-    public void setCost(int cost) {
-        this.cost = cost;
-    }
-
-    @Override
-    public int getStorageType() {
-        return IStorageAware.ID_HBASE;
-    }
-
-    @Override
-    public int getEngineType() {
-        return getDescriptor().getEngineType();
-    }
-
-    @Override
-    public int getSourceType() {
-        return getDataModelDesc().getFactTableDesc().getSourceType();
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/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
deleted file mode 100644
index e869328..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIManager.java
+++ /dev/null
@@ -1,272 +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;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.JsonSerializer;
-import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.kylin.metadata.project.ProjectInstance;
-import org.apache.kylin.metadata.project.ProjectManager;
-import org.apache.kylin.metadata.realization.IRealization;
-import org.apache.kylin.metadata.realization.IRealizationConstants;
-import org.apache.kylin.metadata.realization.IRealizationProvider;
-import org.apache.kylin.metadata.realization.RealizationType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-
-/**
- * @author honma
- */
-public class IIManager implements IRealizationProvider {
-
-    private static String ALPHA_NUM = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ";
-    private static int HBASE_TABLE_LENGTH = 10;
-
-    private static final Serializer<IIInstance> II_SERIALIZER = new JsonSerializer<IIInstance>(IIInstance.class);
-
-    private static final Logger logger = LoggerFactory.getLogger(IIManager.class);
-
-    // static cached instances
-    private static final ConcurrentHashMap<KylinConfig, IIManager> CACHE = new ConcurrentHashMap<KylinConfig, IIManager>();
-
-    public static IIManager getInstance(KylinConfig config) {
-        IIManager r = CACHE.get(config);
-        if (r != null) {
-            return r;
-        }
-
-        synchronized (IIManager.class) {
-            r = CACHE.get(config);
-            if (r != null) {
-                return r;
-            }
-            try {
-                r = new IIManager(config);
-                CACHE.put(config, r);
-                if (CACHE.size() > 1) {
-                    logger.warn("More than one singleton exist");
-                }
-                return r;
-            } catch (IOException e) {
-                throw new IllegalStateException("Failed to init IIManager from " + config, e);
-            }
-        }
-    }
-
-    public static void clearCache() {
-        CACHE.clear();
-    }
-
-    // ============================================================================
-
-    private KylinConfig config;
-    // ii name ==> IIInstance
-    private CaseInsensitiveStringCache<IIInstance> iiMap;
-
-    // for generation hbase table name of a new segment
-    private Multimap<String, String> usedStorageLocation = HashMultimap.create();
-
-    private IIManager(KylinConfig config) throws IOException {
-        logger.info("Initializing IIManager with config " + config);
-        this.config = config;
-        this.iiMap = new CaseInsensitiveStringCache<IIInstance>(config, Broadcaster.TYPE.INVERTED_INDEX);
-        loadAllIIInstance();
-    }
-
-    public List<IIInstance> listAllIIs() {
-        return new ArrayList<IIInstance>(iiMap.values());
-    }
-
-    public IIInstance getII(String iiName) {
-        return iiMap.get(iiName);
-    }
-
-    public List<IIInstance> getIIsByDesc(String descName) {
-
-        List<IIInstance> list = listAllIIs();
-        List<IIInstance> result = new ArrayList<IIInstance>();
-        Iterator<IIInstance> it = list.iterator();
-        while (it.hasNext()) {
-            IIInstance ci = it.next();
-            if (descName.equalsIgnoreCase(ci.getDescName())) {
-                result.add(ci);
-            }
-        }
-        return result;
-    }
-
-    public IIInstance createII(IIInstance ii) throws IOException {
-
-        if (this.getII(ii.getName()) != null)
-            throw new IllegalArgumentException("The II name '" + ii.getName() + "' already exists.");
-
-        this.updateII(ii);
-
-        // FIXME need to pass in project name
-        String projectName = ProjectInstance.DEFAULT_PROJECT_NAME;
-        ProjectManager.getInstance(config).moveRealizationToProject(RealizationType.INVERTED_INDEX, ii.getName(), projectName, ii.getOwner());
-        return ii;
-    }
-
-    public void reloadIILocal(String iiName) {
-        try {
-            reloadIILocalAt(IIInstance.concatResourcePath(iiName));
-        } catch (IOException e) {
-            logger.error(e.getLocalizedMessage(), e);
-        }
-    }
-
-    public IIInstance dropII(String iiName, boolean deleteDesc) throws IOException {
-        logger.info("Dropping II '" + iiName + "'");
-
-        IIInstance ii = getII(iiName);
-
-        if (deleteDesc && ii.getDescriptor() != null) {
-            IIDescManager.getInstance(config).removeIIDesc(ii.getDescriptor());
-        }
-
-        removeII(ii);
-        ProjectManager.getInstance(config).removeRealizationsFromProjects(RealizationType.INVERTED_INDEX, iiName);
-
-        return ii;
-    }
-
-    private void removeII(IIInstance ii) throws IOException {
-        getStore().deleteResource(ii.getResourcePath());
-        iiMap.remove(ii.getName());
-    }
-
-    public void removeIILocal(String name) {
-        iiMap.removeLocal(name);
-        usedStorageLocation.removeAll(name.toUpperCase());
-    }
-
-    public void updateII(IIInstance ii) throws IOException {
-        logger.info("Updating II instance : " + ii.getName());
-        getStore().putResource(ii.getResourcePath(), ii, II_SERIALIZER);
-        iiMap.put(ii.getName(), ii);
-
-        //this is a duplicate call to take care of scenarios where REST cache service unavailable
-        ProjectManager.getInstance(KylinConfig.getInstanceFromEnv()).clearL2Cache();
-    }
-
-    /**
-     * @param IIInstance
-     * @param startDate  (pass 0 if full build)
-     * @param endDate    (pass 0 if full build)
-     * @return
-     */
-    public IISegment buildSegment(IIInstance IIInstance, long startDate, long endDate) {
-        IISegment segment = new IISegment();
-        String incrementalSegName = IISegment.getSegmentName(startDate, endDate);
-        segment.setUuid(UUID.randomUUID().toString());
-        segment.setName(incrementalSegName);
-        segment.setCreateTimeUTC(System.currentTimeMillis());
-        segment.setDateRangeStart(startDate);
-        segment.setDateRangeEnd(endDate);
-        segment.setStatus(SegmentStatusEnum.NEW);
-        segment.setStorageLocationIdentifier(generateStorageLocation());
-
-        segment.setIIInstance(IIInstance);
-
-        return segment;
-    }
-
-    private String generateStorageLocation() {
-        while (true) {
-            StringBuilder sb = new StringBuilder(IRealizationConstants.IIHbaseStorageLocationPrefix);
-            for (int i = 0; i < HBASE_TABLE_LENGTH; i++) {
-                int idx = (int) (Math.random() * ALPHA_NUM.length());
-                sb.append(ALPHA_NUM.charAt(idx));
-            }
-            if (usedStorageLocation.containsValue(sb.toString())) {
-                continue;
-            } else {
-                return sb.toString();
-            }
-        }
-    }
-
-    private void loadAllIIInstance() throws IOException {
-        ResourceStore store = getStore();
-        List<String> paths = store.collectResourceRecursively(ResourceStore.II_RESOURCE_ROOT, ".json");
-
-        logger.debug("Loading II from folder " + store.getReadableResourcePath(ResourceStore.II_RESOURCE_ROOT));
-
-        for (String path : paths) {
-            reloadIILocalAt(path);
-        }
-
-        logger.debug("Loaded " + paths.size() + " II(s)");
-    }
-
-    private synchronized IIInstance reloadIILocalAt(String path) throws IOException {
-        ResourceStore store = getStore();
-        logger.debug("Loading IIInstance " + store.getReadableResourcePath(path));
-
-        IIInstance ii = null;
-        try {
-            ii = store.getResource(path, IIInstance.class, II_SERIALIZER);
-            ii.setConfig(config);
-
-            if (StringUtils.isBlank(ii.getName()))
-                throw new IllegalStateException("IIInstance name must not be blank");
-
-            iiMap.putLocal(ii.getName(), ii);
-
-            for (IISegment segment : ii.getSegments()) {
-                usedStorageLocation.put(ii.getName().toUpperCase(), segment.getStorageLocationIdentifier());
-            }
-
-            return ii;
-        } catch (Exception e) {
-            logger.error("Error during load ii instance " + path, e);
-            return null;
-        }
-    }
-
-    private ResourceStore getStore() {
-        return ResourceStore.getStore(this.config);
-    }
-
-    @Override
-    public RealizationType getRealizationType() {
-        return RealizationType.INVERTED_INDEX;
-    }
-
-    @Override
-    public IRealization getRealization(String name) {
-        return getII(name);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java
deleted file mode 100644
index 34fd67c..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java
+++ /dev/null
@@ -1,279 +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;
-
-import java.text.SimpleDateFormat;
-import java.util.List;
-import java.util.TimeZone;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
-import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
-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.IRealizationSegment;
-
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
-import com.fasterxml.jackson.annotation.JsonBackReference;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Objects;
-
-/**
- * @author honma
- */
-
-// TODO: remove segment concept for II, append old hbase table
-@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class IISegment implements Comparable<IISegment>, IRealizationSegment {
-
-    @JsonBackReference
-    private IIInstance iiInstance;
-    @JsonProperty("uuid")
-    private String uuid;
-    @JsonProperty("name")
-    private String name;
-    @JsonProperty("storage_location_identifier")
-    private String storageLocationIdentifier; // HTable name
-    @JsonProperty("date_range_start")
-    private long dateRangeStart;
-    @JsonProperty("date_range_end")
-    private long dateRangeEnd;
-    @JsonProperty("status")
-    private SegmentStatusEnum status;
-    @JsonProperty("size_kb")
-    private long sizeKB;
-    @JsonProperty("input_records")
-    private long inputRecords;
-    @JsonProperty("input_records_size")
-    private long inputRecordsSize;
-    @JsonProperty("last_build_time")
-    private long lastBuildTime;
-    @JsonProperty("last_build_job_id")
-    private String lastBuildJobID;
-
-    @JsonProperty("create_time_utc")
-    private long createTimeUTC;
-
-    @JsonProperty("binary_signature")
-    private String binarySignature; // a hash of schema and dictionary ID,
-    // used for sanity check
-
-    private transient TableRecordInfo tableRecordInfo;
-
-    /**
-     * @param startDate
-     * @param endDate
-     * @return if(startDate == 0 && endDate == 0), returns "FULL_BUILD", else
-     * returns "yyyyMMddHHmmss_yyyyMMddHHmmss"
-     */
-    public static String getSegmentName(long startDate, long endDate) {
-        if (startDate == 0 && endDate == 0) {
-            return "FULL_BUILD";
-        }
-
-        SimpleDateFormat dateFormat = new SimpleDateFormat("yyyyMMddHHmmss");
-        dateFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
-
-        return dateFormat.format(startDate) + "_" + dateFormat.format(endDate);
-    }
-
-    public IIDesc getIIDesc() {
-        return getIIInstance().getDescriptor();
-    }
-
-    // ============================================================================
-
-    @Override
-    public KylinConfig getConfig() {
-        return iiInstance.getConfig();
-    }
-
-    @Override
-    public String getUuid() {
-        return uuid;
-    }
-
-    public void setUuid(String id) {
-        this.uuid = id;
-    }
-
-    @Override
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public long getDateRangeStart() {
-        return dateRangeStart;
-    }
-
-    public void setDateRangeStart(long dateRangeStart) {
-        this.dateRangeStart = dateRangeStart;
-    }
-
-    public long getDateRangeEnd() {
-        return dateRangeEnd;
-    }
-
-    public void setDateRangeEnd(long dateRangeEnd) {
-        this.dateRangeEnd = dateRangeEnd;
-    }
-
-    public SegmentStatusEnum getStatus() {
-        return status;
-    }
-
-    public void setStatus(SegmentStatusEnum status) {
-        this.status = status;
-    }
-
-    public long getSizeKB() {
-        return sizeKB;
-    }
-
-    public void setSizeKB(long sizeKB) {
-        this.sizeKB = sizeKB;
-    }
-
-    public long getInputRecords() {
-        return inputRecords;
-    }
-
-    public void setInputRecords(long inputRecords) {
-        this.inputRecords = inputRecords;
-    }
-
-    public long getInputRecordsSize() {
-        return inputRecordsSize;
-    }
-
-    public void setInputRecordsSize(long inputRecordsSize) {
-        this.inputRecordsSize = inputRecordsSize;
-    }
-
-    public long getLastBuildTime() {
-        return lastBuildTime;
-    }
-
-    public void setLastBuildTime(long lastBuildTime) {
-        this.lastBuildTime = lastBuildTime;
-    }
-
-    public String getLastBuildJobID() {
-        return lastBuildJobID;
-    }
-
-    public void setLastBuildJobID(String lastBuildJobID) {
-        this.lastBuildJobID = lastBuildJobID;
-    }
-
-    public String getBinarySignature() {
-        return binarySignature;
-    }
-
-    public void setBinarySignature(String binarySignature) {
-        this.binarySignature = binarySignature;
-    }
-
-    public IIInstance getIIInstance() {
-        return iiInstance;
-    }
-
-    public void setIIInstance(IIInstance iiInstance) {
-        this.iiInstance = iiInstance;
-    }
-
-    @Override
-    public String getStorageLocationIdentifier() {
-        return storageLocationIdentifier;
-    }
-
-    /**
-     * @param storageLocationIdentifier the storageLocationIdentifier to set
-     */
-    public void setStorageLocationIdentifier(String storageLocationIdentifier) {
-        this.storageLocationIdentifier = storageLocationIdentifier;
-    }
-
-    @Override
-    public int compareTo(IISegment other) {
-        if (this.dateRangeEnd < other.dateRangeEnd) {
-            return -1;
-        } else if (this.dateRangeEnd > other.dateRangeEnd) {
-            return 1;
-        } else {
-            return 0;
-        }
-    }
-
-    private TableRecordInfo getTableRecordInfo() {
-        if (tableRecordInfo == null)
-            tableRecordInfo = new TableRecordInfo(this);
-        return tableRecordInfo;
-    }
-
-    public List<TblColRef> getColumns() {
-        return this.getTableRecordInfo().getColumns();
-    }
-
-    @Override
-    public String toString() {
-        return Objects.toStringHelper(this).add("uuid", uuid).add("create_time_utc:", createTimeUTC).add("name", name).add("last_build_job_id", lastBuildJobID).add("status", status).toString();
-    }
-
-    public long getCreateTimeUTC() {
-        return createTimeUTC;
-    }
-
-    public void setCreateTimeUTC(long createTimeUTC) {
-        this.createTimeUTC = createTimeUTC;
-    }
-
-    @Override
-    public int getEngineType() {
-        return 0;
-    }
-
-    @Override
-    public int getSourceType() {
-        return 0;
-    }
-
-    @Override
-    public int getStorageType() {
-        return 0;
-    }
-
-    @Override
-    public IRealization getRealization() {
-        return iiInstance;
-    }
-
-    @Override
-    public IJoinedFlatTableDesc getJoinedFlatTableDesc() {
-        return new IIJoinedFlatTableDesc(this.getIIDesc());
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BatchSliceMaker.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BatchSliceMaker.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BatchSliceMaker.java
deleted file mode 100644
index 952cb8f..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BatchSliceMaker.java
+++ /dev/null
@@ -1,95 +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.index;
-
-import java.util.List;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.invertedindex.model.IIDesc;
-
-import com.google.common.base.Preconditions;
-
-/**
- */
-public class BatchSliceMaker {
-
-    private final int nColumns;
-    private final int nRecordsCap;
-    private final short shard;
-    private final IIDesc desc;
-
-    private long sliceTimestamp;
-
-    transient ImmutableBytesWritable temp = new ImmutableBytesWritable();
-
-    public BatchSliceMaker(IIDesc desc, short shard) {
-        this.desc = desc;
-        this.nColumns = desc.listAllColumns().size();
-        this.nRecordsCap = Math.max(1, desc.getSliceSize());
-
-        this.shard = shard;
-        this.sliceTimestamp = Long.MIN_VALUE;
-    }
-
-    public Slice makeSlice(TableRecordInfoDigest digest, List<TableRecord> records) {
-        Preconditions.checkArgument(records != null && !records.isEmpty(), "records cannot be empty");
-        Preconditions.checkArgument(records.size() <= nRecordsCap, "batch count cannot exceed " + nRecordsCap);
-        sliceTimestamp = increaseSliceTimestamp(records.get(0).getTimestamp());
-        ColumnValueContainer[] containers = new ColumnValueContainer[nColumns];
-        for (int i : desc.getValueColumns()) {
-            containers[i] = new CompressedValueContainer(digest, i, nRecordsCap);
-        }
-        for (int i : desc.getMetricsColumns()) {
-            containers[i] = new CompressedValueContainer(digest, i, nRecordsCap);
-        }
-        for (TableRecord record : records) {
-            for (int i = 0; i < nColumns; i++) {
-                record.getValueBytes(i, temp);
-                containers[i].append(temp);
-            }
-        }
-        return new Slice(digest, shard, sliceTimestamp, containers);
-
-    }
-
-    private long increaseSliceTimestamp(long timestamp) {
-        if (timestamp <= sliceTimestamp) {
-            return sliceTimestamp + 1; // ensure slice timestamp increases
-        } else {
-            return timestamp;
-        }
-    }
-
-}


[53/55] [abbrv] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

Posted by sh...@apache.org.
KYLIN-1528 Create a branch for v1.5 with HBase 1.x API


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 7d9a59be2cba4fb0cb08f8a2ee28b7217ea54f52
Parents: 5d679f1
Author: shaofengshi <sh...@apache.org>
Authored: Wed Mar 23 17:07:05 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Jul 26 09:49:00 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfigBase.java   |  4 ++++
 .../kylin/provision/BuildCubeWithEngine.java       |  2 ++
 .../org/apache/kylin/rest/service/CubeService.java |  2 ++
 .../org/apache/kylin/rest/service/UserService.java |  1 -
 .../kylin/storage/hbase/HBaseConnection.java       |  1 -
 .../kylin/storage/hbase/HBaseResourceStore.java    | 12 +++++++-----
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java        | 17 +++++++++++++++++
 .../storage/hbase/steps/HBaseCuboidWriter.java     | 16 ++++++++--------
 .../storage/hbase/util/StorageCleanupJob.java      |  1 -
 9 files changed, 40 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/7d9a59be/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index b146031..87b7c22 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -781,4 +781,8 @@ abstract public class KylinConfigBase implements Serializable {
     public void setAppendDictCacheSize(int cacheSize) {
         setProperty("kylin.dict.append.cache.size", String.valueOf(cacheSize));
     }
+        
+    public String getPatchedFuzzyRowFilterVersion() {
+        return this.getOptional("kylin.hbase.filter.fuzzy.row.filter.version", "1.1.3");
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d9a59be/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
index 2a5979f..30e95d3 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
@@ -58,6 +58,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.metadata.model.IEngineAware;
+import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
 import org.apache.kylin.storage.hbase.util.StorageCleanupJob;

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d9a59be/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index 0503236..4e5d793 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.HTable;
 import java.util.*;
+import java.util.*;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d9a59be/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
index e039534..0e0e213 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
@@ -152,7 +152,6 @@ public class UserService implements UserDetailsManager {
             Put put = new Put(pair.getKey());
 
             put.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
-
             htable.put(put);
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d9a59be/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
index 7dbc7f0..e7ee2f5 100644
--- 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
@@ -283,4 +283,3 @@ public class HBaseConnection {
     }
 
 }
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d9a59be/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
index aa7a4d4..9881569 100644
--- 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
@@ -33,8 +33,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.*;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
@@ -324,9 +322,13 @@ public class HBaseResourceStore extends ResourceStore {
                 get.addColumn(B_FAMILY, B_COLUMN_TS);
         }
 
-        Result result = table.get(get);
-        boolean exists = result != null && (!result.isEmpty() || (result.getExists() != null && result.getExists()));
-        return exists ? result : null;
+        try {
+            Result result = table.get(get);
+            boolean exists = result != null && (!result.isEmpty() || (result.getExists() != null && result.getExists()));
+            return exists ? result : null;
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
     }
 
     private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, Table table) throws IOException {

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d9a59be/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
index d84074f..9f787ad 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
@@ -435,4 +435,21 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         return sb.toString();
 
     }
+
+    private Map<byte[], CubeVisitProtos.CubeVisitResponse> getResults(final CubeVisitProtos.CubeVisitRequest request, Table table, byte[] startKey, byte[] endKey) throws Throwable {
+        Map<byte[], CubeVisitProtos.CubeVisitResponse> results = table.coprocessorService(CubeVisitProtos.CubeVisitService.class, startKey, endKey, new Batch.Call<CubeVisitProtos.CubeVisitService, CubeVisitProtos.CubeVisitResponse>() {
+            public CubeVisitProtos.CubeVisitResponse call(CubeVisitProtos.CubeVisitService rowsService) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<CubeVisitProtos.CubeVisitResponse> rpcCallback = new BlockingRpcCallback<>();
+                rowsService.visitCube(controller, request, rpcCallback);
+                CubeVisitProtos.CubeVisitResponse response = rpcCallback.get();
+                if (controller.failedOnException()) {
+                    throw controller.getFailedOn();
+                }
+                return response;
+            }
+        });
+
+        return results;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d9a59be/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
index c990379..a4a43c9 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
@@ -128,15 +128,15 @@ public class HBaseCuboidWriter implements ICuboidWriter {
 
     @Override
     public final void flush() throws IOException {
-        if (!puts.isEmpty()) {
-            long t = System.currentTimeMillis();
-            if (hTable != null) {
-                hTable.put(puts);
+
+            if (!puts.isEmpty()) {
+                long t = System.currentTimeMillis();
+                if (hTable != null) {
+                    hTable.put(puts);
+                }
+                logger.info("commit total " + puts.size() + " puts, totally cost:" + (System.currentTimeMillis() - t) + "ms");
+                puts.clear();
             }
-            logger.info("commit total " + puts.size() + " puts, totally cost:" + (System.currentTimeMillis() - t) + "ms");
-            puts.clear();
-        }
-        puts.clear();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d9a59be/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
index 874121d..06eafa9 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.CliCommandExecutor;


[04/55] [abbrv] kylin git commit: KYLIN-1858 remove all ii related code

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileMapper.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileMapper.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileMapper.java
deleted file mode 100644
index 545d058..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileMapper.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage.hbase.ii;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.invertedindex.model.IIDesc;
-
-/**
- * @author yangli9
- */
-public class IICreateHFileMapper extends KylinMapper<ImmutableBytesWritable, ImmutableBytesWritable, ImmutableBytesWritable, KeyValue> {
-
-    long timestamp;
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        timestamp = System.currentTimeMillis();
-    }
-
-    @Override
-    protected void map(ImmutableBytesWritable key, ImmutableBytesWritable value, Context context) throws IOException, InterruptedException {
-
-        ByteBuffer buffer = ByteBuffer.wrap(value.get(), value.getOffset(), value.getLength());
-        int totalLength = value.getLength();
-        int valueLength = buffer.getInt();
-        int dictionaryLength = totalLength - valueLength - 4;
-        KeyValue kv = new KeyValue(key.get(), key.getOffset(), key.getLength(), //
-                IIDesc.HBASE_FAMILY_BYTES, 0, IIDesc.HBASE_FAMILY_BYTES.length, //
-                IIDesc.HBASE_QUALIFIER_BYTES, 0, IIDesc.HBASE_QUALIFIER_BYTES.length, //
-                timestamp, Type.Put, //
-                buffer.array(), buffer.position(), valueLength);
-
-        // write value
-        context.write(key, kv);
-
-        kv = new KeyValue(key.get(), key.getOffset(), key.getLength(), //
-                IIDesc.HBASE_FAMILY_BYTES, 0, IIDesc.HBASE_FAMILY_BYTES.length, //
-                IIDesc.HBASE_DICTIONARY_BYTES, 0, IIDesc.HBASE_DICTIONARY_BYTES.length, //
-                timestamp, Type.Put, //
-                buffer.array(), buffer.position() + valueLength, dictionaryLength);
-
-        // write dictionary
-        context.write(key, kv);
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHTableJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHTableJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHTableJob.java
deleted file mode 100644
index 8099276..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHTableJob.java
+++ /dev/null
@@ -1,158 +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.storage.hbase.ii;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.apache.kylin.metadata.realization.IRealizationConstants;
-import org.apache.kylin.storage.hbase.util.IIDeployCoprocessorCLI;
-
-/**
- * @author George Song (ysong1)
- */
-public class IICreateHTableJob extends AbstractHadoopJob {
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-        HBaseAdmin admin = null;
-        try {
-            options.addOption(OPTION_II_NAME);
-            options.addOption(OPTION_HTABLE_NAME);
-            parseOptions(options, args);
-
-            String tableName = getOptionValue(OPTION_HTABLE_NAME);
-            String iiName = getOptionValue(OPTION_II_NAME);
-
-            KylinConfig config = KylinConfig.getInstanceFromEnv();
-            IIManager iiManager = IIManager.getInstance(config);
-            IIInstance ii = iiManager.getII(iiName);
-            int sharding = ii.getDescriptor().getSharding();
-
-            Configuration conf = HBaseConfiguration.create(getConf());
-            // check if the table already exists
-            admin = new HBaseAdmin(conf);
-            if (admin.tableExists(tableName)) {
-                if (admin.isTableEnabled(tableName)) {
-                    logger.info("Table " + tableName + " already exists and is enabled, no need to create.");
-                    return 0;
-                } else {
-                    logger.error("Table " + tableName + " is disabled, couldn't append data");
-                    return 1;
-                }
-            }
-
-            // table doesn't exist, need to create
-
-            HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(tableName));
-            HColumnDescriptor cf = new HColumnDescriptor(IIDesc.HBASE_FAMILY);
-            cf.setMaxVersions(1);
-
-            String hbaseDefaultCC = config.getHbaseDefaultCompressionCodec().toLowerCase();
-
-            switch (hbaseDefaultCC) {
-            case "snappy": {
-                logger.info("hbase will use snappy to compress data");
-                cf.setCompressionType(Compression.Algorithm.SNAPPY);
-                break;
-            }
-            case "lzo": {
-                logger.info("hbase will use lzo to compress data");
-                cf.setCompressionType(Compression.Algorithm.LZO);
-                break;
-            }
-            case "gz":
-            case "gzip": {
-                logger.info("hbase will use gzip to compress data");
-                cf.setCompressionType(Compression.Algorithm.GZ);
-                break;
-            }
-            case "lz4": {
-                logger.info("hbase will use lz4 to compress data");
-                cf.setCompressionType(Compression.Algorithm.LZ4);
-                break;
-            }
-            default: {
-                logger.info("hbase will not user any compression codec to compress data");
-            }
-            }
-
-            cf.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
-            tableDesc.addFamily(cf);
-            tableDesc.setValue(IRealizationConstants.HTableTag, config.getMetadataUrlPrefix());
-            tableDesc.setValue(IRealizationConstants.HTableCreationTime, String.valueOf(System.currentTimeMillis()));
-            tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, DisabledRegionSplitPolicy.class.getName());
-
-            if (User.isHBaseSecurityEnabled(conf)) {
-                // add coprocessor for bulk load
-                tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
-            }
-
-            IIDeployCoprocessorCLI.deployCoprocessor(tableDesc);
-
-            // create table
-            byte[][] splitKeys = getSplits(sharding);
-            if (splitKeys.length == 0)
-                splitKeys = null;
-            admin.createTable(tableDesc, splitKeys);
-            if (splitKeys != null) {
-                for (int i = 0; i < splitKeys.length; i++) {
-                    logger.info("split key " + i + ": " + BytesUtil.toHex(splitKeys[i]));
-                }
-            }
-            logger.info("create hbase table " + tableName + " done.");
-
-            return 0;
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        } finally {
-            if (admin != null)
-                admin.close();
-        }
-    }
-
-    //one region for one shard
-    private byte[][] getSplits(int shard) {
-        byte[][] result = new byte[shard - 1][];
-        for (int i = 1; i < shard; ++i) {
-            byte[] split = new byte[IIKeyValueCodec.SHARD_LEN];
-            BytesUtil.writeUnsigned(i, split, 0, IIKeyValueCodec.SHARD_LEN);
-            result[i - 1] = split;
-        }
-        return result;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/InvertedIndexStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/InvertedIndexStorageQuery.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/InvertedIndexStorageQuery.java
deleted file mode 100644
index fef9662..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/InvertedIndexStorageQuery.java
+++ /dev/null
@@ -1,67 +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.storage.hbase.ii;
-
-import java.util.ArrayList;
-
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.metadata.realization.SQLDigest;
-import org.apache.kylin.metadata.tuple.ITupleIterator;
-import org.apache.kylin.metadata.tuple.TupleInfo;
-import org.apache.kylin.storage.IStorageQuery;
-import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.storage.hbase.HBaseConnection;
-import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.EndpointTupleIterator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author yangli9
- */
-public class InvertedIndexStorageQuery implements IStorageQuery {
-
-    private static Logger logger = LoggerFactory.getLogger(InvertedIndexStorageQuery.class);
-
-    private IISegment seg;
-    private String uuid;
-    private EndpointTupleIterator dataIterator;
-
-    public InvertedIndexStorageQuery(IIInstance ii) {
-        this.seg = ii.getFirstSegment();
-        this.uuid = ii.getUuid();
-    }
-
-    @Override
-    public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo) {
-        String tableName = seg.getStorageLocationIdentifier();
-
-        //HConnection is cached, so need not be closed
-        HConnection conn = HBaseConnection.get(context.getConnUrl());
-        try {
-            dataIterator = new EndpointTupleIterator(seg, sqlDigest.filter, sqlDigest.groupbyColumns, new ArrayList<>(sqlDigest.aggregations), context, conn, returnTupleInfo);
-            return dataIterator;
-        } catch (Throwable e) {
-            logger.error("Error when connecting to II htable " + tableName, e);
-            throw new IllegalStateException("Error when connecting to II htable " + tableName, e);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluator.java
deleted file mode 100644
index 1f024fe..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluator.java
+++ /dev/null
@@ -1,249 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import java.util.List;
-
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- * @author yangli9
- *
- * Evaluate a group of records against a filter in batch.
- */
-public class BitMapFilterEvaluator {
-
-    /** Provides bitmaps for a record group ranging [0..N-1], where N is the size of the group */
-    public static interface BitMapProvider {
-
-        /** return records whose specified column having specified value */
-        ConciseSet getBitMap(TblColRef col, Integer startId, Integer endId);
-
-        /** return the size of the group */
-        int getRecordCount();
-
-        /** return the max value ID of a column according to dictionary */
-        int getMaxValueId(TblColRef col);
-    }
-
-    BitMapProvider provider;
-
-    public BitMapFilterEvaluator(BitMapProvider bitMapProvider) {
-        this.provider = bitMapProvider;
-    }
-
-    /**
-     * @param filter
-     * @return a set of records that match the filter; or null if filter is null or unable to evaluate
-     */
-    public ConciseSet evaluate(TupleFilter filter) {
-        if (filter == null)
-            return null;
-
-        if (filter instanceof LogicalTupleFilter)
-            return evalLogical((LogicalTupleFilter) filter);
-
-        if (filter instanceof CompareTupleFilter)
-            return evalCompare((CompareTupleFilter) filter);
-
-        if (filter instanceof ConstantTupleFilter) {
-            if (!filter.evaluate(null, null)) {
-                return new ConciseSet();
-            }
-        }
-
-        return null; // unable to evaluate
-    }
-
-    private ConciseSet evalCompare(CompareTupleFilter filter) {
-        switch (filter.getOperator()) {
-        case ISNULL:
-            return evalCompareIsNull(filter);
-        case ISNOTNULL:
-            return evalCompareIsNotNull(filter);
-        case EQ:
-            return evalCompareEqual(filter);
-        case NEQ:
-            return evalCompareNotEqual(filter);
-        case IN:
-            return evalCompareIn(filter);
-        case NOTIN:
-            return evalCompareNotIn(filter);
-        case LT:
-            return evalCompareLT(filter);
-        case LTE:
-            return evalCompareLTE(filter);
-        case GT:
-            return evalCompareGT(filter);
-        case GTE:
-            return evalCompareGTE(filter);
-        default:
-            throw new IllegalStateException("Unsupported operator " + filter.getOperator());
-        }
-    }
-
-    private ConciseSet evalCompareLT(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId((String) filter.getFirstValue());
-        return collectRange(filter.getColumn(), null, id - 1);
-    }
-
-    private ConciseSet evalCompareLTE(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId((String) filter.getFirstValue());
-        return collectRange(filter.getColumn(), null, id);
-    }
-
-    private ConciseSet evalCompareGT(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId((String) filter.getFirstValue());
-        return collectRange(filter.getColumn(), id + 1, null);
-    }
-
-    private ConciseSet evalCompareGTE(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId((String) filter.getFirstValue());
-        return collectRange(filter.getColumn(), id, null);
-    }
-
-    private ConciseSet collectRange(TblColRef column, Integer startId, Integer endId) {
-        return provider.getBitMap(column, startId, endId);
-    }
-
-    private ConciseSet evalCompareEqual(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId((String) filter.getFirstValue());
-        ConciseSet bitMap = provider.getBitMap(filter.getColumn(), id, id);
-        if (bitMap == null)
-            return null;
-        return bitMap.clone(); // NOTE the clone() to void messing provider's cache
-    }
-
-    private ConciseSet evalCompareNotEqual(CompareTupleFilter filter) {
-        ConciseSet set = evalCompareEqual(filter);
-        not(set);
-        dropNull(set, filter);
-        return set;
-    }
-
-    private ConciseSet evalCompareIn(CompareTupleFilter filter) {
-        ConciseSet set = new ConciseSet();
-        for (Object value : filter.getValues()) {
-            int id = Dictionary.stringToDictId((String) value);
-            ConciseSet bitMap = provider.getBitMap(filter.getColumn(), id, id);
-            if (bitMap == null)
-                return null;
-            set.addAll(bitMap);
-        }
-        return set;
-    }
-
-    private ConciseSet evalCompareNotIn(CompareTupleFilter filter) {
-        ConciseSet set = evalCompareIn(filter);
-        not(set);
-        dropNull(set, filter);
-        return set;
-    }
-
-    private void dropNull(ConciseSet set, CompareTupleFilter filter) {
-        if (set == null)
-            return;
-
-        ConciseSet nullSet = evalCompareIsNull(filter);
-        set.removeAll(nullSet);
-    }
-
-    private ConciseSet evalCompareIsNull(CompareTupleFilter filter) {
-        ConciseSet bitMap = provider.getBitMap(filter.getColumn(), null, null);
-        if (bitMap == null)
-            return null;
-        return bitMap.clone(); // NOTE the clone() to void messing provider's cache
-    }
-
-    private ConciseSet evalCompareIsNotNull(CompareTupleFilter filter) {
-        ConciseSet set = evalCompareIsNull(filter);
-        not(set);
-        return set;
-    }
-
-    private ConciseSet evalLogical(LogicalTupleFilter filter) {
-        List<? extends TupleFilter> children = filter.getChildren();
-
-        switch (filter.getOperator()) {
-        case AND:
-            return evalLogicalAnd(children);
-        case OR:
-            return evalLogicalOr(children);
-        case NOT:
-            return evalLogicalNot(children);
-        default:
-            throw new IllegalStateException("Unsupported operator " + filter.getOperator());
-        }
-    }
-
-    private ConciseSet evalLogicalAnd(List<? extends TupleFilter> children) {
-        ConciseSet set = new ConciseSet();
-        not(set);
-
-        for (TupleFilter c : children) {
-            ConciseSet t = evaluate(c);
-            if (t == null)
-                continue; // because it's AND
-
-            set.retainAll(t);
-        }
-        return set;
-    }
-
-    private ConciseSet evalLogicalOr(List<? extends TupleFilter> children) {
-        ConciseSet set = new ConciseSet();
-
-        for (TupleFilter c : children) {
-            ConciseSet t = evaluate(c);
-            if (t == null)
-                return null; // because it's OR
-
-            set.addAll(t);
-        }
-        return set;
-    }
-
-    private ConciseSet evalLogicalNot(List<? extends TupleFilter> children) {
-        ConciseSet set = evaluate(children.get(0));
-        not(set);
-        return set;
-    }
-
-    private void not(ConciseSet set) {
-        if (set == null)
-            return;
-
-        set.add(provider.getRecordCount());
-        set.complement();
-    }
-
-    public static void main(String[] args) {
-        ConciseSet s = new ConciseSet();
-        s.add(5);
-        s.complement();
-        System.out.println(s);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/ClearTextDictionary.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/ClearTextDictionary.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/ClearTextDictionary.java
deleted file mode 100644
index 9039165..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/ClearTextDictionary.java
+++ /dev/null
@@ -1,66 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.dimension.DimensionEncoding;
-import org.apache.kylin.dimension.FixedLenDimEnc;
-import org.apache.kylin.dimension.IDimensionEncodingMap;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorRowType;
-
-import com.google.common.collect.Maps;
-
-/**
- */
-public class ClearTextDictionary implements IDimensionEncodingMap {
-
-    private final Map<TblColRef, DimensionEncoding> encMap;
-
-    public ClearTextDictionary(TableRecordInfoDigest digest, CoprocessorRowType coprocessorRowType) {
-        encMap = Maps.newHashMap();
-        for (Entry<TblColRef, Integer> entry : coprocessorRowType.columnIdxMap.entrySet()) {
-            encMap.put(entry.getKey(), new FixedLenDimEnc(digest.length(entry.getValue())));
-        }
-    }
-
-    public ClearTextDictionary(TableRecordInfo tableRecordInfo) {
-        encMap = Maps.newHashMap();
-        TableRecordInfoDigest digest = tableRecordInfo.getDigest();
-        for (int i = 0; i < tableRecordInfo.getColumns().size(); i++) {
-            encMap.put(tableRecordInfo.getColumns().get(i), new FixedLenDimEnc(digest.length(i)));
-        }
-    }
-
-    @Override
-    public DimensionEncoding get(TblColRef col) {
-        return encMap.get(col);
-    }
-
-    @Override
-    public Dictionary<String> getDictionary(TblColRef col) {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
deleted file mode 100644
index affb284..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
+++ /dev/null
@@ -1,46 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
-import org.apache.kylin.storage.hbase.common.coprocessor.AggregationCache;
-
-/**
- */
-public class EndpointAggregationCache extends AggregationCache {
-
-    private EndpointAggregators aggregators;
-
-    public EndpointAggregationCache(EndpointAggregators aggregators) {
-        this.aggregators = aggregators;
-    }
-
-    @Override
-    public MeasureAggregator[] createBuffer() {
-        return this.aggregators.createBuffer();
-    }
-
-    public Set<Map.Entry<AggrKey, MeasureAggregator[]>> getAllEntries() {
-        return aggBufMap.entrySet();
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
deleted file mode 100644
index e481272..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
+++ /dev/null
@@ -1,287 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.BytesSerializer;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.invertedindex.index.RawTableRecord;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
-import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.measure.hllc.HLLCMeasureType;
-import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.LongMutable;
-import org.apache.kylin.metadata.model.FunctionDesc;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-/**
- * @author honma
- */
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class EndpointAggregators {
-
-    private enum MetricType {
-        Count, DimensionAsMetric, DistinctCount, Normal
-    }
-
-    private final static class MetricInfo {
-        private MetricType type;
-        private int refIndex = -1;
-        private int precision = -1;
-
-        public MetricInfo(MetricType type, int refIndex, int precision) {
-            this.type = type;
-            this.refIndex = refIndex;
-            this.precision = precision;
-        }
-
-        public MetricInfo(MetricType type, int refIndex) {
-            this.type = type;
-            this.refIndex = refIndex;
-        }
-
-        public MetricInfo(MetricType type) {
-            this.type = type;
-        }
-
-    }
-
-    private static MetricInfo generateMetricInfo(TableRecordInfo tableInfo, FunctionDesc functionDesc) {
-        if (functionDesc.isCount()) {
-            return new MetricInfo(MetricType.Count);
-        } else if (functionDesc.isDimensionAsMetric()) {
-            return new MetricInfo(MetricType.DimensionAsMetric);
-        } else {
-            int index = tableInfo.findFactTableColumn(functionDesc.getParameter().getValue());
-            Preconditions.checkState(index >= 0, "Column " + functionDesc.getParameter().getValue() + " is not found in II");
-            if (HLLCMeasureType.isCountDistinct(functionDesc)) {
-                return new MetricInfo(MetricType.DistinctCount, index, functionDesc.getReturnDataType().getPrecision());
-            } else {
-                return new MetricInfo(MetricType.Normal, index);
-            }
-        }
-    }
-
-    public static EndpointAggregators fromFunctions(TableRecordInfo tableInfo, List<FunctionDesc> metrics) {
-        final int metricSize = metrics.size();
-        String[] funcNames = new String[metricSize];
-        String[] dataTypes = new String[metricSize];
-        MetricInfo[] metricInfos = new MetricInfo[metricSize];
-        for (int i = 0; i < metricSize; i++) {
-            FunctionDesc functionDesc = metrics.get(i);
-
-            //TODO: what if funcionDesc's type is different from tablDesc? cause scale difference
-            funcNames[i] = functionDesc.getExpression();
-            dataTypes[i] = functionDesc.getReturnType();
-            metricInfos[i] = generateMetricInfo(tableInfo, functionDesc);
-        }
-
-        return new EndpointAggregators(funcNames, dataTypes, metricInfos, tableInfo.getDigest());
-    }
-
-    final String[] funcNames;
-    final String[] dataTypes;
-    final MetricInfo[] metricInfos;
-
-    final transient TableRecordInfoDigest tableRecordInfoDigest;
-    final transient RawTableRecord rawTableRecord;
-    final transient ImmutableBytesWritable byteBuffer;
-    final transient HyperLogLogPlusCounter[] hllcs;
-    final transient FixedLenMeasureCodec[] measureSerializers;
-    final transient Object[] metricValues;
-
-    final LongMutable ONE = new LongMutable(1);
-
-    private EndpointAggregators(String[] funcNames, String[] dataTypes, MetricInfo[] metricInfos, TableRecordInfoDigest tableInfo) {
-        this.funcNames = funcNames;
-        this.dataTypes = dataTypes;
-        this.metricInfos = metricInfos;
-        this.tableRecordInfoDigest = tableInfo;
-        this.rawTableRecord = tableInfo.createTableRecordBytes();
-        this.byteBuffer = new ImmutableBytesWritable();
-
-        this.hllcs = new HyperLogLogPlusCounter[this.metricInfos.length];
-        this.metricValues = new Object[funcNames.length];
-        this.measureSerializers = new FixedLenMeasureCodec[funcNames.length];
-        for (int i = 0; i < this.measureSerializers.length; ++i) {
-            this.measureSerializers[i] = FixedLenMeasureCodec.get(DataType.getType(dataTypes[i]));
-        }
-    }
-
-    public TableRecordInfoDigest getTableRecordInfoDigest() {
-        return tableRecordInfoDigest;
-    }
-
-    public boolean isEmpty() {
-        return !((funcNames != null) && (funcNames.length != 0));
-    }
-
-    public MeasureAggregator[] createBuffer() {
-        MeasureAggregator[] aggrs = new MeasureAggregator[funcNames.length];
-        for (int i = 0; i < aggrs.length; i++) {
-            if (metricInfos[i].type == MetricType.DistinctCount) {
-                aggrs[i] = MeasureAggregator.create(funcNames[i], DataType.getType(dataTypes[i]));
-            } else {
-                //all other fixed length measures can be aggregated as long
-                aggrs[i] = MeasureAggregator.create(funcNames[i], DataType.getType("long"));
-            }
-        }
-        return aggrs;
-    }
-
-    /**
-     * this method is heavily called at coprocessor side,
-     * Make sure as little object creation as possible
-     */
-    public void aggregate(MeasureAggregator[] measureAggrs, byte[] row) {
-
-        rawTableRecord.setBytes(row, 0, row.length);
-
-        for (int metricIndex = 0; metricIndex < metricInfos.length; ++metricIndex) {
-            final MetricInfo metricInfo = metricInfos[metricIndex];
-            if (metricInfo.type == MetricType.Count) {
-                measureAggrs[metricIndex].aggregate(ONE);
-                continue;
-            }
-
-            if (metricInfo.type == MetricType.DimensionAsMetric) {
-                continue;
-            }
-
-            MeasureAggregator aggregator = measureAggrs[metricIndex];
-            FixedLenMeasureCodec measureSerializer = measureSerializers[metricIndex];
-
-            //get the raw bytes
-            rawTableRecord.getValueBytes(metricInfo.refIndex, byteBuffer);
-
-            if (metricInfo.type == MetricType.Normal) {
-                aggregator.aggregate(measureSerializer.read(byteBuffer.get(), byteBuffer.getOffset()));
-            } else if (metricInfo.type == MetricType.DistinctCount) {
-                //TODO: for unified dictionary, this is okay. but if different data blocks uses different dictionary, we'll have to aggregate original data
-                HyperLogLogPlusCounter hllc = hllcs[metricIndex];
-                if (hllc == null) {
-                    int precision = metricInfo.precision;
-                    hllc = new HyperLogLogPlusCounter(precision);
-                }
-                hllc.clear();
-                hllc.add(byteBuffer.get(), byteBuffer.getOffset(), byteBuffer.getLength());
-                aggregator.aggregate(hllc);
-            }
-        }
-    }
-
-    /**
-     * @param aggrs
-     * @param buffer byte buffer to get the metric data
-     * @return length of metric data
-     */
-    public int serializeMetricValues(MeasureAggregator[] aggrs, byte[] buffer, int offset) {
-        for (int i = 0; i < funcNames.length; i++) {
-            metricValues[i] = aggrs[i].getState();
-        }
-
-        int metricBytesOffset = offset;
-        int length = 0;
-        for (int i = 0; i < measureSerializers.length; i++) {
-            measureSerializers[i].write(metricValues[i], buffer, metricBytesOffset);
-            metricBytesOffset += measureSerializers[i].getLength();
-            length += measureSerializers[i].getLength();
-        }
-        return length;
-    }
-
-    public List<Object> deserializeMetricValues(ByteBuffer buffer) {
-        List<Object> ret = Lists.newArrayList();
-        for (int i = 0; i < measureSerializers.length; i++) {
-            measureSerializers[i].read(buffer);
-            Object valueString = measureSerializers[i].getValue();
-            ret.add(valueString);
-        }
-        return ret;
-    }
-
-    public static byte[] serialize(EndpointAggregators o) {
-        ByteBuffer buf = ByteBuffer.allocate(BytesSerializer.SERIALIZE_BUFFER_SIZE);
-        serializer.serialize(o, buf);
-        byte[] result = new byte[buf.position()];
-        System.arraycopy(buf.array(), 0, result, 0, buf.position());
-        return result;
-    }
-
-    public static EndpointAggregators deserialize(byte[] bytes) {
-        return serializer.deserialize(ByteBuffer.wrap(bytes));
-    }
-
-    private static final BytesSerializer<EndpointAggregators> serializer = new BytesSerializer<EndpointAggregators>() {
-
-        @Override
-        public void serialize(EndpointAggregators value, ByteBuffer out) {
-            BytesUtil.writeAsciiStringArray(value.funcNames, out);
-            BytesUtil.writeAsciiStringArray(value.dataTypes, out);
-
-            BytesUtil.writeVInt(value.metricInfos.length, out);
-            for (int i = 0; i < value.metricInfos.length; ++i) {
-                MetricInfo metricInfo = value.metricInfos[i];
-                BytesUtil.writeAsciiString(metricInfo.type.toString(), out);
-                BytesUtil.writeVInt(metricInfo.refIndex, out);
-                BytesUtil.writeVInt(metricInfo.precision, out);
-            }
-
-            BytesUtil.writeByteArray(TableRecordInfoDigest.serialize(value.tableRecordInfoDigest), out);
-        }
-
-        @Override
-        public EndpointAggregators deserialize(ByteBuffer in) {
-
-            String[] funcNames = BytesUtil.readAsciiStringArray(in);
-            String[] dataTypes = BytesUtil.readAsciiStringArray(in);
-
-            int metricInfoLength = BytesUtil.readVInt(in);
-            MetricInfo[] infos = new MetricInfo[metricInfoLength];
-            for (int i = 0; i < infos.length; ++i) {
-                MetricType type = MetricType.valueOf(BytesUtil.readAsciiString(in));
-                int refIndex = BytesUtil.readVInt(in);
-                int presision = BytesUtil.readVInt(in);
-                infos[i] = new MetricInfo(type, refIndex, presision);
-            }
-
-            byte[] temp = BytesUtil.readByteArray(in);
-            TableRecordInfoDigest tableInfo = TableRecordInfoDigest.deserialize(temp);
-
-            return new EndpointAggregators(funcNames, dataTypes, infos, tableInfo);
-        }
-
-    };
-
-    public int getMeasureSerializeLength() {
-        int length = 0;
-        for (int i = 0; i < this.measureSerializers.length; ++i) {
-            length += this.measureSerializers[i].getLength();
-        }
-        return length;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointEnabler.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointEnabler.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointEnabler.java
deleted file mode 100644
index 2ae7f35..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointEnabler.java
+++ /dev/null
@@ -1,53 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author honma
- */
-public class EndpointEnabler {
-
-    private static final Logger logger = LoggerFactory.getLogger(EndpointEnabler.class);
-
-    static final String FORCE_COPROCESSOR = "forceEndpoint";
-
-    public static boolean isCoprocessorBeneficial() {
-        return Boolean.parseBoolean(getForceCoprocessor());
-    }
-
-    public static void forceCoprocessorOn() {
-        System.setProperty(FORCE_COPROCESSOR, "true");
-    }
-
-    public static void forceCoprocessorOff() {
-        System.setProperty(FORCE_COPROCESSOR, "false");
-    }
-
-    public static String getForceCoprocessor() {
-        return System.getProperty(FORCE_COPROCESSOR);
-    }
-
-    public static void forceCoprocessorUnset() {
-        System.clearProperty(FORCE_COPROCESSOR);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointTupleConverter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointTupleConverter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointTupleConverter.java
deleted file mode 100644
index 3fdd5b0..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointTupleConverter.java
+++ /dev/null
@@ -1,94 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import java.util.List;
-
-import org.apache.kylin.invertedindex.index.TableRecord;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.tuple.ITuple;
-import org.apache.kylin.metadata.tuple.Tuple;
-import org.apache.kylin.metadata.tuple.TupleInfo;
-
-public class EndpointTupleConverter {
-
-    final TupleInfo tupleInfo;
-    final List<TblColRef> columns;
-    final int[] columnTupleIdx;
-    final int[] aggrTupleIdx;
-
-    public EndpointTupleConverter(List<TblColRef> columns, List<FunctionDesc> aggrMeasures, TupleInfo returnTupleInfo) {
-        this.tupleInfo = returnTupleInfo;
-        this.columns = columns;
-        this.columnTupleIdx = new int[columns.size()];
-        this.aggrTupleIdx = new int[aggrMeasures.size()];
-
-        for (int i = 0; i < columns.size(); i++) {
-            TblColRef col = columns.get(i);
-            columnTupleIdx[i] = tupleInfo.hasColumn(col) ? tupleInfo.getColumnIndex(col) : -1;
-        }
-
-        for (int i = 0; i < aggrMeasures.size(); i++) {
-            FunctionDesc measure = aggrMeasures.get(i);
-            int tupleIdx;
-            if (measure.isDimensionAsMetric()) {
-                // for dimension playing as metrics, the measure is just a placeholder, the real value comes from columns
-                tupleIdx = -1;
-            } else if (measure.needRewrite()) {
-                // a rewrite metrics is identified by its rewrite field name
-                String rewriteFieldName = measure.getRewriteFieldName();
-                tupleIdx = tupleInfo.hasField(rewriteFieldName) ? tupleInfo.getFieldIndex(rewriteFieldName) : -1;
-            } else {
-                // a non-rewrite metrics (i.e. sum) is like a dimension column
-                TblColRef col = measure.getParameter().getColRefs().get(0);
-                tupleIdx = tupleInfo.hasColumn(col) ? tupleInfo.getColumnIndex(col) : -1;
-            }
-            aggrTupleIdx[i] = tupleIdx;
-        }
-    }
-
-    public ITuple makeTuple(TableRecord tableRecord, List<Object> measureValues, Tuple tuple) {
-        // dimensions and metrics from II table record 
-        for (int i = 0; i < columnTupleIdx.length; i++) {
-            int tupleIdx = columnTupleIdx[i];
-            if (tupleIdx >= 0) {
-                String value = tableRecord.getValueString(i);
-                tuple.setDimensionValue(tupleIdx, value);
-            }
-        }
-
-        // additional aggregations calculated inside end point (like cube measures)
-        if (measureValues != null) {
-            for (int i = 0; i < aggrTupleIdx.length; ++i) {
-                int tupleIdx = aggrTupleIdx[i];
-                if (tupleIdx >= 0) {
-                    Object value = measureValues.get(i);
-                    if (value instanceof String) {
-                        String dataType = tuple.getDataTypeName(tupleIdx);
-                        value = Tuple.convertOptiqCellValue((String) value, dataType);
-                    }
-                    tuple.setMeasureValue(tupleIdx, value);
-                }
-            }
-        }
-        return tuple;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointTupleIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointTupleIterator.java
deleted file mode 100644
index e197e3e..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointTupleIterator.java
+++ /dev/null
@@ -1,364 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.SerializationUtils;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
-import org.apache.hadoop.hbase.ipc.ServerRpcController;
-import org.apache.kylin.common.util.CompressionUtils;
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.common.util.RangeUtil;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.index.TableRecord;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.measure.hllc.HLLCMeasureType;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.tuple.ITuple;
-import org.apache.kylin.metadata.tuple.ITupleIterator;
-import org.apache.kylin.metadata.tuple.Tuple;
-import org.apache.kylin.metadata.tuple.TupleInfo;
-import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.storage.cache.TsConditionExtractor;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorFilter;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorProjector;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorRowType;
-import org.apache.kylin.storage.hbase.common.coprocessor.FilterDecorator;
-import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Range;
-import com.google.common.collect.Ranges;
-import com.google.common.collect.Sets;
-import com.google.protobuf.HBaseZeroCopyByteString;
-
-/**
- */
-public class EndpointTupleIterator implements ITupleIterator {
-
-    private final static Logger logger = LoggerFactory.getLogger(EndpointTupleIterator.class);
-
-    private final IISegment seg;
-
-    private final String factTableName;
-    private final List<TblColRef> columns;
-    private final TupleInfo tupleInfo;
-    private final TableRecordInfo tableRecordInfo;
-    private final EndpointTupleConverter tupleConverter;
-
-    private final CoprocessorRowType pushedDownRowType;
-    private final CoprocessorFilter pushedDownFilter;
-    private final CoprocessorProjector pushedDownProjector;
-    private final EndpointAggregators pushedDownAggregators;
-    private final Range<Long> tsRange;//timestamp column condition's interval
-
-    private Iterator<List<IIProtos.IIResponseInternal.IIRow>> regionResponsesIterator = null;
-    private ITupleIterator tupleIterator = null;
-    private HTableInterface table = null;
-
-    private TblColRef partitionCol;
-    private long lastDataTime = -1;
-    private int rowsInAllMetric = 0;
-
-    public EndpointTupleIterator(IISegment segment, TupleFilter rootFilter, Collection<TblColRef> groupBy, List<FunctionDesc> measures, StorageContext context, HConnection conn, TupleInfo returnTupleInfo) throws Throwable {
-
-        String tableName = segment.getStorageLocationIdentifier();
-        table = conn.getTable(tableName);
-        factTableName = segment.getIIDesc().getFactTableName();
-
-        if (rootFilter == null) {
-            rootFilter = ConstantTupleFilter.TRUE;
-        }
-
-        if (groupBy == null) {
-            groupBy = Sets.newHashSet();
-        }
-
-        if (measures == null) {
-            measures = Lists.newArrayList();
-        }
-
-        //this method will change measures
-        rewriteMeasureParameters(measures, segment.getColumns());
-
-        this.seg = segment;
-        this.columns = segment.getColumns();
-
-        this.tupleInfo = returnTupleInfo;
-        this.tupleConverter = new EndpointTupleConverter(columns, measures, returnTupleInfo);
-        this.tableRecordInfo = new TableRecordInfo(this.seg);
-
-        this.pushedDownRowType = CoprocessorRowType.fromTableRecordInfo(tableRecordInfo, this.columns);
-        this.pushedDownFilter = CoprocessorFilter.fromFilter(new ClearTextDictionary(this.tableRecordInfo), rootFilter, FilterDecorator.FilterConstantsTreatment.AS_IT_IS);
-
-        for (TblColRef column : this.pushedDownFilter.getInevaluableColumns()) {
-            groupBy.add(column);
-        }
-
-        this.pushedDownProjector = CoprocessorProjector.makeForEndpoint(tableRecordInfo, groupBy);
-        this.pushedDownAggregators = EndpointAggregators.fromFunctions(tableRecordInfo, measures);
-
-        int tsCol = this.tableRecordInfo.getTimestampColumn();
-        this.partitionCol = this.columns.get(tsCol);
-        this.tsRange = TsConditionExtractor.extractTsCondition(this.partitionCol, rootFilter);
-
-        if (this.tsRange == null) {
-            logger.info("TsRange conflict for endpoint, return empty directly");
-            this.tupleIterator = ITupleIterator.EMPTY_TUPLE_ITERATOR;
-        } else {
-            logger.info("The tsRange being pushed is " + RangeUtil.formatTsRange(tsRange));
-        }
-
-        IIProtos.IIRequest endpointRequest = prepareRequest();
-        Collection<IIProtos.IIResponse> compressedShardResults = getResults(endpointRequest, table);
-
-        //decompress
-        Collection<IIProtos.IIResponseInternal> shardResults = new ArrayList<>();
-        for (IIProtos.IIResponse input : compressedShardResults) {
-            byte[] compressed = HBaseZeroCopyByteString.zeroCopyGetBytes(input.getBlob());
-            try {
-                byte[] decompressed = CompressionUtils.decompress(compressed);
-                shardResults.add(IIProtos.IIResponseInternal.parseFrom(decompressed));
-            } catch (Exception e) {
-                throw new RuntimeException("decompress endpoint response error");
-            }
-        }
-
-        this.lastDataTime = Collections.min(Collections2.transform(shardResults, new Function<IIProtos.IIResponseInternal, Long>() {
-            @Nullable
-            @Override
-            public Long apply(IIProtos.IIResponseInternal input) {
-
-                IIProtos.IIResponseInternal.Stats status = input.getStats();
-                logger.info("Endpoints all returned, stats from shard {}: start moment:{}, finish moment: {}, elapsed ms: {}, scanned slices: {}, latest slice time is {}", //
-                        new Object[] { String.valueOf(status.getMyShard()), //
-                                DateFormat.formatToTimeStr(status.getServiceStartTime()), //
-                                DateFormat.formatToTimeStr(status.getServiceEndTime()), //
-                                String.valueOf(status.getServiceEndTime() - status.getServiceStartTime()), //
-                                String.valueOf(status.getScannedSlices()), DateFormat.formatToTimeStr(status.getLatestDataTime()) });
-
-                return status.getLatestDataTime();
-            }
-        }));
-
-        this.regionResponsesIterator = Collections2.transform(shardResults, new Function<IIProtos.IIResponseInternal, List<IIProtos.IIResponseInternal.IIRow>>() {
-            @Nullable
-            @Override
-            public List<IIProtos.IIResponseInternal.IIRow> apply(@Nullable IIProtos.IIResponseInternal input) {
-                return input.getRowsList();
-            }
-        }).iterator();
-
-        if (this.regionResponsesIterator.hasNext()) {
-            this.tupleIterator = new SingleRegionTupleIterator(this.regionResponsesIterator.next());
-        } else {
-            this.tupleIterator = ITupleIterator.EMPTY_TUPLE_ITERATOR;
-        }
-    }
-
-    /**
-     * measure comes from query engine, does not contain enough information
-     */
-    private void rewriteMeasureParameters(List<FunctionDesc> measures, List<TblColRef> columns) {
-        for (FunctionDesc functionDesc : measures) {
-            if (functionDesc.isCount()) {
-                functionDesc.setReturnType("bigint");
-            } else {
-                boolean updated = false;
-                for (TblColRef column : columns) {
-                    if (column.isSameAs(factTableName, functionDesc.getParameter().getValue())) {
-                        if (HLLCMeasureType.isCountDistinct(functionDesc)) {
-                            //TODO: default precision might need be configurable
-                            String iiDefaultHLLC = "hllc10";
-                            functionDesc.setReturnType(iiDefaultHLLC);
-                        } else {
-                            functionDesc.setReturnType(column.getColumnDesc().getType().toString());
-                        }
-                        functionDesc.getParameter().setColRefs(ImmutableList.of(column));
-                        updated = true;
-                        break;
-                    }
-                }
-                if (!updated) {
-                    throw new RuntimeException("Func " + functionDesc + " is not related to any column in fact table " + factTableName);
-                }
-            }
-        }
-    }
-
-    @Override
-    public boolean hasNext() {
-        while (!this.tupleIterator.hasNext()) {
-            if (this.regionResponsesIterator.hasNext()) {
-                this.tupleIterator = new SingleRegionTupleIterator(this.regionResponsesIterator.next());
-            } else {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    @Override
-    public ITuple next() {
-        rowsInAllMetric++;
-
-        if (!hasNext()) {
-            throw new IllegalStateException("No more ITuple in EndpointTupleIterator");
-        }
-
-        ITuple tuple = this.tupleIterator.next();
-        return tuple;
-    }
-
-    @Override
-    public void remove() {
-        throw new UnsupportedOperationException();
-
-    }
-
-    @Override
-    public void close() {
-        IOUtils.closeQuietly(table);
-        logger.info("Closed after " + rowsInAllMetric + " rows are fetched");
-    }
-
-    /**
-     * tells storage layer cache what time period of data should not be cached.
-     * for static storage like cube, it will return null
-     * for dynamic storage like ii, it will for example exclude the last two minutes for possible data latency
-     * @return
-     */
-    public Range<Long> getCacheExcludedPeriod() {
-        Preconditions.checkArgument(lastDataTime != -1, "lastDataTime is not set yet");
-        return Ranges.greaterThan(lastDataTime);
-    }
-
-    private IIProtos.IIRequest prepareRequest() throws IOException {
-        IIProtos.IIRequest.Builder builder = IIProtos.IIRequest.newBuilder();
-
-        if (this.tsRange != null) {
-            byte[] tsRangeBytes = SerializationUtils.serialize(this.tsRange);
-            builder.setTsRange(HBaseZeroCopyByteString.wrap(tsRangeBytes));
-        }
-
-        builder.setType(HBaseZeroCopyByteString.wrap(CoprocessorRowType.serialize(pushedDownRowType))) //
-                .setFilter(HBaseZeroCopyByteString.wrap(CoprocessorFilter.serialize(pushedDownFilter))) //
-                .setProjector(HBaseZeroCopyByteString.wrap(CoprocessorProjector.serialize(pushedDownProjector))) //
-                .setAggregator(HBaseZeroCopyByteString.wrap(EndpointAggregators.serialize(pushedDownAggregators)));
-
-        IIProtos.IIRequest request = builder.build();
-
-        return request;
-    }
-
-    private Collection<IIProtos.IIResponse> getResults(final IIProtos.IIRequest request, HTableInterface table) throws Throwable {
-        Map<byte[], IIProtos.IIResponse> results = table.coprocessorService(IIProtos.RowsService.class, null, null, new Batch.Call<IIProtos.RowsService, IIProtos.IIResponse>() {
-            public IIProtos.IIResponse call(IIProtos.RowsService rowsService) throws IOException {
-                ServerRpcController controller = new ServerRpcController();
-                BlockingRpcCallback<IIProtos.IIResponse> rpcCallback = new BlockingRpcCallback<>();
-                rowsService.getRows(controller, request, rpcCallback);
-                IIProtos.IIResponse response = rpcCallback.get();
-                if (controller.failedOnException()) {
-                    throw controller.getFailedOn();
-                }
-
-                return response;
-            }
-        });
-
-        return results.values();
-    }
-
-    /**
-     * Internal class to handle iterators for a single region's returned rows
-     */
-    class SingleRegionTupleIterator implements ITupleIterator {
-        private List<IIProtos.IIResponseInternal.IIRow> rows;
-        private int index = 0;
-
-        //not thread safe!
-        private TableRecord tableRecord;
-        private List<Object> measureValues;
-        private Tuple tuple;
-
-        public SingleRegionTupleIterator(List<IIProtos.IIResponseInternal.IIRow> rows) {
-            this.rows = rows;
-            this.index = 0;
-            this.tableRecord = tableRecordInfo.createTableRecord();
-            this.tuple = new Tuple(tupleInfo);
-        }
-
-        @Override
-        public boolean hasNext() {
-            return index < rows.size();
-        }
-
-        @Override
-        public ITuple next() {
-            if (!hasNext()) {
-                throw new IllegalStateException("No more Tuple in the SingleRegionTupleIterator");
-            }
-
-            IIProtos.IIResponseInternal.IIRow currentRow = rows.get(index);
-            byte[] columnsBytes = HBaseZeroCopyByteString.zeroCopyGetBytes(currentRow.getColumns());
-            this.tableRecord.setBytes(columnsBytes, 0, columnsBytes.length);
-            if (currentRow.hasMeasures()) {
-                ByteBuffer buffer = currentRow.getMeasures().asReadOnlyByteBuffer();
-                this.measureValues = pushedDownAggregators.deserializeMetricValues(buffer);
-            }
-
-            index++;
-
-            return tupleConverter.makeTuple(this.tableRecord, this.measureValues, this.tuple);
-        }
-
-        @Override
-        public void remove() {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void close() {
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/HbaseServerKVIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/HbaseServerKVIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/HbaseServerKVIterator.java
deleted file mode 100644
index a1d0e35..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/HbaseServerKVIterator.java
+++ /dev/null
@@ -1,104 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.kylin.invertedindex.model.IIRow;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-public class HbaseServerKVIterator implements Iterable<IIRow>, Closeable {
-
-    private RegionScanner innerScanner;
-    private Logger logger = LoggerFactory.getLogger(HbaseServerKVIterator.class);
-
-    public HbaseServerKVIterator(RegionScanner innerScanner) {
-        this.innerScanner = innerScanner;
-    }
-
-    @Override
-    public void close() throws IOException {
-        IOUtils.closeQuietly(this.innerScanner);
-    }
-
-    private static class IIRowIterator implements Iterator<IIRow> {
-
-        private final RegionScanner regionScanner;
-        private final IIRow row = new IIRow();
-        List<Cell> results = Lists.newArrayList();
-
-        private boolean hasMore;
-
-        IIRowIterator(RegionScanner innerScanner) {
-            this.regionScanner = innerScanner;
-            try {
-                hasMore = regionScanner.nextRaw(results);
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
-        }
-
-        @Override
-        public boolean hasNext() {
-            return !results.isEmpty();
-        }
-
-        @Override
-        public IIRow next() {
-            if (results.size() < 1) {
-                throw new NoSuchElementException();
-            }
-            for (Cell c : results) {
-                row.updateWith(c);
-            }
-            results.clear();
-            try {
-                if (hasMore) {
-                    hasMore = regionScanner.nextRaw(results);
-                }
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
-            return row;
-        }
-
-        @Override
-        public void remove() {
-            throw new UnsupportedOperationException();
-        }
-    }
-
-    @Override
-    public Iterator<IIRow> iterator() {
-        return new IIRowIterator(innerScanner);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
deleted file mode 100644
index ef7de3a..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
+++ /dev/null
@@ -1,336 +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.storage.hbase.ii.coprocessor.endpoint;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.commons.lang.SerializationUtils;
-import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.protobuf.ResponseConverter;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.kylin.common.util.Array;
-import org.apache.kylin.common.util.BytesSerializer;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.CompressionUtils;
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.cube.kv.RowKeyColumnIO;
-import org.apache.kylin.dict.TrieDictionary;
-import org.apache.kylin.dimension.FixedLenDimEnc;
-import org.apache.kylin.invertedindex.index.RawTableRecord;
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorFilter;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorProjector;
-import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorRowType;
-import org.apache.kylin.storage.hbase.common.coprocessor.FilterDecorator;
-import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Range;
-import com.google.protobuf.HBaseZeroCopyByteString;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- */
-public class IIEndpoint extends IIProtos.RowsService implements Coprocessor, CoprocessorService {
-
-    private static final Logger logger = LoggerFactory.getLogger(IIEndpoint.class);
-    private static final int MEMORY_LIMIT = 500 * 1024 * 1024;
-
-    private RegionCoprocessorEnvironment env;
-    private long serviceStartTime;
-    private int shard;
-
-    public IIEndpoint() {
-    }
-
-    private Scan prepareScan(IIProtos.IIRequest request, HRegion region) throws IOException {
-        Scan scan = new Scan();
-
-        scan.addColumn(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
-        scan.addColumn(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_DICTIONARY_BYTES);
-
-        if (request.hasTsRange()) {
-            Range<Long> tsRange = (Range<Long>) SerializationUtils.deserialize(HBaseZeroCopyByteString.zeroCopyGetBytes(request.getTsRange()));
-            byte[] regionStartKey = region.getStartKey();
-            if (!ArrayUtils.isEmpty(regionStartKey)) {
-                shard = BytesUtil.readUnsigned(regionStartKey, 0, IIKeyValueCodec.SHARD_LEN);
-            } else {
-                shard = 0;
-            }
-            logger.info("Start key of the region is: " + BytesUtil.toReadableText(regionStartKey) + ", making shard to be :" + shard);
-
-            if (tsRange.hasLowerBound()) {
-                //differentiate GT and GTE seems not very beneficial
-                Preconditions.checkArgument(shard != -1, "Shard is -1!");
-                long tsStart = tsRange.lowerEndpoint();
-                logger.info("ts start is " + tsStart);
-
-                byte[] idealStartKey = new byte[IIKeyValueCodec.SHARD_LEN + IIKeyValueCodec.TIMEPART_LEN];
-                BytesUtil.writeUnsigned(shard, idealStartKey, 0, IIKeyValueCodec.SHARD_LEN);
-                BytesUtil.writeLong(tsStart, idealStartKey, IIKeyValueCodec.SHARD_LEN, IIKeyValueCodec.TIMEPART_LEN);
-                logger.info("ideaStartKey is(readable) :" + BytesUtil.toReadableText(idealStartKey));
-                Result result = region.getClosestRowBefore(idealStartKey, IIDesc.HBASE_FAMILY_BYTES);
-                if (result != null) {
-                    byte[] actualStartKey = Arrays.copyOf(result.getRow(), IIKeyValueCodec.SHARD_LEN + IIKeyValueCodec.TIMEPART_LEN);
-                    scan.setStartRow(actualStartKey);
-                    logger.info("The start key is set to " + BytesUtil.toReadableText(actualStartKey));
-                } else {
-                    logger.info("There is no key before ideaStartKey so ignore tsStart");
-                }
-            }
-
-            if (tsRange.hasUpperBound()) {
-                //differentiate LT and LTE seems not very beneficial
-                Preconditions.checkArgument(shard != -1, "Shard is -1");
-                long tsEnd = tsRange.upperEndpoint();
-                logger.info("ts end is " + tsEnd);
-
-                byte[] actualEndKey = new byte[IIKeyValueCodec.SHARD_LEN + IIKeyValueCodec.TIMEPART_LEN];
-                BytesUtil.writeUnsigned(shard, actualEndKey, 0, IIKeyValueCodec.SHARD_LEN);
-                BytesUtil.writeLong(tsEnd + 1, actualEndKey, IIKeyValueCodec.SHARD_LEN, IIKeyValueCodec.TIMEPART_LEN);//notice +1 here
-                scan.setStopRow(actualEndKey);
-                logger.info("The stop key is set to " + BytesUtil.toReadableText(actualEndKey));
-            }
-        }
-
-        return scan;
-    }
-
-    @Override
-    public void getRows(RpcController controller, IIProtos.IIRequest request, RpcCallback<IIProtos.IIResponse> done) {
-
-        this.serviceStartTime = System.currentTimeMillis();
-
-        RegionScanner innerScanner = null;
-        HRegion region = null;
-
-        try {
-            region = env.getRegion();
-            region.startRegionOperation();
-
-            innerScanner = region.getScanner(prepareScan(request, region));
-
-            CoprocessorRowType type = CoprocessorRowType.deserialize(HBaseZeroCopyByteString.zeroCopyGetBytes(request.getType()));
-            CoprocessorProjector projector = CoprocessorProjector.deserialize(HBaseZeroCopyByteString.zeroCopyGetBytes(request.getProjector()));
-            EndpointAggregators aggregators = EndpointAggregators.deserialize(HBaseZeroCopyByteString.zeroCopyGetBytes(request.getAggregator()));
-            CoprocessorFilter filter = CoprocessorFilter.deserialize(HBaseZeroCopyByteString.zeroCopyGetBytes(request.getFilter()));
-
-            //compression
-            IIProtos.IIResponseInternal response = getResponse(innerScanner, type, projector, aggregators, filter);
-            byte[] compressed = CompressionUtils.compress(response.toByteArray());
-            IIProtos.IIResponse compressedR = IIProtos.IIResponse.newBuilder().setBlob(HBaseZeroCopyByteString.wrap(compressed)).build();
-
-            done.run(compressedR);
-        } catch (IOException ioe) {
-            logger.error(ioe.toString());
-            ResponseConverter.setControllerException(controller, ioe);
-        } finally {
-            IOUtils.closeQuietly(innerScanner);
-            if (region != null) {
-                try {
-                    region.closeRegionOperation();
-                } catch (IOException e) {
-                    e.printStackTrace();
-                    throw new RuntimeException(e);
-                }
-            }
-        }
-    }
-
-    public IIProtos.IIResponseInternal getResponse(RegionScanner innerScanner, CoprocessorRowType type, CoprocessorProjector projector, EndpointAggregators aggregators, CoprocessorFilter filter) {
-
-        TableRecordInfoDigest tableRecordInfoDigest = aggregators.getTableRecordInfoDigest();
-
-        IIProtos.IIResponseInternal response;
-
-        synchronized (innerScanner) {
-            IIKeyValueCodec codec = new IIKeyValueCodec(tableRecordInfoDigest);
-            //TODO pass projector to codec to skip loading columns
-            Iterable<Slice> slices = codec.decodeKeyValue(new HbaseServerKVIterator(innerScanner));
-
-            response = getResponseInternal(slices, tableRecordInfoDigest, filter, type, projector, aggregators);
-        }
-        return response;
-    }
-
-    private IIProtos.IIResponseInternal getResponseInternal(Iterable<Slice> slices, TableRecordInfoDigest recordInfo, CoprocessorFilter filter, CoprocessorRowType type, CoprocessorProjector projector, EndpointAggregators aggregators) {
-        boolean needAgg = projector.hasGroupby() || !aggregators.isEmpty();
-
-        //for needAgg use
-        EndpointAggregationCache aggCache = new EndpointAggregationCache(aggregators);
-        //for no needAgg use
-        final int byteFormLen = recordInfo.getByteFormLen();
-        int totalByteFormLen = 0;
-
-        IIProtos.IIResponseInternal.Builder responseBuilder = IIProtos.IIResponseInternal.newBuilder();
-        ClearTextDictionary clearTextDictionary = new ClearTextDictionary(recordInfo, type);
-        RowKeyColumnIO rowKeyColumnIO = new RowKeyColumnIO(clearTextDictionary);
-
-        byte[] recordBuffer = new byte[recordInfo.getByteFormLen()];
-        byte[] buffer = new byte[BytesSerializer.SERIALIZE_BUFFER_SIZE];
-
-        int iteratedSliceCount = 0;
-        long latestSliceTs = Long.MIN_VALUE;
-        for (Slice slice : slices) {
-            latestSliceTs = slice.getTimestamp();
-            iteratedSliceCount++;
-
-            //dictionaries for fact table columns can not be determined while streaming.
-            //a piece of dict coincide with each Slice, we call it "local dict"
-            final Dictionary<?>[] localDictionaries = slice.getLocalDictionaries();
-            CoprocessorFilter newFilter;
-            final boolean emptyDictionary = Array.isEmpty(localDictionaries);
-            if (emptyDictionary) {
-                newFilter = filter;
-            } else {
-                for (Dictionary<?> localDictionary : localDictionaries) {
-                    if (localDictionary instanceof TrieDictionary) {
-                        ((TrieDictionary) localDictionary).enableIdToValueBytesCache();
-                    }
-                }
-                newFilter = CoprocessorFilter.fromFilter(new LocalDictionary(localDictionaries, type, slice.getInfo()), filter.getFilter(), FilterDecorator.FilterConstantsTreatment.REPLACE_WITH_LOCAL_DICT);
-            }
-
-            ConciseSet result = null;
-            if (filter != null) {
-                result = new BitMapFilterEvaluator(new SliceBitMapProvider(slice, type)).evaluate(newFilter.getFilter());
-            }
-
-            Iterator<RawTableRecord> iterator = slice.iterateWithBitmap(result);
-
-            TblColRef[] columns = type.columns;
-            int[] finalColumnLength = new int[columns.length];
-            for (int i = 0; i < columns.length; ++i) {
-                finalColumnLength[i] = rowKeyColumnIO.getColumnLength(columns[i]);
-            }
-
-            while (iterator.hasNext()) {
-                final RawTableRecord rawTableRecord = iterator.next();
-                decodeWithDictionary(recordBuffer, rawTableRecord, localDictionaries, recordInfo, rowKeyColumnIO, finalColumnLength);
-
-                if (needAgg) {
-                    //if has group by, group them first, and extract entries later
-                    AggrKey aggKey = projector.getAggrKey(recordBuffer);
-                    MeasureAggregator[] bufs = aggCache.getBuffer(aggKey);
-                    aggregators.aggregate(bufs, recordBuffer);
-                    aggCache.checkMemoryUsage();
-                } else {
-                    //otherwise directly extract entry and put into response
-                    if (totalByteFormLen >= MEMORY_LIMIT) {
-                        throw new RuntimeException("the query has exceeded the memory limit, please check the query");
-                    }
-                    IIProtos.IIResponseInternal.IIRow.Builder rowBuilder = IIProtos.IIResponseInternal.IIRow.newBuilder().setColumns(HBaseZeroCopyByteString.wrap(recordBuffer));
-                    responseBuilder.addRows(rowBuilder.build());
-                    totalByteFormLen += byteFormLen;
-                }
-            }
-        }
-
-        logger.info("Iterated Slices count: " + iteratedSliceCount);
-
-        if (needAgg) {
-            int offset = 0;
-            int measureLength = aggregators.getMeasureSerializeLength();
-            for (Map.Entry<AggrKey, MeasureAggregator[]> entry : aggCache.getAllEntries()) {
-                AggrKey aggrKey = entry.getKey();
-                IIProtos.IIResponseInternal.IIRow.Builder rowBuilder = IIProtos.IIResponseInternal.IIRow.newBuilder().setColumns(HBaseZeroCopyByteString.wrap(aggrKey.get(), aggrKey.offset(), aggrKey.length()));
-                if (offset + measureLength > buffer.length) {
-                    buffer = new byte[BytesSerializer.SERIALIZE_BUFFER_SIZE];
-                    offset = 0;
-                }
-                int length = aggregators.serializeMetricValues(entry.getValue(), buffer, offset);
-                rowBuilder.setMeasures(HBaseZeroCopyByteString.wrap(buffer, offset, length));
-                offset += length;
-                responseBuilder.addRows(rowBuilder.build());
-            }
-        }
-
-        responseBuilder.setStats(IIProtos.IIResponseInternal.Stats.newBuilder().setLatestDataTime(latestSliceTs).setServiceStartTime(this.serviceStartTime).setServiceEndTime(System.currentTimeMillis()).setScannedSlices(iteratedSliceCount));
-        return responseBuilder.build();
-    }
-
-    private void decodeWithDictionary(byte[] recordBuffer, RawTableRecord encodedRecord, Dictionary<?>[] localDictionaries, TableRecordInfoDigest digest, RowKeyColumnIO rowKeyColumnIO, int[] finalColumnLengths) {
-        final boolean[] isMetric = digest.isMetrics();
-        final boolean emptyDictionary = Array.isEmpty(localDictionaries);
-        for (int i = 0; i < finalColumnLengths.length; i++) {
-            if (isMetric[i]) {
-                writeColumnWithoutDictionary(encodedRecord.getBytes(), encodedRecord.offset(i), encodedRecord.length(i), recordBuffer, digest.offset(i), finalColumnLengths[i]);
-            } else {
-                if (emptyDictionary) {
-                    writeColumnWithoutDictionary(encodedRecord.getBytes(), encodedRecord.offset(i), encodedRecord.length(i), recordBuffer, digest.offset(i), finalColumnLengths[i]);
-                } else {
-                    final Dictionary<?> localDictionary = localDictionaries[i];
-                    final byte[] valueBytesFromId = localDictionary.getValueBytesFromId(encodedRecord.getValueID(i));
-                    writeColumnWithoutDictionary(valueBytesFromId, 0, valueBytesFromId.length, recordBuffer, digest.offset(i), finalColumnLengths[i]);
-                }
-            }
-        }
-    }
-
-    private void writeColumnWithoutDictionary(byte[] src, int srcOffset, int srcLength, byte[] dst, int dstOffset, int dstLength) {
-        if (srcLength >= dstLength) {
-            System.arraycopy(src, srcOffset, dst, dstOffset, dstLength);
-        } else {
-            System.arraycopy(src, srcOffset, dst, dstOffset, srcLength);
-            Arrays.fill(dst, dstOffset + srcLength, dstOffset + dstLength, FixedLenDimEnc.ROWKEY_PLACE_HOLDER_BYTE);
-        }
-    }
-
-    @Override
-    public void start(CoprocessorEnvironment env) throws IOException {
-        if (env instanceof RegionCoprocessorEnvironment) {
-            this.env = (RegionCoprocessorEnvironment) env;
-        } else {
-            throw new CoprocessorException("Must be loaded on a table region!");
-        }
-    }
-
-    @Override
-    public void stop(CoprocessorEnvironment env) throws IOException {
-    }
-
-    @Override
-    public Service getService() {
-        return this;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2cc0b9c4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIResponseAdapter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIResponseAdapter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIResponseAdapter.java
deleted file mode 100644
index e62f41f..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIResponseAdapter.java
+++ /dev/null
@@ -1,24 +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.storage.hbase.ii.coprocessor.endpoint;
-
-/**
- */
-public class IIResponseAdapter {
-}


[47/55] [abbrv] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

Posted by sh...@apache.org.
KYLIN-1528 Create a branch for v1.5 with HBase 1.x API


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 1b6ed0899cc2efb575b24e98ddd6f31922979883
Parents: 76d3bd2
Author: shaofengshi <sh...@apache.org>
Authored: Wed Mar 23 17:07:05 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jul 25 20:13:38 2016 +0800

----------------------------------------------------------------------
 examples/test_case_data/sandbox/hbase-site.xml  |  19 +-
 .../test_case_data/sandbox/kylin_job_conf.xml   |  86 +++--
 examples/test_case_data/sandbox/mapred-site.xml |  23 +-
 .../kylin/provision/BuildCubeWithEngine.java    |  50 +--
 pom.xml                                         |  18 +-
 .../kylin/rest/security/AclHBaseStorage.java    |   4 +-
 .../rest/security/MockAclHBaseStorage.java      |   3 +-
 .../apache/kylin/rest/security/MockHTable.java  |  95 +----
 .../rest/security/RealAclHBaseStorage.java      |   9 +-
 .../apache/kylin/rest/service/AclService.java   |  25 +-
 .../apache/kylin/rest/service/CubeService.java  |  35 +-
 .../apache/kylin/rest/service/QueryService.java |  23 +-
 .../apache/kylin/rest/service/UserService.java  |  23 +-
 .../kylin/storage/hbase/HBaseConnection.java    | 169 +++------
 .../kylin/storage/hbase/HBaseResourceStore.java |  37 +-
 .../storage/hbase/cube/SimpleHBaseStore.java    |  20 +-
 .../hbase/cube/v1/CubeSegmentTupleIterator.java |  11 +-
 .../storage/hbase/cube/v1/CubeStorageQuery.java |   4 +-
 .../hbase/cube/v1/RegionScannerAdapter.java     |  11 +-
 .../cube/v1/SerializedHBaseTupleIterator.java   |   4 +-
 .../observer/AggregateRegionObserver.java       |   4 +-
 .../observer/AggregationScanner.java            |  15 +-
 .../observer/ObserverAggregationCache.java      |  15 +-
 .../coprocessor/observer/ObserverEnabler.java   |   4 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     |  10 +-
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |   9 +-
 .../coprocessor/endpoint/CubeVisitService.java  |   4 +-
 .../storage/hbase/steps/CubeHTableUtil.java     |  16 +-
 .../storage/hbase/steps/DeprecatedGCStep.java   |  23 +-
 .../storage/hbase/steps/HBaseCuboidWriter.java  |   9 +-
 .../hbase/steps/HBaseStreamingOutput.java       |   9 +-
 .../kylin/storage/hbase/steps/MergeGCStep.java  |  23 +-
 .../storage/hbase/util/CleanHtableCLI.java      |  12 +-
 .../storage/hbase/util/CubeMigrationCLI.java    | 353 ++++++++++---------
 .../hbase/util/CubeMigrationCheckCLI.java       |  19 +-
 .../hbase/util/DeployCoprocessorCLI.java        |  23 +-
 .../hbase/util/ExtendCubeToHybridCLI.java       |   8 +-
 .../hbase/util/GridTableHBaseBenchmark.java     |  34 +-
 .../kylin/storage/hbase/util/HBaseClean.java    |  18 +-
 .../hbase/util/HBaseRegionSizeCalculator.java   |  42 +--
 .../kylin/storage/hbase/util/HBaseUsage.java    |  10 +-
 .../storage/hbase/util/HbaseStreamingInput.java |  31 +-
 .../hbase/util/HtableAlterMetadataCLI.java      |   9 +-
 .../storage/hbase/util/OrphanHBaseCleanJob.java |  18 +-
 .../kylin/storage/hbase/util/PingHBaseCLI.java  |  15 +-
 .../kylin/storage/hbase/util/RowCounterCLI.java |  11 +-
 .../storage/hbase/util/StorageCleanupJob.java   |  16 +-
 .../storage/hbase/util/UpdateHTableHostCLI.java |  17 +-
 .../observer/AggregateRegionObserverTest.java   |  31 +-
 .../v1/filter/TestFuzzyRowFilterV2EndToEnd.java |   5 +-
 50 files changed, 681 insertions(+), 801 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/examples/test_case_data/sandbox/hbase-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hbase-site.xml b/examples/test_case_data/sandbox/hbase-site.xml
index 46d5345..734908e 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -190,22 +190,5 @@
         <name>zookeeper.znode.parent</name>
         <value>/hbase-unsecure</value>
     </property>
-    <property>
-        <name>hbase.client.pause</name>
-        <value>100</value>
-        <description>General client pause value.  Used mostly as value to wait
-            before running a retry of a failed get, region lookup, etc.
-            See hbase.client.retries.number for description of how we backoff from
-            this initial pause amount and how this pause works w/ retries.</description>
-    </property>
-    <property>
-        <name>hbase.client.retries.number</name>
-        <value>5</value>
-        <description>Maximum retries.  Used as maximum for all retryable
-            operations such as the getting of a cell's value, starting a row update,
-            etc.  Retry interval is a rough function based on hbase.client.pause.  At
-            first we retry at this interval but then with backoff, we pretty quickly reach
-            retrying every ten seconds.  See HConstants#RETRY_BACKOFF for how the backup
-            ramps up.  Change this setting and hbase.client.pause to suit your workload.</description>
-    </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/examples/test_case_data/sandbox/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_job_conf.xml b/examples/test_case_data/sandbox/kylin_job_conf.xml
index bd947af..6082fa9 100644
--- a/examples/test_case_data/sandbox/kylin_job_conf.xml
+++ b/examples/test_case_data/sandbox/kylin_job_conf.xml
@@ -1,20 +1,18 @@
 <?xml version="1.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
+Licensed 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
+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.
+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. See accompanying LICENSE file.
 -->
+
 <configuration>
 
     <property>
@@ -26,44 +24,41 @@
         </description>
     </property>
 
-    <property>
-        <name>mapreduce.map.maxattempts</name>
-        <value>2</value>
-    </property>
+    <!-- uncomment the following 5 properties to enable lzo compressing
 
-    <!--
-    <property>
-        <name>mapred.compress.map.output</name>
-        <value>true</value>
-        <description>Compress map outputs</description>
-    </property>
+	<property>
+		<name>mapred.compress.map.output</name>
+		<value>true</value>
+		<description>Compress map outputs</description>
+	</property>
 
-    <property>
-        <name>mapred.map.output.compression.codec</name>
-        <value>org.apache.hadoop.io.compress.SnappyCodec</value>
-        <description>The compression codec to use for map outputs
-        </description>
-    </property>
+	<property>
+		<name>mapred.map.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for map outputs
+		</description>
+	</property>
 
-    <property>
-        <name>mapred.output.compress</name>
-        <value>true</value>
-        <description>Compress the output of a MapReduce job</description>
-    </property>
+	<property>
+		<name>mapred.output.compress</name>
+		<value>true</value>
+		<description>Compress the output of a MapReduce job</description>
+	</property>
 
-    <property>
-        <name>mapred.output.compression.codec</name>
-        <value>org.apache.hadoop.io.compress.SnappyCodec</value>
-        <description>The compression codec to use for job outputs
-        </description>
-    </property>
+	<property>
+		<name>mapred.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for job outputs
+		</description>
+	</property>
 
-    <property>
-        <name>mapred.output.compression.type</name>
-        <value>BLOCK</value>
-        <description>The compression type to use for job outputs</description>
-    </property>
--->
+	<property>
+		<name>mapred.output.compression.type</name>
+		<value>BLOCK</value>
+		<description>The compression type to use for job outputs</description>
+	</property>
+
+	!-->
 
     <property>
         <name>mapreduce.job.max.split.locations</name>
@@ -76,5 +71,4 @@
         <value>2</value>
         <description>Block replication</description>
     </property>
-
 </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/examples/test_case_data/sandbox/mapred-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/mapred-site.xml b/examples/test_case_data/sandbox/mapred-site.xml
index 18f6feb..ff1c7eb 100644
--- a/examples/test_case_data/sandbox/mapred-site.xml
+++ b/examples/test_case_data/sandbox/mapred-site.xml
@@ -18,7 +18,7 @@
 
     <property>
         <name>io.sort.mb</name>
-        <value>128</value>
+        <value>64</value>
     </property>
 
     <property>
@@ -28,12 +28,12 @@
 
     <property>
         <name>mapred.job.map.memory.mb</name>
-        <value>512</value>
+        <value>250</value>
     </property>
 
     <property>
         <name>mapred.job.reduce.memory.mb</name>
-        <value>512</value>
+        <value>250</value>
     </property>
 
     <property>
@@ -58,7 +58,7 @@
 
     <property>
         <name>mapreduce.application.classpath</name>
-        <value>/tmp/kylin/*,$HADOOP_CONF_DIR,/usr/hdp/${hdp.version}/hbase/lib/hbase-common.jar,/usr/hdp/current/hive-client/conf/,$PWD/mr-framework/hadoop/share/hadoop/mapreduce/*:$PWD/mr-framework/hadoop/share/hadoop/mapreduce/lib/*:$PWD/mr-framework/hadoop/share/hadoop/common/*:$PWD/mr-framework/hadoop/share/hadoop/common/lib/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/lib/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/lib/*:/usr/hdp/${hdp.version}/hadoop/lib/hadoop-lzo-0.6.0.${hdp.version}.jar:/usr/hdp/${hdp.version}/hadoop/lib/snappy-java-1.0.4.1.jar:/etc/hadoop/conf/secure</value>
+        <value>/tmp/kylin/*,$HADOOP_CONF_DIR,/usr/hdp/${hdp.version}/hbase/lib/hbase-common.jar,/usr/hdp/current/hive-client/conf/,$PWD/mr-framework/hadoop/share/hadoop/mapreduce/*:$PWD/mr-framework/hadoop/share/hadoop/mapreduce/lib/*:$PWD/mr-framework/hadoop/share/hadoop/common/*:$PWD/mr-framework/hadoop/share/hadoop/common/lib/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/lib/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/lib/*:/usr/hdp/${hdp.version}/hadoop/lib/hadoop-lzo-0.6.0.${hdp.version}.jar:/etc/hadoop/conf/secure</value>
     </property>
 
     <property>
@@ -81,10 +81,9 @@
         <value>false</value>
     </property>
 
-    <!--the default value on hdp is 0.05, however for test environments we need to be conservative on resource -->
     <property>
         <name>mapreduce.job.reduce.slowstart.completedmaps</name>
-        <value>1</value>
+        <value>0.05</value>
     </property>
 
     <property>
@@ -114,7 +113,7 @@
 
     <property>
         <name>mapreduce.map.java.opts</name>
-        <value>-Xmx512m</value>
+        <value>-Xmx200m</value>
     </property>
 
     <property>
@@ -124,7 +123,7 @@
 
     <property>
         <name>mapreduce.map.memory.mb</name>
-        <value>512</value>
+        <value>250</value>
     </property>
 
     <property>
@@ -169,7 +168,7 @@
 
     <property>
         <name>mapreduce.reduce.memory.mb</name>
-        <value>512</value>
+        <value>250</value>
     </property>
 
     <property>
@@ -219,7 +218,7 @@
 
     <property>
         <name>mapreduce.task.io.sort.mb</name>
-        <value>128</value>
+        <value>64</value>
     </property>
 
     <property>
@@ -234,7 +233,7 @@
 
     <property>
         <name>yarn.app.mapreduce.am.command-opts</name>
-        <value>-Xmx512m</value>
+        <value>-Xmx200m</value>
     </property>
 
     <property>
@@ -244,7 +243,7 @@
 
     <property>
         <name>yarn.app.mapreduce.am.resource.mb</name>
-        <value>512</value>
+        <value>250</value>
     </property>
 
     <property>

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
index a37d7b8..4cee1ed 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
@@ -38,7 +38,7 @@ 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.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
@@ -421,32 +421,32 @@ public class BuildCubeWithEngine {
     private void checkHFilesInHBase(CubeSegment segment) throws IOException {
         Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
         String tableName = segment.getStorageLocationIdentifier();
-        try (HTable table = new HTable(conf, tableName)) {
-            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
-            Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
-            long totalSize = 0;
-            for (Long size : sizeMap.values()) {
-                totalSize += size;
-            }
-            if (totalSize == 0) {
-                return;
-            }
-            Map<byte[], Pair<Integer, Integer>> countMap = cal.getRegionHFileCountMap();
-            // check if there's region contains more than one hfile, which means the hfile config take effects
-            boolean hasMultiHFileRegions = false;
-            for (Pair<Integer, Integer> count : countMap.values()) {
-                // check if hfile count is greater than store count
-                if (count.getSecond() > count.getFirst()) {
-                    hasMultiHFileRegions = true;
-                    break;
-                }
-            }
-            if (KylinConfig.getInstanceFromEnv().getHBaseHFileSizeGB() == 0 && hasMultiHFileRegions) {
-                throw new IOException("hfile size set to 0, but found region contains more than one hfiles");
-            } else if (KylinConfig.getInstanceFromEnv().getHBaseHFileSizeGB() > 0 && !hasMultiHFileRegions) {
-                throw new IOException("hfile size set greater than 0, but all regions still has only one hfile");
+
+        HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
+        Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
+        long totalSize = 0;
+        for (Long size : sizeMap.values()) {
+            totalSize += size;
+        }
+        if (totalSize == 0) {
+            return;
+        }
+        
+        Map<byte[], Pair<Integer, Integer>> countMap = cal.getRegionHFileCountMap();
+        // check if there's region contains more than one hfile, which means the hfile config take effects
+        boolean hasMultiHFileRegions = false;
+        for (Pair<Integer, Integer> count : countMap.values()) {
+            // check if hfile count is greater than store count
+            if (count.getSecond() > count.getFirst()) {
+                hasMultiHFileRegions = true;
+                break;
             }
         }
+        if (KylinConfig.getInstanceFromEnv().getHBaseHFileSizeGB() == 0 && hasMultiHFileRegions) {
+            throw new IOException("hfile size set to 0, but found region contains more than one hfiles");
+        } else if (KylinConfig.getInstanceFromEnv().getHBaseHFileSizeGB() > 0 && !hasMultiHFileRegions) {
+            throw new IOException("hfile size set greater than 0, but all regions still has only one hfile");
+        }
     }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0715e87..025e31a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -47,12 +47,12 @@
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
         <!-- Hadoop versions -->
-        <hadoop2.version>2.6.0</hadoop2.version>
-        <yarn.version>2.6.0</yarn.version>
+        <hadoop2.version>2.7.1</hadoop2.version>
+        <yarn.version>2.7.1</yarn.version>
         <zookeeper.version>3.4.6</zookeeper.version>
-        <hive.version>0.14.0</hive.version>
-        <hive-hcatalog.version>0.14.0</hive-hcatalog.version>
-        <hbase-hadoop2.version>0.98.8-hadoop2</hbase-hadoop2.version>
+        <hive.version>1.2.1</hive.version>
+        <hive-hcatalog.version>1.2.1</hive-hcatalog.version>
+        <hbase-hadoop2.version>1.1.1</hbase-hadoop2.version>
         <kafka.version>0.8.1</kafka.version>
 
         <!-- Dependency versions -->
@@ -64,6 +64,7 @@
 
         <!-- Commons -->
         <commons-cli.version>1.2</commons-cli.version>
+        <commons-codec.version>1.4</commons-codec.version>
         <commons-lang.version>2.6</commons-lang.version>
         <commons-lang3.version>3.4</commons-lang3.version>
         <commons-collections.version>3.2.1</commons-collections.version>
@@ -102,7 +103,7 @@
         <calcite.version>1.6.0</calcite.version>
 
         <!-- Curator.version Version -->
-        <curator.version>2.6.0</curator.version>
+        <curator.version>2.7.1</curator.version>
 
         <!-- Sonar -->
         <sonar.java.coveragePlugin>jacoco</sonar.java.coveragePlugin>
@@ -323,6 +324,11 @@
                 <version>${commons-cli.version}</version>
             </dependency>
             <dependency>
+                <groupId>commons-codec</groupId>
+                <artifactId>commons-codec</artifactId>
+                <version>${commons-codec.version}</version>
+            </dependency>
+            <dependency>
                 <groupId>commons-lang</groupId>
                 <artifactId>commons-lang</artifactId>
                 <version>${commons-lang.version}</version>

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
index 38f299e..bfb5fe4 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
@@ -20,7 +20,7 @@ package org.apache.kylin.rest.security;
 
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 
 /**
  */
@@ -37,6 +37,6 @@ public interface AclHBaseStorage {
 
     String prepareHBaseTable(Class<?> clazz) throws IOException;
 
-    HTableInterface getTable(String tableName) throws IOException;
+    Table getTable(String tableName) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
index 6c8081d..492c176 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.rest.service.AclService;
 import org.apache.kylin.rest.service.UserService;
 
@@ -64,7 +65,7 @@ public class MockAclHBaseStorage implements AclHBaseStorage {
     }
 
     @Override
-    public HTableInterface getTable(String tableName) throws IOException {
+    public Table getTable(String tableName) throws IOException {
         if (realAcl != null) {
             return realAcl.getTable(tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java b/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
index d0aa0ed..972eea9 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
@@ -91,7 +91,7 @@ import com.google.protobuf.ServiceException;
  *     <li>remove some methods for loading data, checking values ...</li>
  * </ul>
  */
-public class MockHTable implements HTableInterface {
+public class MockHTable implements Table {
     private final String tableName;
     private final List<String> columnFamilies = new ArrayList<>();
 
@@ -114,14 +114,6 @@ public class MockHTable implements HTableInterface {
         this.columnFamilies.add(columnFamily);
     }
 
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public byte[] getTableName() {
-        return tableName.getBytes();
-    }
-
     @Override
     public TableName getName() {
         return null;
@@ -200,8 +192,8 @@ public class MockHTable implements HTableInterface {
     }
 
     @Override
-    public Boolean[] exists(List<Get> gets) throws IOException {
-        return new Boolean[0];
+    public boolean[] existsAll(List<Get> list) throws IOException {
+        return new boolean[0];
     }
 
     /**
@@ -306,15 +298,6 @@ public class MockHTable implements HTableInterface {
      * {@inheritDoc}
      */
     @Override
-    public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
-        // FIXME: implement
-        return null;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
     public ResultScanner getScanner(Scan scan) throws IOException {
         final List<Result> ret = new ArrayList<Result>();
         byte[] st = scan.getStartRow();
@@ -446,7 +429,7 @@ public class MockHTable implements HTableInterface {
              */
         }
         if (filter.hasFilterRow() && !filteredOnRowKey) {
-            filter.filterRow(nkvs);
+            filter.filterRow();
         }
         if (filter.filterRow() || filteredOnRowKey) {
             nkvs.clear();
@@ -535,6 +518,11 @@ public class MockHTable implements HTableInterface {
         return false;
     }
 
+    @Override
+    public boolean checkAndPut(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Put put) throws IOException {
+        return false;
+    }
+
     /**
      * {@inheritDoc}
      */
@@ -555,7 +543,7 @@ public class MockHTable implements HTableInterface {
                 continue;
             }
             for (KeyValue kv : delete.getFamilyMap().get(family)) {
-                if (kv.isDeleteFamily()) {
+                if (kv.isDelete()) {
                     data.get(row).get(kv.getFamily()).clear();
                 } else {
                     data.get(row).get(kv.getFamily()).remove(kv.getQualifier());
@@ -592,6 +580,11 @@ public class MockHTable implements HTableInterface {
         return false;
     }
 
+    @Override
+    public boolean checkAndDelete(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Delete delete) throws IOException {
+        return false;
+    }
+
     /**
      * {@inheritDoc}
      */
@@ -605,7 +598,7 @@ public class MockHTable implements HTableInterface {
      */
     @Override
     public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException {
-        return incrementColumnValue(row, family, qualifier, amount, true);
+        return incrementColumnValue(row, family, qualifier, amount, null);
     }
 
     @Override
@@ -617,37 +610,6 @@ public class MockHTable implements HTableInterface {
      * {@inheritDoc}
      */
     @Override
-    public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
-        if (check(row, family, qualifier, null)) {
-            Put put = new Put(row);
-            put.add(family, qualifier, Bytes.toBytes(amount));
-            put(put);
-            return amount;
-        }
-        long newValue = Bytes.toLong(data.get(row).get(family).get(qualifier).lastEntry().getValue()) + amount;
-        data.get(row).get(family).get(qualifier).put(System.currentTimeMillis(), Bytes.toBytes(newValue));
-        return newValue;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public boolean isAutoFlush() {
-        return true;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public void flushCommits() throws IOException {
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
     public void close() throws IOException {
     }
 
@@ -673,29 +635,6 @@ public class MockHTable implements HTableInterface {
      * {@inheritDoc}
      */
     @Override
-    public void setAutoFlush(boolean autoFlush) {
-        throw new NotImplementedException();
-
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
-        throw new NotImplementedException();
-
-    }
-
-    @Override
-    public void setAutoFlushTo(boolean autoFlush) {
-        throw new NotImplementedException();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
     public long getWriteBufferSize() {
         throw new NotImplementedException();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
index 27d9720..222eb70 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
@@ -21,7 +21,8 @@ package org.apache.kylin.rest.security;
 import java.io.IOException;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.rest.service.AclService;
 import org.apache.kylin.rest.service.UserService;
@@ -56,11 +57,11 @@ public class RealAclHBaseStorage implements AclHBaseStorage {
     }
 
     @Override
-    public HTableInterface getTable(String tableName) throws IOException {
+    public Table getTable(String tableName) throws IOException {
         if (StringUtils.equals(tableName, aclTableName)) {
-            return HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
         } else if (StringUtils.equals(tableName, userTableName)) {
-            return HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
         } else {
             throw new IllegalStateException("getTable failed" + tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java b/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
index d693a67..3e3efec 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
@@ -33,7 +33,7 @@ import javax.annotation.PostConstruct;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -124,7 +124,7 @@ public class AclService implements MutableAclService {
     @Override
     public List<ObjectIdentity> findChildren(ObjectIdentity parentIdentity) {
         List<ObjectIdentity> oids = new ArrayList<ObjectIdentity>();
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -173,7 +173,7 @@ public class AclService implements MutableAclService {
     @Override
     public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> oids, List<Sid> sids) throws NotFoundException {
         Map<ObjectIdentity, Acl> aclMaps = new HashMap<ObjectIdentity, Acl>();
-        HTableInterface htable = null;
+        Table htable = null;
         Result result = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
@@ -226,17 +226,16 @@ public class AclService implements MutableAclService {
         Authentication auth = SecurityContextHolder.getContext().getAuthentication();
         PrincipalSid sid = new PrincipalSid(auth);
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
             Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
-            put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
-            put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
-            put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
+            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
+            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
+            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
 
             htable.put(put);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " created successfully.");
         } catch (IOException e) {
@@ -250,7 +249,7 @@ public class AclService implements MutableAclService {
 
     @Override
     public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -266,7 +265,6 @@ public class AclService implements MutableAclService {
             }
 
             htable.delete(delete);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " deleted successfully.");
         } catch (IOException e) {
@@ -284,7 +282,7 @@ public class AclService implements MutableAclService {
             throw e;
         }
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -295,17 +293,16 @@ public class AclService implements MutableAclService {
             Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
 
             if (null != acl.getParentAcl()) {
-                put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
+                put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
             }
 
             for (AccessControlEntry ace : acl.getEntries()) {
                 AceInfo aceInfo = new AceInfo(ace);
-                put.add(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
+                put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
             }
 
             if (!put.isEmpty()) {
                 htable.put(put);
-                htable.flushCommits();
 
                 logger.debug("ACL of " + acl.getObjectIdentity() + " updated successfully.");
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index cfb4cf8..00b07d5 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -30,6 +30,7 @@ import java.util.WeakHashMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.HTable;
+import java.util.*;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
@@ -64,7 +65,6 @@ import org.apache.kylin.rest.security.AclPermission;
 import org.apache.kylin.source.hive.HiveSourceTableLoader;
 import org.apache.kylin.source.hive.cardinality.HiveColumnCardinalityJob;
 import org.apache.kylin.source.hive.cardinality.HiveColumnCardinalityUpdateJob;
-import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -444,35 +444,24 @@ public class CubeService extends BasicService {
         if (htableInfoCache.containsKey(tableName)) {
             return htableInfoCache.get(tableName);
         }
-
-        Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
-        HTable table = null;
+        Connection conn = HBaseConnection.get(this.getConfig().getStorageUrl());
         HBaseResponse hr = null;
         long tableSize = 0;
         int regionCount = 0;
 
-        try {
-            table = new HTable(hconf, tableName);
-
-            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
-            Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
-
-            for (long s : sizeMap.values()) {
-                tableSize += s;
-            }
-
-            regionCount = sizeMap.size();
+        HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
+        Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
 
-            // Set response.
-            hr = new HBaseResponse();
-            hr.setTableSize(tableSize);
-            hr.setRegionCount(regionCount);
-        } finally {
-            if (null != table) {
-                table.close();
-            }
+        for (long s : sizeMap.values()) {
+            tableSize += s;
         }
 
+        regionCount = sizeMap.size();
+
+        // Set response.
+        hr = new HBaseResponse();
+        hr.setTableSize(tableSize);
+        hr.setRegionCount(regionCount);
         htableInfoCache.put(tableName, hr);
 
         return hr;

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 84a5c67..783616d 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -44,9 +44,10 @@ import javax.sql.DataSource;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.kylin.common.KylinConfig;
@@ -131,14 +132,13 @@ public class QueryService extends BasicService {
         Query[] queryArray = new Query[queries.size()];
 
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -164,14 +164,13 @@ public class QueryService extends BasicService {
 
         Query[] queryArray = new Query[queries.size()];
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -183,12 +182,16 @@ public class QueryService extends BasicService {
         }
 
         List<Query> queries = new ArrayList<Query>();
-        HTableInterface htable = null;
+        Table htable = null;
         try {
+<<<<<<< b6296775f7793a63baf7f6a97cf4c0759d654341:server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
             HConnection conn = HBaseConnection.get(hbaseUrl);
             HBaseConnection.createHTableIfNeeded(conn, userTableName, USER_QUERY_FAMILY);
 
             htable = conn.getTable(userTableName);
+=======
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
+>>>>>>> KYLIN-1528 Create a branch for v1.5 with HBase 1.x API:server/src/main/java/org/apache/kylin/rest/service/QueryService.java
             Get get = new Get(Bytes.toBytes(creator));
             get.addFamily(Bytes.toBytes(USER_QUERY_FAMILY));
             Result result = htable.get(get);

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
index 07c7c6f..64c2c7d 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
@@ -30,7 +30,7 @@ import javax.annotation.PostConstruct;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -72,7 +72,7 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public UserDetails loadUserByUsername(String username) throws UsernameNotFoundException {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
@@ -144,16 +144,21 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public void updateUser(UserDetails user) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
+            byte[] userAuthorities = serialize(user.getAuthorities());
             htable = aclHBaseStorage.getTable(userTableName);
 
+<<<<<<< b6296775f7793a63baf7f6a97cf4c0759d654341:server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
             Pair<byte[], byte[]> pair = userToHBaseRow(user);
             Put put = new Put(pair.getKey());
             put.add(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
+=======
+            Put put = new Put(Bytes.toBytes(user.getUsername()));
+            put.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), userAuthorities);
+>>>>>>> KYLIN-1528 Create a branch for v1.5 with HBase 1.x API:server/src/main/java/org/apache/kylin/rest/service/UserService.java
 
             htable.put(put);
-            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -163,14 +168,13 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public void deleteUser(String username) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
             Delete delete = new Delete(Bytes.toBytes(username));
 
             htable.delete(delete);
-            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -185,7 +189,7 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public boolean userExists(String username) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
@@ -215,8 +219,13 @@ public class UserService implements UserDetailsManager {
         Scan s = new Scan();
         s.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN));
 
+<<<<<<< b6296775f7793a63baf7f6a97cf4c0759d654341:server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
         List<UserDetails> all = new ArrayList<UserDetails>();
         HTableInterface htable = null;
+=======
+        List<String> authorities = new ArrayList<String>();
+        Table htable = null;
+>>>>>>> KYLIN-1528 Create a branch for v1.5 with HBase 1.x API:server/src/main/java/org/apache/kylin/rest/service/UserService.java
         ResultScanner scanner = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/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
index d4dd3ae..05170a0 100644
--- 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
@@ -19,16 +19,9 @@
 package org.apache.kylin.storage.hbase;
 
 import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -39,18 +32,15 @@ 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.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.StorageException;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Sets;
-
 /**
  * @author yangli9
  * 
@@ -61,19 +51,14 @@ 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>();
-    private static final ThreadLocal<Configuration> configThreadLocal = new ThreadLocal<>();
-
-    private static ExecutorService coprocessorPool = null;
+    private static final Map<String, Configuration> ConfigCache = new ConcurrentHashMap<String, Configuration>();
+    private static final Map<String, Connection> ConnPool = new ConcurrentHashMap<String, Connection>();
 
     static {
         Runtime.getRuntime().addShutdownHook(new Thread() {
             @Override
             public void run() {
-                closeCoprocessorPool();
-
-                for (HConnection conn : connPool.values()) {
+                for (Connection conn : ConnPool.values()) {
                     try {
                         conn.close();
                     } catch (IOException e) {
@@ -83,64 +68,24 @@ public class HBaseConnection {
             }
         });
     }
-
-    public static ExecutorService getCoprocessorPool() {
-        if (coprocessorPool != null) {
-            return coprocessorPool;
-        }
-
-        synchronized (HBaseConnection.class) {
-            if (coprocessorPool != null) {
-                return coprocessorPool;
-            }
-
-            KylinConfig config = KylinConfig.getInstanceFromEnv();
-
-            // copy from HConnectionImplementation.getBatchPool()
-            int maxThreads = config.getHBaseMaxConnectionThreads();
-            int coreThreads = config.getHBaseCoreConnectionThreads();
-            long keepAliveTime = config.getHBaseConnectionThreadPoolAliveSeconds();
-            LinkedBlockingQueue<Runnable> workQueue = new LinkedBlockingQueue<Runnable>(maxThreads * 100);
-            ThreadPoolExecutor tpe = new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue, //
-                    Threads.newDaemonThreadFactory("kylin-coproc-"));
-            tpe.allowCoreThreadTimeOut(true);
-
-            logger.info("Creating coprocessor thread pool with max of {}, core of {}", maxThreads, coreThreads);
-
-            coprocessorPool = tpe;
-            return coprocessorPool;
-        }
-    }
-
-    private static void closeCoprocessorPool() {
-        if (coprocessorPool == null)
-            return;
-
-        coprocessorPool.shutdown();
-        try {
-            if (!coprocessorPool.awaitTermination(10, TimeUnit.SECONDS)) {
-                coprocessorPool.shutdownNow();
-            }
-        } catch (InterruptedException e) {
-            coprocessorPool.shutdownNow();
-        }
-    }
-
+    
     public static void clearConnCache() {
-        connPool.clear();
+        ConnPool.clear();
     }
 
+    private static final ThreadLocal<Configuration> hbaseConfig = new ThreadLocal<>();
+
     public static Configuration getCurrentHBaseConfiguration() {
-        if (configThreadLocal.get() == null) {
+        if (hbaseConfig.get() == null) {
             String storageUrl = KylinConfig.getInstanceFromEnv().getStorageUrl();
-            configThreadLocal.set(newHBaseConfiguration(storageUrl));
+            hbaseConfig.set(newHBaseConfiguration(storageUrl));
         }
-        return configThreadLocal.get();
+        return hbaseConfig.get();
     }
 
     private static Configuration newHBaseConfiguration(String url) {
         Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
-
+        
         // using a hbase:xxx URL is deprecated, instead hbase config is always loaded from hbase-site.xml in classpath
         if (!(StringUtils.isEmpty(url) || "hbase".equals(url)))
             throw new IllegalArgumentException("to use hbase storage, pls set 'kylin.storage.url=hbase' in kylin.properties");
@@ -150,7 +95,7 @@ public class HBaseConnection {
         if (StringUtils.isNotEmpty(hbaseClusterFs)) {
             conf.set(FileSystem.FS_DEFAULT_NAME_KEY, hbaseClusterFs);
         }
-
+        
         // https://issues.apache.org/jira/browse/KYLIN-953
         if (StringUtils.isBlank(conf.get("hadoop.tmp.dir"))) {
             conf.set("hadoop.tmp.dir", "/tmp");
@@ -167,7 +112,7 @@ public class HBaseConnection {
 
         return conf;
     }
-
+    
     public static String makeQualifiedPathInHBaseCluster(String path) {
         try {
             FileSystem fs = FileSystem.get(getCurrentHBaseConfiguration());
@@ -178,25 +123,25 @@ public class HBaseConnection {
     }
 
     // ============================================================================
-
-    // returned HConnection can be shared by multiple threads and does not require close()
+    
+    // returned Connection can be shared by multiple threads and does not require close()
     @SuppressWarnings("resource")
-    public static HConnection get(String url) {
+    public static Connection get(String url) {
         // find configuration
-        Configuration conf = configCache.get(url);
+        Configuration conf = ConfigCache.get(url);
         if (conf == null) {
             conf = newHBaseConfiguration(url);
-            configCache.put(url, conf);
+            ConfigCache.put(url, conf);
         }
 
-        HConnection connection = connPool.get(url);
+        Connection 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);
+                    connection = ConnectionFactory.createConnection(conf);
+                    ConnPool.put(url, connection);
                 }
 
                 if (connection == null || connection.isClosed()) {
@@ -214,8 +159,8 @@ public class HBaseConnection {
         return connection;
     }
 
-    public static boolean tableExists(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    public static boolean tableExists(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
         try {
             return hbase.tableExists(TableName.valueOf(tableName));
         } finally {
@@ -235,39 +180,23 @@ public class HBaseConnection {
         deleteTable(HBaseConnection.get(hbaseUrl), tableName);
     }
 
-    public static void createHTableIfNeeded(HConnection conn, String table, String... families) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    public static void createHTableIfNeeded(Connection conn, String tableName, String... families) throws IOException {
+        Admin hbase = conn.getAdmin();
 
         try {
-            if (tableExists(conn, table)) {
-                logger.debug("HTable '" + table + "' already exists");
-                Set<String> existingFamilies = getFamilyNames(hbase.getTableDescriptor(TableName.valueOf(table)));
-                boolean wait = false;
-                for (String family : families) {
-                    if (existingFamilies.contains(family) == false) {
-                        logger.debug("Adding family '" + family + "' to HTable '" + table + "'");
-                        hbase.addColumn(table, newFamilyDescriptor(family));
-                        // addColumn() is async, is there a way to wait it finish?
-                        wait = true;
-                    }
-                }
-                if (wait) {
-                    try {
-                        Thread.sleep(10000);
-                    } catch (InterruptedException e) {
-                        logger.warn("", e);
-                    }
-                }
+            if (tableExists(conn, tableName)) {
+                logger.debug("HTable '" + tableName + "' already exists");
                 return;
             }
 
-            logger.debug("Creating HTable '" + table + "'");
+            logger.debug("Creating HTable '" + tableName + "'");
 
-            HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+            HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
 
             if (null != families && families.length > 0) {
                 for (String family : families) {
-                    HColumnDescriptor fd = newFamilyDescriptor(family);
+                    HColumnDescriptor fd = new HColumnDescriptor(family);
+                    fd.setInMemory(true); // metadata tables are best in memory
                     desc.addFamily(fd);
                 }
             }
@@ -275,32 +204,14 @@ public class HBaseConnection {
             desc.setValue(HTABLE_UUID_TAG, UUID.randomUUID().toString());
             hbase.createTable(desc);
 
-            logger.debug("HTable '" + table + "' created");
+            logger.debug("HTable '" + tableName + "' created");
         } finally {
             hbase.close();
         }
     }
 
-    private static Set<String> getFamilyNames(HTableDescriptor desc) {
-        HashSet<String> result = Sets.newHashSet();
-        for (byte[] bytes : desc.getFamiliesKeys()) {
-            try {
-                result.add(new String(bytes, "UTF-8"));
-            } catch (UnsupportedEncodingException e) {
-                logger.error(e.toString());
-            }
-        }
-        return result;
-    }
-
-    private static HColumnDescriptor newFamilyDescriptor(String family) {
-        HColumnDescriptor fd = new HColumnDescriptor(family);
-        fd.setInMemory(true); // metadata tables are best in memory
-        return fd;
-    }
-
-    public static void deleteTable(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    public static void deleteTable(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
 
         try {
             if (!tableExists(conn, tableName)) {
@@ -310,10 +221,10 @@ public class HBaseConnection {
 
             logger.debug("delete HTable '" + tableName + "'");
 
-            if (hbase.isTableEnabled(tableName)) {
-                hbase.disableTable(tableName);
+            if (hbase.isTableEnabled(TableName.valueOf(tableName))) {
+                hbase.disableTable(TableName.valueOf(tableName));
             }
-            hbase.deleteTable(tableName);
+            hbase.deleteTable(TableName.valueOf(tableName));
 
             logger.debug("HTable '" + tableName + "' deleted");
         } finally {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/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
index e2f3661..f988dea 100644
--- 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
@@ -31,10 +31,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -69,7 +70,7 @@ public class HBaseResourceStore extends ResourceStore {
     final String tableNameBase;
     final String hbaseUrl;
 
-    private HConnection getConnection() throws IOException {
+    private Connection getConnection() throws IOException {
         return HBaseConnection.get(hbaseUrl);
     }
 
@@ -120,7 +121,7 @@ public class HBaseResourceStore extends ResourceStore {
         byte[] endRow = Bytes.toBytes(lookForPrefix);
         endRow[endRow.length - 1]++;
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         Scan scan = new Scan(startRow, endRow);
         if ((filter != null && filter instanceof KeyOnlyFilter) == false) {
             scan.addColumn(B_FAMILY, B_COLUMN_TS);
@@ -238,13 +239,12 @@ public class HBaseResourceStore extends ResourceStore {
         IOUtils.copy(content, bout);
         bout.close();
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             byte[] row = Bytes.toBytes(resPath);
             Put put = buildPut(resPath, ts, row, bout.toByteArray(), table);
 
             table.put(put);
-            table.flushCommits();
         } finally {
             IOUtils.closeQuietly(table);
         }
@@ -252,7 +252,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             byte[] row = Bytes.toBytes(resPath);
             byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
@@ -265,8 +265,6 @@ public class HBaseResourceStore extends ResourceStore {
                 throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but it is " + real);
             }
 
-            table.flushCommits();
-
             return newTS;
         } finally {
             IOUtils.closeQuietly(table);
@@ -275,7 +273,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected void deleteResourceImpl(String resPath) throws IOException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             boolean hdfsResourceExist = false;
             Result result = internalGetFromHTable(table, resPath, true, false);
@@ -332,12 +330,17 @@ public class HBaseResourceStore extends ResourceStore {
                 get.addColumn(B_FAMILY, B_COLUMN_TS);
         }
 
-        Result result = table.get(get);
-        boolean exists = result != null && (!result.isEmpty() || (result.getExists() != null && result.getExists()));
-        return exists ? result : null;
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
+        try {
+            Result result = table.get(get);
+            boolean exists = result != null && (!result.isEmpty() || (result.getExists() != null && result.getExists()));
+            return exists ? result : null;
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
     }
 
-    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
+    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, Table table) throws IOException {
         Path redirectPath = bigCellHDFSPath(resPath);
         Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
@@ -363,7 +366,7 @@ public class HBaseResourceStore extends ResourceStore {
         return redirectPath;
     }
 
-    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
+    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, Table table) throws IOException {
         int kvSizeLimit = this.kylinConfig.getHBaseKeyValueSize();
         if (content.length > kvSizeLimit) {
             writeLargeCellToHdfs(resPath, content, table);
@@ -371,8 +374,8 @@ public class HBaseResourceStore extends ResourceStore {
         }
 
         Put put = new Put(row);
-        put.add(B_FAMILY, B_COLUMN, content);
-        put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
+        put.addColumn(B_FAMILY, B_COLUMN, content);
+        put.addColumn(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
 
         return put;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
index 2462873..7272d8d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
@@ -26,12 +26,13 @@ import java.util.NoSuchElementException;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.kv.RowConstants;
@@ -86,14 +87,13 @@ public class SimpleHBaseStore implements IGTStore {
     }
 
     private class Writer implements IGTWriter {
-        final HTableInterface table;
+        final BufferedMutator table;
         final ByteBuffer rowkey = ByteBuffer.allocate(50);
         final ByteBuffer value = ByteBuffer.allocate(50);
 
         Writer() throws IOException {
-            HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-            table = conn.getTable(htableName);
-            table.setAutoFlush(false, true);
+            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            table = conn.getBufferedMutator(htableName);
         }
 
         @Override
@@ -113,24 +113,24 @@ public class SimpleHBaseStore implements IGTStore {
 
             Put put = new Put(rowkey);
             put.addImmutable(CF_B, ByteBuffer.wrap(COL_B), HConstants.LATEST_TIMESTAMP, value);
-            table.put(put);
+            table.mutate(put);
         }
 
         @Override
         public void close() throws IOException {
-            table.flushCommits();
+            table.flush();
             table.close();
         }
     }
 
     class Reader implements IGTScanner {
-        final HTableInterface table;
+        final Table table;
         final ResultScanner scanner;
 
         int count = 0;
 
         Reader() throws IOException {
-            HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
             table = conn.getTable(htableName);
 
             Scan scan = new Scan();

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
index 8ac3832..21166bc 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
@@ -25,8 +25,9 @@ import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -70,7 +71,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     protected final List<RowValueDecoder> rowValueDecoders;
     private final StorageContext context;
     private final String tableName;
-    private final HTableInterface table;
+    private final Table table;
 
     protected CubeTupleConverter tupleConverter;
     protected final Iterator<HBaseKeyRange> rangeIterator;
@@ -88,7 +89,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     private int advMeasureRowsRemaining;
     private int advMeasureRowIndex;
 
-    public CubeSegmentTupleIterator(CubeSegment cubeSeg, List<HBaseKeyRange> keyRanges, HConnection conn, //
+    public CubeSegmentTupleIterator(CubeSegment cubeSeg, List<HBaseKeyRange> keyRanges, Connection conn, //
             Set<TblColRef> dimensions, TupleFilter filter, Set<TblColRef> groupBy, //
             List<RowValueDecoder> rowValueDecoders, StorageContext context, TupleInfo returnTupleInfo) {
         this.cubeSeg = cubeSeg;
@@ -108,7 +109,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
         this.rangeIterator = keyRanges.iterator();
 
         try {
-            this.table = conn.getTable(tableName);
+            this.table = conn.getTable(TableName.valueOf(tableName));
         } catch (Throwable t) {
             throw new StorageException("Error when open connection to table " + tableName, t);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
index 672bcbe..f864bed 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
@@ -33,7 +33,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.Dictionary;
@@ -148,7 +148,7 @@ public class CubeStorageQuery implements IStorageQuery {
         setCoprocessor(groupsCopD, valueDecoders, context); // enable coprocessor if beneficial
         setLimit(filter, context);
 
-        HConnection conn = HBaseConnection.get(context.getConnUrl());
+        Connection conn = HBaseConnection.get(context.getConnUrl());
 
         // notice we're passing filterD down to storage instead of flatFilter
         return new SerializedHBaseTupleIterator(conn, scans, cubeInstance, dimensionsD, filterD, groupsCopD, valueDecoders, context, returnTupleInfo);

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
index 6342c5c..ae442fe 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
@@ -23,9 +23,11 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 
 /**
  * @author yangli9
@@ -50,7 +52,7 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
@@ -60,7 +62,7 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
@@ -94,4 +96,9 @@ public class RegionScannerAdapter implements RegionScanner {
         return Long.MAX_VALUE;
     }
 
+    @Override
+    public int getBatch() {
+        return -1;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
index e8dd5b9..d033c77 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.metadata.filter.TupleFilter;
@@ -57,7 +57,7 @@ public class SerializedHBaseTupleIterator implements ITupleIterator {
     private int scanCount;
     private ITuple next;
 
-    public SerializedHBaseTupleIterator(HConnection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, //
+    public SerializedHBaseTupleIterator(Connection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, //
             Set<TblColRef> dimensions, TupleFilter filter, Set<TblColRef> groupBy, List<RowValueDecoder> rowValueDecoders, //
             StorageContext context, TupleInfo returnTupleInfo) {
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
index c7b650a..8dba1b1 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorBehavior;
@@ -99,7 +99,7 @@ public class AggregateRegionObserver extends BaseRegionObserver {
         // start/end region operation & sync on scanner is suggested by the
         // javadoc of RegionScanner.nextRaw()
         // FIXME: will the lock still work when a iterator is returned? is it safe? Is readonly attribute helping here? by mhb
-        HRegion region = ctxt.getEnvironment().getRegion();
+        Region region = ctxt.getEnvironment().getRegion();
         region.startRegionOperation();
         try {
             synchronized (innerScanner) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
index be26142..7d73a73 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
@@ -24,7 +24,9 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorBehavior;
@@ -116,8 +118,8 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
-        return outerScanner.next(result, limit);
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.next(result, scannerContext);
     }
 
     @Override
@@ -126,8 +128,8 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-        return outerScanner.nextRaw(result, limit);
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.nextRaw(result, scannerContext);
     }
 
     @Override
@@ -160,6 +162,11 @@ public class AggregationScanner implements RegionScanner {
         return outerScanner.getMvccReadPoint();
     }
 
+    @Override
+    public int getBatch() {
+        return outerScanner.getBatch();
+    }
+
     private static class Stats {
         long inputRows = 0;
         long inputBytes = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
index 8404262..1809a44 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
@@ -24,12 +24,10 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggregationCache;
@@ -112,7 +110,7 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -122,7 +120,7 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+        public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -161,6 +159,11 @@ public class ObserverAggregationCache extends AggregationCache {
             // AggregateRegionObserver.LOG.info("Kylin Scanner getMvccReadPoint()");
             return Long.MAX_VALUE;
         }
+
+        @Override
+        public int getBatch() {
+            return innerScanner.getBatch();
+        }
     }
 
 }

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

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6ed089/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
index 78ad18d..830aca7 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
@@ -23,6 +23,7 @@ import java.nio.BufferOverflowException;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ExecutorService;
@@ -35,8 +36,10 @@ import javax.annotation.Nullable;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
@@ -279,7 +282,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         final ImmutableBitSet selectedColBlocks = scanRequest.getSelectedColBlocks().set(0);
 
         // globally shared connection, does not require close
-        final HConnection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+        final Connection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
 
         final List<IntList> hbaseColumnsToGTIntList = Lists.newArrayList();
         List<List<Integer>> hbaseColumnsToGT = getHBaseColumnsGTMapping(selectedColBlocks);
@@ -357,7 +360,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                     final boolean[] abnormalFinish = new boolean[1];
 
                     try {
-                        HTableInterface table = conn.getTable(cubeSeg.getStorageLocationIdentifier(), HBaseConnection.getCoprocessorPool());
+                        HTableInterface table = conn.get(cubeSeg.getStorageLocationIdentifier(), HBaseConnection.getCoprocessorPool());
 
                         final CubeVisitRequest request = builder.build();
                         final byte[] startKey = epRange.getFirst();
@@ -432,5 +435,4 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         return sb.toString();
 
     }
-
 }


[32/55] [abbrv] kylin git commit: KYLIN-1905 datetimepicker init date update

Posted by sh...@apache.org.
KYLIN-1905 datetimepicker init date update


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 4bd685a6a1577841c04c46ca07d907e158bb974a
Parents: 53face5
Author: Jason <ji...@163.com>
Authored: Tue Jul 19 18:47:22 2016 +0800
Committer: Jason <ji...@163.com>
Committed: Tue Jul 19 18:47:22 2016 +0800

----------------------------------------------------------------------
 webapp/app/js/directives/datetimepicker.js | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4bd685a6/webapp/app/js/directives/datetimepicker.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/directives/datetimepicker.js b/webapp/app/js/directives/datetimepicker.js
index 4541836..b21822f 100644
--- a/webapp/app/js/directives/datetimepicker.js
+++ b/webapp/app/js/directives/datetimepicker.js
@@ -370,7 +370,7 @@
             var date = new Date();
             if(!modelValue){
               var tmp = new Date();
-              date = new Date(tmp.getFullYear(),tmp.getMonth()+1,tmp.getDate());
+              date = new Date(tmp.getFullYear(),tmp.getMonth(),tmp.getDate());
             }else{
               date = new Date(modelValue);
             }


[39/55] [abbrv] kylin git commit: KYLIN-1909 wrong ACL when getting cubes

Posted by sh...@apache.org.
KYLIN-1909 wrong ACL when getting cubes


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: ae9ccf39aa21afe7296db39505243a5843180645
Parents: 658839d
Author: shaofengshi <sh...@apache.org>
Authored: Fri Jul 22 15:47:55 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Jul 22 15:47:55 2016 +0800

----------------------------------------------------------------------
 .../metadata/project/learn_kylin.json            |  2 +-
 .../kylin/rest/controller/CubeController.java    | 12 ++++++++++--
 .../kylin/rest/controller/ProjectController.java |  2 +-
 .../apache/kylin/rest/service/CubeService.java   | 19 +++++++++----------
 .../kylin/rest/service/ProjectService.java       | 10 ++++++++++
 5 files changed, 31 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/ae9ccf39/examples/sample_cube/metadata/project/learn_kylin.json
----------------------------------------------------------------------
diff --git a/examples/sample_cube/metadata/project/learn_kylin.json b/examples/sample_cube/metadata/project/learn_kylin.json
index fcfd505..e468214 100644
--- a/examples/sample_cube/metadata/project/learn_kylin.json
+++ b/examples/sample_cube/metadata/project/learn_kylin.json
@@ -1,5 +1,5 @@
 {
-  "uuid": "1eaca32a-a33e-4b69-83dd-0bb8b1f8c91b",
+  "uuid": "2fbca32a-a33e-4b69-83dd-0bb8b1f8c91b",
   "name": "learn_kylin",
   "realizations": [
     {

http://git-wip-us.apache.org/repos/asf/kylin/blob/ae9ccf39/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index c049a15..7932211 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -30,6 +30,7 @@ import java.util.UUID;
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.model.CubeBuildTypeEnum;
 import org.apache.kylin.cube.model.CubeDesc;
@@ -172,8 +173,13 @@ public class CubeController extends BasicController {
     @RequestMapping(value = "/{cubeName}/cost", method = { RequestMethod.PUT })
     @ResponseBody
     public CubeInstance updateCubeCost(@PathVariable String cubeName, @RequestParam(value = "cost") int cost) {
+        CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+        if (cube == null) {
+            throw new InternalErrorException("Cannot find cube " + cubeName);
+        }
+
         try {
-            return cubeService.updateCubeCost(cubeName, cost);
+            return cubeService.updateCubeCost(cube, cost);
         } catch (Exception e) {
             String message = "Failed to update cube cost: " + cubeName + " : " + cost;
             logger.error(message, e);
@@ -203,7 +209,9 @@ public class CubeController extends BasicController {
     @ResponseBody
     public CubeInstance rebuildLookupSnapshot(@PathVariable String cubeName, @PathVariable String segmentName, @RequestParam(value = "lookupTable") String lookupTable) {
         try {
-            return cubeService.rebuildLookupSnapshot(cubeName, segmentName, lookupTable);
+            final CubeManager cubeMgr = cubeService.getCubeManager();
+            final CubeInstance cube = cubeMgr.getCube(cubeName);
+            return cubeService.rebuildLookupSnapshot(cube, segmentName, lookupTable);
         } catch (IOException e) {
             logger.error(e.getLocalizedMessage(), e);
             throw new InternalErrorException(e.getLocalizedMessage());

http://git-wip-us.apache.org/repos/asf/kylin/blob/ae9ccf39/server-base/src/main/java/org/apache/kylin/rest/controller/ProjectController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/ProjectController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/ProjectController.java
index f829fff..496e44a 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/ProjectController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/ProjectController.java
@@ -76,7 +76,7 @@ public class ProjectController extends BasicController {
     @RequestMapping(value = "", method = { RequestMethod.GET })
     @ResponseBody
     public List<ProjectInstance> getProjects(@RequestParam(value = "limit", required = false) Integer limit, @RequestParam(value = "offset", required = false) Integer offset) {
-        return projectService.listAllProjects(limit, offset);
+        return projectService.listProjects(limit, offset);
     }
 
     @RequestMapping(value = "/readable", method = { RequestMethod.GET })

http://git-wip-us.apache.org/repos/asf/kylin/blob/ae9ccf39/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index 72942e8..cfb4cf8 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -128,6 +128,7 @@ public class CubeService extends BasicService {
         return filterCubes;
     }
 
+    @PostFilter(Constant.ACCESS_POST_FILTER_READ)
     public List<CubeInstance> getCubes(final String cubeName, final String projectName, final String modelName, final Integer limit, final Integer offset) {
 
         List<CubeInstance> cubes;
@@ -147,12 +148,9 @@ public class CubeService extends BasicService {
         return cubes.subList(coffset, coffset + climit);
     }
 
-    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
-    public CubeInstance updateCubeCost(String cubeName, int cost) throws IOException {
-        CubeInstance cube = getCubeManager().getCube(cubeName);
-        if (cube == null) {
-            throw new IOException("Cannot find cube " + cubeName);
-        }
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public CubeInstance updateCubeCost(CubeInstance cube, int cost) throws IOException {
+
         if (cube.getCost() == cost) {
             // Do nothing
             return cube;
@@ -167,6 +165,7 @@ public class CubeService extends BasicService {
         return getCubeManager().updateCube(cubeBuilder);
     }
 
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or " + Constant.ACCESS_HAS_ROLE_MODELER)
     public CubeInstance createCubeAndDesc(String cubeName, String projectName, CubeDesc desc) throws IOException {
         if (getCubeManager().getCube(cubeName) != null) {
             throw new InternalErrorException("The cube named " + cubeName + " already exists");
@@ -485,6 +484,7 @@ public class CubeService extends BasicService {
      *
      * @param tableName
      */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_MODELER + " or " + Constant.ACCESS_HAS_ROLE_ADMIN)
     public void calculateCardinality(String tableName, String submitter) {
         String[] dbTableName = HadoopUtil.parseHiveTableName(tableName);
         tableName = dbTableName[0] + "." + dbTableName[1];
@@ -526,11 +526,10 @@ public class CubeService extends BasicService {
         getCubeDescManager().updateCubeDesc(desc);
     }
 
-    public CubeInstance rebuildLookupSnapshot(String cubeName, String segmentName, String lookupTable) throws IOException {
-        CubeManager cubeMgr = getCubeManager();
-        CubeInstance cube = cubeMgr.getCube(cubeName);
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION')  or hasPermission(#cube, 'MANAGEMENT')")
+    public CubeInstance rebuildLookupSnapshot(CubeInstance cube, String segmentName, String lookupTable) throws IOException {
         CubeSegment seg = cube.getSegment(segmentName, SegmentStatusEnum.READY);
-        cubeMgr.buildSnapshotTable(seg, lookupTable);
+        getCubeManager().buildSnapshotTable(seg, lookupTable);
 
         return cube;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/ae9ccf39/server-base/src/main/java/org/apache/kylin/rest/service/ProjectService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ProjectService.java b/server-base/src/main/java/org/apache/kylin/rest/service/ProjectService.java
index c0610a2..b4cceb2 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/ProjectService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ProjectService.java
@@ -32,6 +32,7 @@ import org.apache.kylin.rest.security.AclPermission;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.prepost.PostFilter;
 import org.springframework.security.access.prepost.PreAuthorize;
 import org.springframework.security.core.context.SecurityContextHolder;
 import org.springframework.stereotype.Component;
@@ -48,6 +49,7 @@ public class ProjectService extends BasicService {
     @Autowired
     private AccessService accessService;
 
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
     public ProjectInstance createProject(CreateProjectRequest projectRequest) throws IOException {
         String projectName = projectRequest.getName();
         String description = projectRequest.getDescription();
@@ -81,6 +83,14 @@ public class ProjectService extends BasicService {
         return updatedProject;
     }
 
+
+    @PostFilter(Constant.ACCESS_POST_FILTER_READ)
+    public List<ProjectInstance> listProjects(final Integer limit, final Integer offset) {
+        List<ProjectInstance> projects = listAllProjects(limit, offset);
+        return projects;
+    }
+
+    @Deprecated
     public List<ProjectInstance> listAllProjects(final Integer limit, final Integer offset) {
         List<ProjectInstance> projects = getProjectManager().listAllProjects();
 


[15/55] [abbrv] kylin git commit: KYLIN-1766 date/time encoding UI update & add tip for int encoding

Posted by sh...@apache.org.
KYLIN-1766 date/time encoding UI update & add tip for int encoding

Signed-off-by: Jason <ji...@163.com>


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

Branch: refs/heads/1.5.x-CDH5.7
Commit: 2224eec343790c0676b4d92e2de6ce58bc2abf8b
Parents: 8735c28
Author: Jason <ji...@163.com>
Authored: Sun Jul 10 20:31:11 2016 +0800
Committer: Jason <ji...@163.com>
Committed: Sun Jul 10 20:36:01 2016 +0800

----------------------------------------------------------------------
 webapp/app/js/controllers/cubeAdvanceSetting.js         | 10 +++++++---
 webapp/app/js/controllers/cubeSchema.js                 |  3 +++
 webapp/app/partials/cubeDesigner/advanced_settings.html |  2 +-
 3 files changed, 11 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/2224eec3/webapp/app/js/controllers/cubeAdvanceSetting.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeAdvanceSetting.js b/webapp/app/js/controllers/cubeAdvanceSetting.js
index c3110b5..09fb3d4 100644
--- a/webapp/app/js/controllers/cubeAdvanceSetting.js
+++ b/webapp/app/js/controllers/cubeAdvanceSetting.js
@@ -28,7 +28,7 @@ KylinApp.controller('CubeAdvanceSettingCtrl', function ($scope, $modal,cubeConfi
     //var _isDictionary = item.encoding === "dict"?"true":"false";
     var _isFixedLength = item.encoding.substring(0,12) === "fixed_length"?"true":"false";//fixed_length:12
     var _isIntLength = item.encoding.substring(0,3) === "int"?"true":"false";//fixed_length:12
-    var _encoding = "dict";
+    var _encoding = item.encoding;
     var _valueLength ;
     if(_isFixedLength !=="false"){
       _valueLength = item.encoding.substring(13,item.encoding.length);
@@ -38,6 +38,7 @@ KylinApp.controller('CubeAdvanceSettingCtrl', function ($scope, $modal,cubeConfi
       _valueLength = item.encoding.substring(4,item.encoding.length);
       _encoding = "int";
     }
+
     var rowkeyObj = {
       column:item.column,
       encoding:_encoding,
@@ -56,17 +57,20 @@ KylinApp.controller('CubeAdvanceSettingCtrl', function ($scope, $modal,cubeConfi
   }
 
   $scope.refreshRowKey = function(list,index,item,checkShard){
-    var encoding = "dict";
+    var encoding;
     var column = item.column;
     var isShardBy = item.isShardBy;
-    if(item.encoding!=="dict"){
+    if(item.encoding!=="dict" && item.encoding!=="date"&& item.encoding!=="time"){
       if(item.encoding=="fixed_length" && item.valueLength){
         encoding = "fixed_length:"+item.valueLength;
       }
       else if(item.encoding=="int" && item.valueLength){
         encoding = "int:"+item.valueLength;
+      }else{
+        encoding = item.encoding;
       }
     }else{
+      encoding = item.encoding;
       item.valueLength=0;
     }
     $scope.cubeMetaFrame.rowkey.rowkey_columns[index].column = column;

http://git-wip-us.apache.org/repos/asf/kylin/blob/2224eec3/webapp/app/js/controllers/cubeSchema.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeSchema.js b/webapp/app/js/controllers/cubeSchema.js
index 2aafd33..d7dfc09 100755
--- a/webapp/app/js/controllers/cubeSchema.js
+++ b/webapp/app/js/controllers/cubeSchema.js
@@ -267,6 +267,9 @@ KylinApp.controller('CubeSchemaCtrl', function ($scope, QueryService, UserServic
           if(rowkey.isShardBy == true){
             shardRowkeyList.push(rowkey.column);
           }
+          if(rowkey.encoding.substr(0,3)=='int' && (rowkey.encoding.substr(4)<1 || rowkey.encoding.substr(4)>8)){
+            errors.push("int encoding column length should between 1 and 8.");
+          }
         })
         if(shardRowkeyList.length >1){
           errors.push("At most one 'shard by' column is allowed.");

http://git-wip-us.apache.org/repos/asf/kylin/blob/2224eec3/webapp/app/partials/cubeDesigner/advanced_settings.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/advanced_settings.html b/webapp/app/partials/cubeDesigner/advanced_settings.html
index 5f18b51..62e1dd5 100755
--- a/webapp/app/partials/cubeDesigner/advanced_settings.html
+++ b/webapp/app/partials/cubeDesigner/advanced_settings.html
@@ -261,7 +261,7 @@
                       <!--Column Length -->
                       <input type="text" class="form-control" placeholder="Column Length.." ng-if="state.mode=='edit'"
                              tooltip="rowkey column length.." tooltip-trigger="focus"
-                             ng-disabled="rowkey_column.encoding=='dict'"
+                             ng-disabled="rowkey_column.encoding=='dict'||rowkey_column.encoding=='date'||rowkey_column.encoding=='time'"
                              ng-change="refreshRowKey(convertedRowkeys,$index,rowkey_column);"
                              ng-model="rowkey_column.valueLength" class="form-control">