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

[1/8] kylin git commit: KYLIN-1858 remove all ii related code

Repository: kylin
Updated Branches:
  refs/heads/master cd5e603dd -> 2cc0b9c43


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);
-
-    }
-}


[6/8] kylin git commit: KYLIN-1858 remove all ii related code

Posted by ma...@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();
-    }
-
-}


[7/8] kylin git commit: KYLIN-1858 remove all ii related code

Posted by ma...@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;
-        }
-    }
-
-}


[5/8] kylin git commit: KYLIN-1858 remove all ii related code

Posted by ma...@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;
-        }
-    }
-
-}


[3/8] kylin git commit: KYLIN-1858 remove all ii related code

Posted by ma...@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();
-    }
-}


[2/8] kylin git commit: KYLIN-1858 remove all ii related code

Posted by ma...@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>

[4/8] kylin git commit: KYLIN-1858 remove all ii related code

Posted by ma...@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 {
-}


[8/8] kylin git commit: KYLIN-1858 remove all ii related code

Posted by ma...@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/master
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