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

[01/19] incubator-kylin git commit: KYLIN-1112 on the way: Reorganize InvertedIndex source codes into plug-in architecture

Repository: incubator-kylin
Updated Branches:
  refs/heads/KYLIN-1112-2 [created] 262b38a93


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/tools/IICLI.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/tools/IICLI.java b/invertedindex/src/main/java/org/apache/kylin/job/tools/IICLI.java
deleted file mode 100644
index 8c39aa1..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/tools/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.tools;
-
-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();
-                    }
-                };
-            }
-        };
-    }
-}


[04/19] incubator-kylin git commit: support global dict in HBaseClientKVIterator

Posted by sh...@apache.org.
support global dict in HBaseClientKVIterator


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

Branch: refs/heads/KYLIN-1112-2
Commit: ca6f578d289d4c358f6af71c7dc3380c4e4422ff
Parents: 37d0602
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 16:19:08 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/invertedindex/model/IIKeyValueCodec.java  | 2 +-
 .../kylin/storage/hbase/cube/v1/HBaseClientKVIterator.java     | 6 +++++-
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ca6f578d/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
index eebf756..7e54a98 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
@@ -199,7 +199,7 @@ public class IIKeyValueCodec implements KeyValueCodec {
                             valueContainers[curCol] = c;
                         } else {
                             final ImmutableBytesWritable dictBytes = row.getDictionary();
-                            if (dictBytes.get() != null && dictBytes.getLength() != 0) {
+                            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());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ca6f578d/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
index b18d258..7920a8a 100644
--- 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
@@ -69,6 +69,7 @@ public class HBaseClientKVIterator implements Iterable<IIRow>, Closeable {
         ImmutableBytesWritable value = new ImmutableBytesWritable();
         ImmutableBytesWritable dict = new ImmutableBytesWritable();
         IIRow pair = new IIRow(key, value, dict);
+        static final byte[] EMPTY_BYTES = new byte[0];
 
         @Override
         public boolean hasNext() {
@@ -82,8 +83,11 @@ public class HBaseClientKVIterator implements Iterable<IIRow>, Closeable {
             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);
-            if (c != null)
+            if (c != null) {
                 dict.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
+            } else {
+                dict.set(EMPTY_BYTES);
+            }
             return pair;
         }
 


[19/19] incubator-kylin git commit: some revert

Posted by sh...@apache.org.
some revert


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

Branch: refs/heads/KYLIN-1112-2
Commit: 262b38a939805f11a468c0af8d088ebcf98ebd6f
Parents: 0e5d07c
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 17:19:59 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:18 2015 +0800

----------------------------------------------------------------------
 .../kylin/engine/mr/invertedindex/InvertedIndexMapper.java    | 2 +-
 .../kylin/storage/hbase/cube/v1/HBaseClientKVIterator.java    | 7 +------
 2 files changed, 2 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/262b38a9/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
index 670972a..88249ed 100644
--- 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
@@ -76,7 +76,7 @@ public class InvertedIndexMapper<KEYIN> extends KylinMapper<KEYIN, Object, LongW
         for (int i = 0; i < row.length; i++) {
             Object fieldValue = row[i];
             if (fieldValue != null)
-                rec.setValueString(i, fieldValue == null ? null : fieldValue.toString());
+                rec.setValueString(i, fieldValue.toString());
         }
 
         outputKey.set(rec.getTimestamp());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/262b38a9/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
index 24defef..8aace22 100644
--- 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
@@ -69,7 +69,6 @@ public class HBaseClientKVIterator implements Iterable<IIRow>, Closeable {
         ImmutableBytesWritable value = new ImmutableBytesWritable();
         ImmutableBytesWritable dict = new ImmutableBytesWritable();
         IIRow pair = new IIRow(key, value, dict);
-        final byte[] EMPTY_BYTES = new byte[0];
 
         @Override
         public boolean hasNext() {
@@ -83,11 +82,7 @@ public class HBaseClientKVIterator implements Iterable<IIRow>, Closeable {
             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);
-            if (c != null) {
-                dict.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-            } else {
-                dict.set(EMPTY_BYTES);
-            }
+            dict.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
             return pair;
         }
 


[05/19] incubator-kylin git commit: check whether path exists before set permission

Posted by sh...@apache.org.
check whether path exists before set permission


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

Branch: refs/heads/KYLIN-1112-2
Commit: ed2784aeadcd35d1d78e499dc28e146ab1aae179
Parents: d8c689a
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 14:33:44 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/storage/hbase/ii/IIBulkLoadJob.java | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ed2784ae/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
index 149791b..4098797 100644
--- 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
@@ -46,12 +46,17 @@ public class IIBulkLoadJob extends AbstractHadoopJob {
             options.addOption(OPTION_II_NAME);
             parseOptions(options, args);
 
-            String tableName = getOptionValue(OPTION_HTABLE_NAME);
+            String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
             String input = getOptionValue(OPTION_INPUT_PATH);
 
             FileSystem fs = FileSystem.get(getConf());
-            FsPermission permission = new FsPermission((short) 0777);
-            fs.setPermission(new Path(input, IIDesc.HBASE_FAMILY), permission);
+            Path columnFamilyPath = new Path(input, IIDesc.HBASE_FAMILY);
+
+            // File may have already been auto-loaded (in the case of MapR DB)
+            if (fs.exists(columnFamilyPath)) {
+                FsPermission permission = new FsPermission((short) 0777);
+                fs.setPermission(columnFamilyPath, permission);
+            }
 
             return ToolRunner.run(new LoadIncrementalHFiles(getConf()), new String[] { input, tableName });
 


[10/19] incubator-kylin git commit: add attachKylinPropsAndMetadata in IIDistinctColumnsJob

Posted by sh...@apache.org.
add attachKylinPropsAndMetadata in IIDistinctColumnsJob


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

Branch: refs/heads/KYLIN-1112-2
Commit: 3d652ee96628ceae0b8089152152b15403017b06
Parents: c6b3862
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 09:57:21 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../engine/mr/common/AbstractHadoopJob.java     | 22 ++++++++++++++++++++
 .../mr/invertedindex/IIDistinctColumnsJob.java  |  3 +++
 .../mr/invertedindex/InvertedIndexJob.java      | 21 +------------------
 3 files changed, 26 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3d652ee9/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 ffa7c13..8782bbe 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
@@ -58,6 +58,8 @@ import org.apache.kylin.common.util.StringSplitter;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.job.JobInstance;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.job.exception.JobException;
@@ -322,6 +324,26 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
         attachKylinPropsAndMetadata(dumpList, conf);
     }
 
+    protected void attachKylinPropsAndMetadata(IIInstance ii, Configuration conf) throws IOException {
+        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+        // 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());
+        }
+        for (IISegment segment : ii.getSegments()) {
+            dumpList.addAll(segment.getDictionaryPaths());
+        }
+
+        attachKylinPropsAndMetadata(dumpList, conf);
+    }
+
     protected void attachKylinPropsAndMetadata(ArrayList<String> dumpList, 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/incubator-kylin/blob/3d652ee9/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
index b691dc1..e9534f2 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
@@ -21,6 +21,7 @@ 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.io.NullWritable;
 import org.apache.hadoop.io.ShortWritable;
@@ -81,6 +82,8 @@ public class IIDistinctColumnsJob extends AbstractHadoopJob {
             setupMapper(ii.getFirstSegment());
             setupReducer(output);
 
+            Configuration conf = job.getConfiguration();
+            attachKylinPropsAndMetadata(ii, conf);
             return waitForCompletion(job);
 
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3d652ee9/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
index cff7219..bcae524 100644
--- 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
@@ -108,26 +108,7 @@ public class InvertedIndexJob extends AbstractHadoopJob {
         conf.set(BatchConstants.CFG_II_SEGMENT_NAME, seg.getName());
     }
 
-    protected void attachKylinPropsAndMetadata(IIInstance ii, Configuration conf) throws IOException {
-        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
-
-        // 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());
-        }
-        for (IISegment segment : ii.getSegments()) {
-            dumpList.addAll(segment.getDictionaryPaths());
-        }
-
-        attachKylinPropsAndMetadata(dumpList, conf);
-    }
-
+   
     private void setupMapper(IISegment segment) throws IOException {
 
 //        String[] dbTableNames = HadoopUtil.parseHiveTableName(intermediateTable);


[18/19] incubator-kylin git commit: enable ITIIQueryTest

Posted by sh...@apache.org.
enable ITIIQueryTest


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

Branch: refs/heads/KYLIN-1112-2
Commit: 805a4a62fb8f2166c9529cdb7fa63b3b0b7bc868
Parents: 2189c2f
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 15:56:22 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/storage/hbase/cube/v1/HBaseClientKVIterator.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/805a4a62/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
index 8aace22..b18d258 100644
--- 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
@@ -82,7 +82,8 @@ public class HBaseClientKVIterator implements Iterable<IIRow>, Closeable {
             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());
+            if (c != null)
+                dict.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
             return pair;
         }
 


[09/19] incubator-kylin git commit: bulk load with Hbase configurations

Posted by sh...@apache.org.
bulk load with Hbase configurations


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

Branch: refs/heads/KYLIN-1112-2
Commit: 2189c2f124ce029b96721fffb531b4a8ce51a2dd
Parents: ed2784a
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 15:28:38 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/storage/hbase/ii/IIBulkLoadJob.java   | 13 ++++++-------
 1 file changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2189c2f1/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
index 4098797..8cf921a 100644
--- 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
@@ -19,18 +19,15 @@
 package org.apache.kylin.storage.hbase.ii;
 
 import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.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.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 
 /**
  */
@@ -49,7 +46,9 @@ public class IIBulkLoadJob extends AbstractHadoopJob {
             String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
             String input = getOptionValue(OPTION_INPUT_PATH);
 
-            FileSystem fs = FileSystem.get(getConf());
+            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+            FileSystem fs = FileSystem.get(conf);
+
             Path columnFamilyPath = new Path(input, IIDesc.HBASE_FAMILY);
 
             // File may have already been auto-loaded (in the case of MapR DB)
@@ -58,7 +57,7 @@ public class IIBulkLoadJob extends AbstractHadoopJob {
                 fs.setPermission(columnFamilyPath, permission);
             }
 
-            return ToolRunner.run(new LoadIncrementalHFiles(getConf()), new String[] { input, tableName });
+            return ToolRunner.run(new LoadIncrementalHFiles(conf), new String[] { input, tableName });
 
         } catch (Exception e) {
             printUsage(options);


[06/19] incubator-kylin git commit: KYLIN-1112 make code compile

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJobBuilder.java
index 2605e65..18d3001 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJobBuilder.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJobBuilder.java
@@ -1,224 +1,219 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine.mr.invertedindex;
-
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.TimeZone;
-
-import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
-import org.apache.kylin.engine.mr.common.MapReduceExecutable;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
-import org.apache.kylin.source.hive.HiveMRInput.BatchCubingInputSide;
-
-import com.google.common.base.Preconditions;
-
-/**
- */
-public final class IIJobBuilder {
-
-    final JobEngineConfig engineConfig;
-
-    public IIJobBuilder(JobEngineConfig engineConfig) {
-        this.engineConfig = engineConfig;
-    }
-
-    public IIJob buildJob(IISegment seg, String submitter) {
-        checkPreconditions(seg);
-
-        IIJob result = initialJob(seg, "BUILD", submitter);
-        final String jobId = result.getId();
-        final IIJoinedFlatTableDesc intermediateTableDesc = new IIJoinedFlatTableDesc(seg.getIIDesc());
-        final String intermediateTableIdentity = getIntermediateTableIdentity(intermediateTableDesc);
-        final String factDistinctColumnsPath = getIIDistinctColumnsPath(seg, jobId);
-        final String iiRootPath = getJobWorkingDir(jobId) + "/" + seg.getIIInstance().getName() + "/";
-        final String iiPath = iiRootPath + "*";
-
-        final AbstractExecutable intermediateHiveTableStep = createFlatHiveTableStep(intermediateTableDesc, jobId);
-        result.addTask(intermediateHiveTableStep);
-
-        result.addTask(createFactDistinctColumnsStep(seg, intermediateTableIdentity, jobId, factDistinctColumnsPath));
-
-        result.addTask(createBuildDictionaryStep(seg, factDistinctColumnsPath));
-
-        result.addTask(createInvertedIndexStep(seg, intermediateTableIdentity, iiRootPath));
-
-        // create htable step
-        result.addTask(createCreateHTableStep(seg));
-
-        // generate hfiles step
-        result.addTask(createConvertToHfileStep(seg, iiPath, jobId));
-
-        // bulk load step
-        result.addTask(createBulkLoadStep(seg, jobId));
-
-        return result;
-    }
-
-    private AbstractExecutable createFlatHiveTableStep(IIJoinedFlatTableDesc intermediateTableDesc, String jobId) {
-        return BatchCubingInputSide.createFlatHiveTableStep(engineConfig, intermediateTableDesc, jobId);
-    }
-
-    private IIJob initialJob(IISegment seg, String type, String submitter) {
-        IIJob result = new IIJob();
-        SimpleDateFormat format = new SimpleDateFormat("z yyyy-MM-dd HH:mm:ss");
-        format.setTimeZone(TimeZone.getTimeZone(engineConfig.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;
-    }
-
-    private void checkPreconditions(IISegment seg) {
-        Preconditions.checkNotNull(seg, "segment cannot be null");
-        Preconditions.checkNotNull(engineConfig, "jobEngineConfig cannot be null");
-    }
-
-    private void appendMapReduceParameters(StringBuilder builder, JobEngineConfig engineConfig) {
-        try {
-            String jobConf = engineConfig.getHadoopJobConfFilePath(RealizationCapacity.MEDIUM);
-            if (jobConf != null && jobConf.length() > 0) {
-                builder.append(" -conf ").append(jobConf);
-            }
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private String getIIDistinctColumnsPath(IISegment seg, String jobUuid) {
-        return getJobWorkingDir(jobUuid) + "/" + seg.getIIInstance().getName() + "/ii_distinct_columns";
-    }
-
-    private String getHFilePath(IISegment seg, String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getIIInstance().getName() + "/hfile/";
-    }
-
-    private MapReduceExecutable createFactDistinctColumnsStep(IISegment seg, String factTableName, String jobId, String output) {
-        MapReduceExecutable result = new MapReduceExecutable();
-        result.setName(ExecutableConstants.STEP_NAME_FACT_DISTINCT_COLUMNS);
-        result.setMapReduceJobClass(IIDistinctColumnsJob.class);
-        StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd, engineConfig);
-        appendExecCmdParameters(cmd, "tablename", factTableName);
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "output", output);
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Fact_Distinct_Columns_" + seg.getIIInstance().getName() + "_Step");
-
-        result.setMapReduceParams(cmd.toString());
-        return result;
-    }
-
-    private HadoopShellExecutable createBuildDictionaryStep(IISegment seg, String factDistinctColumnsPath) {
-        // base cuboid job
-        HadoopShellExecutable buildDictionaryStep = new HadoopShellExecutable();
-        buildDictionaryStep.setName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY);
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "input", factDistinctColumnsPath);
-
-        buildDictionaryStep.setJobParams(cmd.toString());
-        buildDictionaryStep.setJobClass(CreateInvertedIndexDictionaryJob.class);
-        return buildDictionaryStep;
-    }
-
-    private MapReduceExecutable createInvertedIndexStep(IISegment seg, String intermediateHiveTable, String iiOutputTempPath) {
-        // base cuboid job
-        MapReduceExecutable buildIIStep = new MapReduceExecutable();
-
-        StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd, engineConfig);
-
-        buildIIStep.setName(ExecutableConstants.STEP_NAME_BUILD_II);
-
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "tablename", intermediateHiveTable);
-        appendExecCmdParameters(cmd, "output", iiOutputTempPath);
-        appendExecCmdParameters(cmd, "jobname", ExecutableConstants.STEP_NAME_BUILD_II);
-
-        buildIIStep.setMapReduceParams(cmd.toString());
-        buildIIStep.setMapReduceJobClass(InvertedIndexJob.class);
-        return buildIIStep;
-    }
-
-    private HadoopShellExecutable createCreateHTableStep(IISegment seg) {
-        HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
-        createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
-
-        createHtableStep.setJobParams(cmd.toString());
-        createHtableStep.setJobClass(IICreateHTableJob.class);
-
-        return createHtableStep;
-    }
-
-    private MapReduceExecutable createConvertToHfileStep(IISegment seg, String inputPath, String jobId) {
-        MapReduceExecutable createHFilesStep = new MapReduceExecutable();
-        createHFilesStep.setName(ExecutableConstants.STEP_NAME_CONVERT_II_TO_HFILE);
-        StringBuilder cmd = new StringBuilder();
-
-        appendMapReduceParameters(cmd, engineConfig);
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "input", inputPath);
-        appendExecCmdParameters(cmd, "output", getHFilePath(seg, jobId));
-        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
-        appendExecCmdParameters(cmd, "jobname", "Kylin_HFile_Generator_" + seg.getIIInstance().getName() + "_Step");
-
-        createHFilesStep.setMapReduceParams(cmd.toString());
-        createHFilesStep.setMapReduceJobClass(IICreateHFileJob.class);
-
-        return createHFilesStep;
-    }
-
-    private HadoopShellExecutable createBulkLoadStep(IISegment seg, String jobId) {
-        HadoopShellExecutable bulkLoadStep = new HadoopShellExecutable();
-        bulkLoadStep.setName(ExecutableConstants.STEP_NAME_BULK_LOAD_HFILE);
-
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "input", getHFilePath(seg, jobId));
-        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-
-        bulkLoadStep.setJobParams(cmd.toString());
-        bulkLoadStep.setJobClass(IIBulkLoadJob.class);
-
-        return bulkLoadStep;
-
-    }
-
-    private StringBuilder appendExecCmdParameters(StringBuilder buf, String paraName, String paraValue) {
-        return buf.append(" -").append(paraName).append(" ").append(paraValue);
-    }
-
-    private String getJobWorkingDir(String uuid) {
-        return engineConfig.getHdfsWorkingDirectory() + "kylin-" + uuid;
-    }
-
-    private String getIntermediateTableIdentity(IIJoinedFlatTableDesc intermediateTableDesc) {
-        return engineConfig.getConfig().getHiveDatabaseForIntermediateTable() + "." + intermediateTableDesc.getTableName();
-    }
-}
+///*
+// * 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.text.SimpleDateFormat;
+//import java.util.Date;
+//import java.util.TimeZone;
+//
+//import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
+//import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+//import org.apache.kylin.invertedindex.IISegment;
+//import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
+//import org.apache.kylin.job.constant.ExecutableConstants;
+//import org.apache.kylin.job.engine.JobEngineConfig;
+//import org.apache.kylin.job.execution.AbstractExecutable;
+//import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
+//
+//import com.google.common.base.Preconditions;
+//
+///**
+// */
+//public final class IIJobBuilder {
+//
+//    final JobEngineConfig engineConfig;
+//
+//    public IIJobBuilder(JobEngineConfig engineConfig) {
+//        this.engineConfig = engineConfig;
+//    }
+//
+//    public IIJob buildJob(IISegment seg, String submitter) {
+//        checkPreconditions(seg);
+//
+//        IIJob result = initialJob(seg, "BUILD", submitter);
+//        final String jobId = result.getId();
+//        final IIJoinedFlatTableDesc intermediateTableDesc = new IIJoinedFlatTableDesc(seg.getIIDesc());
+//        final String intermediateTableIdentity = getIntermediateTableIdentity(intermediateTableDesc);
+//        final String factDistinctColumnsPath = getIIDistinctColumnsPath(seg, jobId);
+//        final String iiRootPath = getJobWorkingDir(jobId) + "/" + seg.getIIInstance().getName() + "/";
+//        final String iiPath = iiRootPath + "*";
+//
+//        final AbstractExecutable intermediateHiveTableStep = createFlatHiveTableStep(intermediateTableDesc, jobId);
+//        result.addTask(intermediateHiveTableStep);
+//
+//        result.addTask(createFactDistinctColumnsStep(seg, intermediateTableIdentity, jobId, factDistinctColumnsPath));
+//
+//        result.addTask(createBuildDictionaryStep(seg, factDistinctColumnsPath));
+//
+//        result.addTask(createInvertedIndexStep(seg, intermediateTableIdentity, iiRootPath));
+//
+//        // create htable step
+//        result.addTask(createCreateHTableStep(seg));
+//
+//        // generate hfiles step
+//        result.addTask(createConvertToHfileStep(seg, iiPath, jobId));
+//
+//        // bulk load step
+//        result.addTask(createBulkLoadStep(seg, jobId));
+//
+//        return result;
+//    }
+//
+//    private IIJob initialJob(IISegment seg, String type, String submitter) {
+//        IIJob result = new IIJob();
+//        SimpleDateFormat format = new SimpleDateFormat("z yyyy-MM-dd HH:mm:ss");
+//        format.setTimeZone(TimeZone.getTimeZone(engineConfig.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;
+//    }
+//
+//    private void checkPreconditions(IISegment seg) {
+//        Preconditions.checkNotNull(seg, "segment cannot be null");
+//        Preconditions.checkNotNull(engineConfig, "jobEngineConfig cannot be null");
+//    }
+//
+//    private void appendMapReduceParameters(StringBuilder builder, JobEngineConfig engineConfig) {
+//        try {
+//            String jobConf = engineConfig.getHadoopJobConfFilePath(RealizationCapacity.MEDIUM);
+//            if (jobConf != null && jobConf.length() > 0) {
+//                builder.append(" -conf ").append(jobConf);
+//            }
+//        } catch (IOException e) {
+//            throw new RuntimeException(e);
+//        }
+//    }
+//
+//    private String getIIDistinctColumnsPath(IISegment seg, String jobUuid) {
+//        return getJobWorkingDir(jobUuid) + "/" + seg.getIIInstance().getName() + "/ii_distinct_columns";
+//    }
+//
+//    private String getHFilePath(IISegment seg, String jobId) {
+//        return getJobWorkingDir(jobId) + "/" + seg.getIIInstance().getName() + "/hfile/";
+//    }
+//
+//    private MapReduceExecutable createFactDistinctColumnsStep(IISegment seg, String factTableName, String jobId, String output) {
+//        MapReduceExecutable result = new MapReduceExecutable();
+//        result.setName(ExecutableConstants.STEP_NAME_FACT_DISTINCT_COLUMNS);
+//        result.setMapReduceJobClass(IIDistinctColumnsJob.class);
+//        StringBuilder cmd = new StringBuilder();
+//        appendMapReduceParameters(cmd, engineConfig);
+//        appendExecCmdParameters(cmd, "tablename", factTableName);
+//        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+//        appendExecCmdParameters(cmd, "output", output);
+//        appendExecCmdParameters(cmd, "jobname", "Kylin_Fact_Distinct_Columns_" + seg.getIIInstance().getName() + "_Step");
+//
+//        result.setMapReduceParams(cmd.toString());
+//        return result;
+//    }
+//
+//    private HadoopShellExecutable createBuildDictionaryStep(IISegment seg, String factDistinctColumnsPath) {
+//        // base cuboid job
+//        HadoopShellExecutable buildDictionaryStep = new HadoopShellExecutable();
+//        buildDictionaryStep.setName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY);
+//        StringBuilder cmd = new StringBuilder();
+//        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+//        appendExecCmdParameters(cmd, "input", factDistinctColumnsPath);
+//
+//        buildDictionaryStep.setJobParams(cmd.toString());
+//        buildDictionaryStep.setJobClass(CreateInvertedIndexDictionaryJob.class);
+//        return buildDictionaryStep;
+//    }
+//
+//    private MapReduceExecutable createInvertedIndexStep(IISegment seg, String intermediateHiveTable, String iiOutputTempPath) {
+//        // base cuboid job
+//        MapReduceExecutable buildIIStep = new MapReduceExecutable();
+//
+//        StringBuilder cmd = new StringBuilder();
+//        appendMapReduceParameters(cmd, engineConfig);
+//
+//        buildIIStep.setName(ExecutableConstants.STEP_NAME_BUILD_II);
+//
+//        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+//        appendExecCmdParameters(cmd, "tablename", intermediateHiveTable);
+//        appendExecCmdParameters(cmd, "output", iiOutputTempPath);
+//        appendExecCmdParameters(cmd, "jobname", ExecutableConstants.STEP_NAME_BUILD_II);
+//
+//        buildIIStep.setMapReduceParams(cmd.toString());
+//        buildIIStep.setMapReduceJobClass(InvertedIndexJob.class);
+//        return buildIIStep;
+//    }
+//
+//    private HadoopShellExecutable createCreateHTableStep(IISegment seg) {
+//        HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
+//        createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
+//        StringBuilder cmd = new StringBuilder();
+//        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+//        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
+//
+//        createHtableStep.setJobParams(cmd.toString());
+//        createHtableStep.setJobClass(IICreateHTableJob.class);
+//
+//        return createHtableStep;
+//    }
+//
+//    private MapReduceExecutable createConvertToHfileStep(IISegment seg, String inputPath, String jobId) {
+//        MapReduceExecutable createHFilesStep = new MapReduceExecutable();
+//        createHFilesStep.setName(ExecutableConstants.STEP_NAME_CONVERT_II_TO_HFILE);
+//        StringBuilder cmd = new StringBuilder();
+//
+//        appendMapReduceParameters(cmd, engineConfig);
+//        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+//        appendExecCmdParameters(cmd, "input", inputPath);
+//        appendExecCmdParameters(cmd, "output", getHFilePath(seg, jobId));
+//        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
+//        appendExecCmdParameters(cmd, "jobname", "Kylin_HFile_Generator_" + seg.getIIInstance().getName() + "_Step");
+//
+//        createHFilesStep.setMapReduceParams(cmd.toString());
+//        createHFilesStep.setMapReduceJobClass(IICreateHFileJob.class);
+//
+//        return createHFilesStep;
+//    }
+//
+//    private HadoopShellExecutable createBulkLoadStep(IISegment seg, String jobId) {
+//        HadoopShellExecutable bulkLoadStep = new HadoopShellExecutable();
+//        bulkLoadStep.setName(ExecutableConstants.STEP_NAME_BULK_LOAD_HFILE);
+//
+//        StringBuilder cmd = new StringBuilder();
+//        appendExecCmdParameters(cmd, "input", getHFilePath(seg, jobId));
+//        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
+//        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+//
+//        bulkLoadStep.setJobParams(cmd.toString());
+//        bulkLoadStep.setJobClass(IIBulkLoadJob.class);
+//
+//        return bulkLoadStep;
+//
+//    }
+//
+//    private StringBuilder appendExecCmdParameters(StringBuilder buf, String paraName, String paraValue) {
+//        return buf.append(" -").append(paraName).append(" ").append(paraValue);
+//    }
+//
+//    private String getJobWorkingDir(String uuid) {
+//        return engineConfig.getHdfsWorkingDirectory() + "kylin-" + uuid;
+//    }
+//
+//    private String getIntermediateTableIdentity(IIJoinedFlatTableDesc intermediateTableDesc) {
+//        return engineConfig.getConfig().getHiveDatabaseForIntermediateTable() + "." + intermediateTableDesc.getTableName();
+//    }
+//}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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
index fc5f939..cff7219 100644
--- 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
@@ -30,9 +30,9 @@ 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.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.engine.mr.HadoopUtil;
+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;
@@ -74,7 +74,7 @@ public class InvertedIndexJob extends AbstractHadoopJob {
 
             setJobClasspath(job);
 
-            setupMapper(intermediateTable);
+            setupMapper(ii.getFirstSegment());
             setupReducer(output, sharding);
             attachMetadata(ii);
 
@@ -128,12 +128,15 @@ public class InvertedIndexJob extends AbstractHadoopJob {
         attachKylinPropsAndMetadata(dumpList, conf);
     }
 
-    private void setupMapper(String intermediateTable) throws IOException {
+    private void setupMapper(IISegment segment) throws IOException {
 
-        String[] dbTableNames = HadoopUtil.parseHiveTableName(intermediateTable);
-        HCatInputFormat.setInput(job, dbTableNames[0], dbTableNames[1]);
+//        String[] dbTableNames = HadoopUtil.parseHiveTableName(intermediateTable);
+//        HCatInputFormat.setInput(job, dbTableNames[0], dbTableNames[1]);
+//
+//        job.setInputFormatClass(HCatInputFormat.class);
+        IMRInput.IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(segment).getFlatTableInputFormat();
+        flatTableInputFormat.configureJob(job);
 
-        job.setInputFormatClass(HCatInputFormat.class);
 
         job.setMapperClass(InvertedIndexMapper.class);
         job.setMapOutputKeyClass(LongWritable.class);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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
index 81921f1..8247295 100644
--- 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
@@ -24,12 +24,10 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hive.hcatalog.data.HCatRecord;
-import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
-import org.apache.hive.hcatalog.data.schema.HCatSchema;
-import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.kylin.common.KylinConfig;
+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;
@@ -42,15 +40,14 @@ import org.apache.kylin.metadata.model.SegmentStatusEnum;
 /**
  * @author yangli9
  */
-public class InvertedIndexMapper<KEYIN> extends KylinMapper<KEYIN, HCatRecord, LongWritable, ImmutableBytesWritable> {
+public class InvertedIndexMapper<KEYIN> extends KylinMapper<KEYIN, Object, LongWritable, ImmutableBytesWritable> {
 
     private TableRecordInfo info;
     private TableRecord rec;
 
     private LongWritable outputKey;
     private ImmutableBytesWritable outputValue;
-    private HCatSchema schema = null;
-    private List<HCatFieldSchema> fields;
+    private IMRInput.IMRTableInputFormat flatTableInputFormat;
 
     @Override
     protected void setup(Context context) throws IOException {
@@ -68,17 +65,16 @@ public class InvertedIndexMapper<KEYIN> extends KylinMapper<KEYIN, HCatRecord, L
         outputKey = new LongWritable();
         outputValue = new ImmutableBytesWritable(rec.getBytes());
 
-        schema = HCatInputFormat.getTableSchema(context.getConfiguration());
-
-        fields = schema.getFields();
+        flatTableInputFormat = MRUtil.getBatchCubingInputSide(ii.getFirstSegment()).getFlatTableInputFormat();
     }
 
     @Override
-    public void map(KEYIN key, HCatRecord record, Context context) throws IOException, InterruptedException {
+    public void map(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
 
+        String[] row = flatTableInputFormat.parseMapperInput(record);
         rec.reset();
-        for (int i = 0; i < fields.size(); i++) {
-            Object fieldValue = record.get(i);
+        for (int i = 0; i < row.length; i++) {
+            Object fieldValue = row[i];
             rec.setValueString(i, fieldValue == null ? null : fieldValue.toString());
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/UpdateInvertedIndexInfoAfterBuildStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/UpdateInvertedIndexInfoAfterBuildStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/UpdateInvertedIndexInfoAfterBuildStep.java
new file mode 100644
index 0000000..277dea5
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/UpdateInvertedIndexInfoAfterBuildStep.java
@@ -0,0 +1,93 @@
+/*
+ * 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.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.CubingJob;
+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 java.io.IOException;
+
+/**
+ */
+public class UpdateInvertedIndexInfoAfterBuildStep extends AbstractExecutable {
+
+    private static final String SEGMENT_ID = "segmentId";
+    private static final String II_NAME = "iiName";
+    private static final String JOB_ID = "jobId";
+
+    public UpdateInvertedIndexInfoAfterBuildStep() {
+        super();
+    }
+
+    public void setInvertedIndexName(String cubeName) {
+        this.setParam(II_NAME, cubeName);
+    }
+
+    private String getInvertedIndexName() {
+        return getParam(II_NAME);
+    }
+
+    public void setSegmentId(String segmentId) {
+        this.setParam(SEGMENT_ID, segmentId);
+    }
+
+    private String getSegmentId() {
+        return getParam(SEGMENT_ID);
+    }
+    
+    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/incubator-kylin/blob/aa4944d4/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java
index 2e65ae6..9f17d60 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java
@@ -25,6 +25,7 @@ import org.apache.kylin.engine.mr.IMROutput2;
 import org.apache.kylin.engine.mr.JobBuilderSupport;
 import org.apache.kylin.engine.mr.MRUtil;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,18 +48,18 @@ public class SparkCubingJobBuilder extends JobBuilderSupport {
     }
 
     public DefaultChainedExecutable build() {
-        final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
+        final CubingJob result = CubingJob.createBuildJob((CubeSegment)seg, submitter, config);
         final String jobId = result.getId();
 
         inputSide.addStepPhase1_CreateFlatTable(result);
-        final CubeJoinedFlatTableDesc joinedFlatTableDesc = new CubeJoinedFlatTableDesc(seg.getCubeDesc(), seg);
+        final IJoinedFlatTableDesc joinedFlatTableDesc = seg.getJoinedFlatTableDesc();
         final String tableName = joinedFlatTableDesc.getTableName();
         logger.info("intermediate table:" + tableName);
 
         final SparkExecutable sparkExecutable = new SparkExecutable();
         sparkExecutable.setClassName(SparkCubing.class.getName());
         sparkExecutable.setParam("hiveTable", tableName);
-        sparkExecutable.setParam("cubeName", seg.getCubeInstance().getName());
+        sparkExecutable.setParam("cubeName", seg.getRealization().getName());
         sparkExecutable.setParam("segmentId", seg.getUuid());
         sparkExecutable.setParam("confPath", confPath);
         sparkExecutable.setParam("coprocessor", coprocessor);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/invertedindex/pom.xml
----------------------------------------------------------------------
diff --git a/invertedindex/pom.xml b/invertedindex/pom.xml
index cd22162..06e8059 100644
--- a/invertedindex/pom.xml
+++ b/invertedindex/pom.xml
@@ -51,6 +51,10 @@
             <groupId>com.n3twork.druid</groupId>
             <artifactId>extendedset</artifactId>
         </dependency>
+        <dependency>
+            <groupId>com.ning</groupId>
+            <artifactId>compress-lzf</artifactId>
+        </dependency>
 
         <!-- Env & Test -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
index 5ce11f8..20f289c 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
@@ -26,12 +26,7 @@ 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.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.model.*;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.metadata.realization.RealizationType;
@@ -48,7 +43,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
  */
 @SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class IIInstance extends RootPersistentEntity implements IRealization {
+public class IIInstance extends RootPersistentEntity implements IRealization, IBuildable {
 
     public static IIInstance create(String iiName, String projectName, IIDesc iiDesc) {
         IIInstance iii = new IIInstance();
@@ -373,4 +368,13 @@ public class IIInstance extends RootPersistentEntity implements IRealization {
         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/incubator-kylin/blob/aa4944d4/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
index af848de..adcca8b 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java
@@ -29,6 +29,9 @@ import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.IDictionaryAware;
 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.IBuildable;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -37,6 +40,8 @@ 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;
+import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.metadata.realization.IRealizationSegment;
 
 /**
  * @author honma
@@ -44,7 +49,7 @@ import com.google.common.base.Objects;
 
 // 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>, IDictionaryAware {
+public class IISegment implements Comparable<IISegment>, IDictionaryAware, IRealizationSegment {
 
     @JsonBackReference
     private IIInstance iiInstance;
@@ -108,6 +113,7 @@ public class IISegment implements Comparable<IISegment>, IDictionaryAware {
 
     // ============================================================================
 
+    @Override
     public String getUuid() {
         return uuid;
     }
@@ -116,6 +122,7 @@ public class IISegment implements Comparable<IISegment>, IDictionaryAware {
         this.uuid = id;
     }
 
+    @Override
     public String getName() {
         return name;
     }
@@ -204,6 +211,7 @@ public class IISegment implements Comparable<IISegment>, IDictionaryAware {
         this.iiInstance = iiInstance;
     }
 
+    @Override
     public String getStorageLocationIdentifier() {
         return storageLocationIdentifier;
     }
@@ -289,4 +297,28 @@ public class IISegment implements Comparable<IISegment>, IDictionaryAware {
         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/incubator-kylin/blob/aa4944d4/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
index fee16ba..bfa4eaa 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java
@@ -35,14 +35,7 @@ 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.DimensionDesc;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.model.*;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
@@ -87,6 +80,13 @@ public class IIDesc extends RootPersistentEntity {
     private int sliceSize = 50000; // no. rows
     @JsonProperty("useLocalDictionary")
     private boolean useLocalDictionary = true;
+
+    @JsonProperty("engine_type")
+    private int engineType = IEngineAware.ID_MR_II;
+
+    @JsonProperty("storage_type")
+    private int storageType = IStorageAware.ID_HBASE;
+    
     @JsonProperty("signature")
     private String signature;
 
@@ -399,4 +399,20 @@ public class IIDesc extends RootPersistentEntity {
 
     }
 
+    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/incubator-kylin/blob/aa4944d4/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index d51638f..f2f9e32 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -58,6 +58,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-source-hive</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-source-kafka</artifactId>
             <version>${project.parent.version}</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index 4571852..1eb2683 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -32,6 +32,8 @@ import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.engine.mr.IMRInput;
 import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
 import org.apache.kylin.job.JoinedFlatTable;
 import org.apache.kylin.job.common.ShellExecutable;
 import org.apache.kylin.job.constant.ExecutableConstants;
@@ -43,11 +45,12 @@ import org.apache.kylin.job.execution.ExecutableContext;
 import org.apache.kylin.job.execution.ExecuteResult;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.realization.IRealizationSegment;
 
 public class HiveMRInput implements IMRInput {
 
     @Override
-    public IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg) {
+    public IMRBatchCubingInputSide getBatchCubingInputSide(IRealizationSegment seg) {
         return new BatchCubingInputSide(seg);
     }
 
@@ -90,13 +93,13 @@ public class HiveMRInput implements IMRInput {
     public static class BatchCubingInputSide implements IMRBatchCubingInputSide {
 
         final JobEngineConfig conf;
-        final CubeSegment seg;
-        final CubeJoinedFlatTableDesc flatHiveTableDesc;
+        final IRealizationSegment seg;
+        final IJoinedFlatTableDesc flatHiveTableDesc;
 
-        public BatchCubingInputSide(CubeSegment seg) {
+        public BatchCubingInputSide(IRealizationSegment seg) {
             this.conf = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
             this.seg = seg;
-            this.flatHiveTableDesc = new CubeJoinedFlatTableDesc(seg.getCubeDesc(), seg);
+            this.flatHiveTableDesc = seg.getJoinedFlatTableDesc();
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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
new file mode 100644
index 0000000..149791b
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IIBulkLoadJob.java
@@ -0,0 +1,64 @@
+/*
+ * 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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.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.invertedindex.model.IIDesc;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+
+/**
+ */
+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);
+            String input = getOptionValue(OPTION_INPUT_PATH);
+
+            FileSystem fs = FileSystem.get(getConf());
+            FsPermission permission = new FsPermission((short) 0777);
+            fs.setPermission(new Path(input, IIDesc.HBASE_FAMILY), permission);
+
+            return ToolRunner.run(new LoadIncrementalHFiles(getConf()), new String[] { input, tableName });
+
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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
new file mode 100644
index 0000000..4781f2b
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileJob.java
@@ -0,0 +1,81 @@
+/*
+ * 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.engine.mr.common.AbstractHadoopJob;
+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));
+
+            setJobClasspath(job);
+
+            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;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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
new file mode 100644
index 0000000..e4b688f
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHFileMapper.java
@@ -0,0 +1,55 @@
+/*
+ * 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.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 {
+
+        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, //
+                value.get(), value.getOffset(), value.getLength());
+
+        context.write(key, kv);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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
new file mode 100644
index 0000000..0a72a91
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/IICreateHTableJob.java
@@ -0,0 +1,149 @@
+/*
+ * 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.storage.hbase.util.IIDeployCoprocessorCLI;
+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;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class IICreateHTableJob extends AbstractHadoopJob {
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        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();
+
+            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());
+
+            Configuration conf = HBaseConfiguration.create(getConf());
+            if (User.isHBaseSecurityEnabled(conf)) {
+                // add coprocessor for bulk load
+                tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
+            }
+
+            IIDeployCoprocessorCLI.deployCoprocessor(tableDesc);
+
+            // drop the table first
+            HBaseAdmin admin = new HBaseAdmin(conf);
+            if (admin.tableExists(tableName)) {
+                admin.disableTable(tableName);
+                admin.deleteTable(tableName);
+            }
+
+            // 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++) {
+                    System.out.println("split key " + i + ": " + BytesUtil.toHex(splitKeys[i]));
+                }
+            }
+            System.out.println("create hbase table " + tableName + " done.");
+            admin.close();
+
+            return 0;
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+    }
+
+    //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/incubator-kylin/blob/aa4944d4/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 c634a1d..ff8b659 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,6 +20,7 @@ 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 {
@@ -42,6 +43,23 @@ 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.addCubingGarbageCollectionSteps(jobFlow);
+            }
+        };
+    }
+
+    @Override
     public IMRBatchMergeOutputSide getBatchMergeOutputSide(final CubeSegment seg) {
         return new IMRBatchMergeOutputSide() {
             HBaseMRSteps steps = new HBaseMRSteps(seg);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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 43cbb70..1267d2d 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
@@ -3,14 +3,19 @@ package org.apache.kylin.storage.hbase.steps;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.CubingJob;
 import org.apache.kylin.engine.mr.JobBuilderSupport;
 import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
 import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+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 org.apache.kylin.engine.mr.steps.RangeKeyDistributionJob;
 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 com.google.common.base.Preconditions;
@@ -18,7 +23,7 @@ import com.google.common.collect.Lists;
 
 public class HBaseMRSteps extends JobBuilderSupport {
 
-    public HBaseMRSteps(CubeSegment seg) {
+    public HBaseMRSteps(IRealizationSegment seg) {
         super(seg, null);
     }
 
@@ -42,11 +47,11 @@ public class HBaseMRSteps extends JobBuilderSupport {
         rowkeyDistributionStep.setName(ExecutableConstants.STEP_NAME_GET_CUBOID_KEY_DISTRIBUTION);
         StringBuilder cmd = new StringBuilder();
 
-        appendMapReduceParameters(cmd, seg);
+        appendMapReduceParameters(cmd, seg.getRealization().getDataModelDesc());
         appendExecCmdParameters(cmd, "input", inputPath);
         appendExecCmdParameters(cmd, "output", getRowkeyDistributionOutputPath(jobId));
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Region_Splits_Calculator_" + seg.getCubeInstance().getName() + "_Step");
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Region_Splits_Calculator_" + seg.getRealization().getName() + "_Step");
 
         rowkeyDistributionStep.setMapReduceParams(cmd.toString());
         rowkeyDistributionStep.setMapReduceJobClass(RangeKeyDistributionJob.class);
@@ -65,7 +70,7 @@ public class HBaseMRSteps extends JobBuilderSupport {
         HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
         createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
         StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
         appendExecCmdParameters(cmd, "segmentname", seg.getName());
         appendExecCmdParameters(cmd, "input", getRowkeyDistributionOutputPath(jobId) + "/part-r-00000");
         appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
@@ -84,12 +89,12 @@ public class HBaseMRSteps extends JobBuilderSupport {
         createHFilesStep.setName(ExecutableConstants.STEP_NAME_CONVERT_CUBOID_TO_HFILE);
         StringBuilder cmd = new StringBuilder();
 
-        appendMapReduceParameters(cmd, seg);
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendMapReduceParameters(cmd, seg.getRealization().getDataModelDesc());
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
         appendExecCmdParameters(cmd, "input", inputPath);
         appendExecCmdParameters(cmd, "output", getHFilePath(jobId));
         appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
-        appendExecCmdParameters(cmd, "jobname", "Kylin_HFile_Generator_" + seg.getCubeInstance().getName() + "_Step");
+        appendExecCmdParameters(cmd, "jobname", "Kylin_HFile_Generator_" + seg.getRealization().getName() + "_Step");
 
         createHFilesStep.setMapReduceParams(cmd.toString());
         createHFilesStep.setMapReduceJobClass(CubeHFileJob.class);
@@ -105,7 +110,7 @@ public class HBaseMRSteps extends JobBuilderSupport {
         StringBuilder cmd = new StringBuilder();
         appendExecCmdParameters(cmd, "input", getHFilePath(jobId));
         appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
 
         bulkLoadStep.setJobParams(cmd.toString());
         bulkLoadStep.setJobClass(BulkLoadJob.class);
@@ -121,7 +126,7 @@ public class HBaseMRSteps extends JobBuilderSupport {
     }
 
     public List<String> getMergingHTables() {
-        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(seg);
+        final List<CubeSegment> mergingSegments = ((CubeInstance)seg.getRealization()).getMergingSegments((CubeSegment)seg);
         Preconditions.checkState(mergingSegments.size() > 1, "there should be more than 2 segments to merge");
         final List<String> mergingHTables = Lists.newArrayList();
         for (CubeSegment merging : mergingSegments) {
@@ -131,7 +136,7 @@ public class HBaseMRSteps extends JobBuilderSupport {
     }
 
     public List<String> getMergingHDFSPaths() {
-        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(seg);
+        final List<CubeSegment> mergingSegments = ((CubeInstance)seg.getRealization()).getMergingSegments((CubeSegment)seg);
         Preconditions.checkState(mergingSegments.size() > 1, "there should be more than 2 segments to merge");
         final List<String> mergingHDFSPaths = Lists.newArrayList();
         for (CubeSegment merging : mergingSegments) {
@@ -141,11 +146,11 @@ public class HBaseMRSteps extends JobBuilderSupport {
     }
 
     public String getHFilePath(String jobId) {
-        return HBaseConnection.makeQualifiedPathInHBaseCluster(getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/hfile/");
+        return HBaseConnection.makeQualifiedPathInHBaseCluster(getJobWorkingDir(jobId) + "/" + seg.getRealization().getName() + "/hfile/");
     }
 
     public String getRowkeyDistributionOutputPath(String jobId) {
-        return HBaseConnection.makeQualifiedPathInHBaseCluster(getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats");
+        return HBaseConnection.makeQualifiedPathInHBaseCluster(getJobWorkingDir(jobId) + "/" + seg.getRealization().getName() + "/rowkey_stats");
     }
 
     public void addMergingGarbageCollectionSteps(DefaultChainedExecutable jobFlow) {
@@ -177,4 +182,67 @@ public class HBaseMRSteps extends JobBuilderSupport {
 
         jobFlow.addTask(step);
     }
+
+    public void addSaveIIToHTableSteps(DefaultChainedExecutable jobFlow, String rootPath) {
+        // create htable step
+        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()));
+        
+    }
+
+
+    private HadoopShellExecutable createCreateIIHTableStep(IRealizationSegment seg) {
+        HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
+        createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
+        StringBuilder cmd = new StringBuilder();
+        appendExecCmdParameters(cmd, "iiname", seg.getRealization().getName());
+        appendExecCmdParameters(cmd, "htablename", 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, seg.getRealization().getDataModelDesc());
+        appendExecCmdParameters(cmd, "iiname", seg.getRealization().getName());
+        appendExecCmdParameters(cmd, "input", inputPath);
+        appendExecCmdParameters(cmd, "output", getHFilePath(jobId));
+        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
+        appendExecCmdParameters(cmd, "jobname", "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, "input", getHFilePath(jobId));
+        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
+        appendExecCmdParameters(cmd, "iiname", seg.getRealization().getName());
+
+        bulkLoadStep.setJobParams(cmd.toString());
+        bulkLoadStep.setJobClass(IIBulkLoadJob.class);
+
+        return bulkLoadStep;
+
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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
new file mode 100644
index 0000000..93a9e67
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/IIDeployCoprocessorCLI.java
@@ -0,0 +1,157 @@
+/*
+ * 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;
+    }
+
+}


[13/19] incubator-kylin git commit: fix NPE in IIKeyValueCodec

Posted by sh...@apache.org.
fix NPE in IIKeyValueCodec


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

Branch: refs/heads/KYLIN-1112-2
Commit: fff4a3054a976626d142849e885032fe6a760745
Parents: 662d163
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 11:25:29 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java | 1 -
 .../org/apache/kylin/invertedindex/model/IIKeyValueCodec.java     | 3 ++-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fff4a305/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
index d97b644..b64a7c5 100644
--- a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
@@ -105,7 +105,6 @@ public class BuildIIWithStreamTest {
 
         kylinConfig = KylinConfig.getInstanceFromEnv();
         iiManager = IIManager.getInstance(kylinConfig);
-        iiManager = IIManager.getInstance(kylinConfig);
         for (String iiInstance : II_NAME) {
 
             IIInstance ii = iiManager.getII(iiInstance);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fff4a305/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
index 4f3ab80..7e54a98 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
@@ -19,6 +19,7 @@
 package org.apache.kylin.invertedindex.model;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 
@@ -70,7 +71,7 @@ public class IIKeyValueCodec implements KeyValueCodec {
     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()[col];
+        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 {


[17/19] incubator-kylin git commit: support global dict in IIKeyValueCodec

Posted by sh...@apache.org.
support global dict in IIKeyValueCodec


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

Branch: refs/heads/KYLIN-1112-2
Commit: 37d0602c1d7a0d8af3c1fdf2908cb4917f8b5ad2
Parents: 805a4a6
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 16:03:38 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/37d0602c/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
index 7e54a98..eebf756 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
@@ -199,7 +199,7 @@ public class IIKeyValueCodec implements KeyValueCodec {
                             valueContainers[curCol] = c;
                         } else {
                             final ImmutableBytesWritable dictBytes = row.getDictionary();
-                            if (dictBytes.getLength() != 0) {
+                            if (dictBytes.get() != null && 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());


[15/19] incubator-kylin git commit: reduce base retries in sandbox setting

Posted by sh...@apache.org.
reduce base retries in sandbox setting


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

Branch: refs/heads/KYLIN-1112-2
Commit: d8c689af35c18e50869a6092c026b9cd7f5c80de
Parents: 2d2895a
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 13:40:58 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 examples/test_case_data/sandbox/hbase-site.xml | 19 ++++++++++++++++++-
 1 file changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


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


[08/19] incubator-kylin git commit: enable BuildIIWithEngineTest

Posted by sh...@apache.org.
enable BuildIIWithEngineTest


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

Branch: refs/heads/KYLIN-1112-2
Commit: 604989bbf69c73f89fe3f58e9b936f7e4e21b24a
Parents: aa4944d
Author: shaofengshi <sh...@apache.org>
Authored: Sun Nov 1 17:14:42 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/604989bb/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
index 3c49c74..20493f4 100644
--- a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
@@ -144,7 +144,6 @@ public class BuildIIWithEngineTest {
     }
 
     @Test
-    @Ignore
     public void testBuildII() throws Exception {
 
         String[] testCase = new String[] { "buildIIInnerJoin", "buildIILeftJoin" };


[11/19] incubator-kylin git commit: enable BuildIIWithEngineTest, set KYLIN_CONF

Posted by sh...@apache.org.
enable BuildIIWithEngineTest, set KYLIN_CONF


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

Branch: refs/heads/KYLIN-1112-2
Commit: c6b386210212db3b63d1f55a3485c3c8a3518742
Parents: 604989b
Author: shaofengshi <sh...@apache.org>
Authored: Sun Nov 1 17:53:50 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c6b38621/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
index 20493f4..0158fad 100644
--- a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
@@ -93,6 +93,7 @@ public class BuildIIWithEngineTest {
     public static void beforeClass() throws Exception {
         logger.info("Adding to classpath: " + new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
         ClassUtil.addClasspath(new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
+        System.setProperty(KylinConfig.KYLIN_CONF, "../examples/test_case_data/sandbox");
         if (System.getProperty("hdp.version") == null) {
             throw new RuntimeException("No hdp.version set; Please set hdp.version in your jvm option, for example: -Dhdp.version=2.2.4.2-2");
         }


[03/19] incubator-kylin git commit: KYLIN-1112 on the way: Reorganize InvertedIndex source codes into plug-in architecture

Posted by sh...@apache.org.
KYLIN-1112 on the way: Reorganize InvertedIndex source codes into plug-in architecture 


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

Branch: refs/heads/KYLIN-1112-2
Commit: d921f3ca55a20e095f619a2fc1c585543d05c0c1
Parents: b74a83b
Author: shaofengshi <sh...@apache.org>
Authored: Thu Oct 29 16:52:17 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:16 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/job/BuildIIWithEngineTest.java |   4 +-
 .../apache/kylin/job/BuildIIWithStreamTest.java |   9 +-
 .../kylin/job/hadoop/invertedindex/IICLI.java   | 106 +++++++++
 .../kylin/job/hadoop/invertedindex/IITest.java  |   2 +-
 engine-mr/pom.xml                               |  21 +-
 .../CreateInvertedIndexDictionaryJob.java       |  70 ++++++
 .../engine/mr/invertedindex/IIBulkLoadJob.java  |  74 ++++++
 .../mr/invertedindex/IICreateHFileJob.java      |  81 +++++++
 .../mr/invertedindex/IICreateHFileMapper.java   |  55 +++++
 .../mr/invertedindex/IICreateHTableJob.java     | 148 ++++++++++++
 .../invertedindex/IIDeployCoprocessorCLI.java   | 157 +++++++++++++
 .../IIDistinctColumnsCombiner.java              |  58 +++++
 .../mr/invertedindex/IIDistinctColumnsJob.java  | 136 +++++++++++
 .../invertedindex/IIDistinctColumnsMapper.java  |  66 ++++++
 .../invertedindex/IIDistinctColumnsReducer.java |  77 +++++++
 .../kylin/engine/mr/invertedindex/IIJob.java    |  50 ++++
 .../engine/mr/invertedindex/IIJobBuilder.java   | 224 ++++++++++++++++++
 .../mr/invertedindex/InvertedIndexJob.java      | 164 +++++++++++++
 .../mr/invertedindex/InvertedIndexMapper.java   |  90 ++++++++
 .../invertedindex/InvertedIndexPartitioner.java |  73 ++++++
 .../mr/invertedindex/InvertedIndexReducer.java  | 100 ++++++++
 engine-streaming/pom.xml                        |   5 +
 .../streaming/invertedindex/SliceBuilder.java   |  81 +++++++
 invertedindex/pom.xml                           |  19 +-
 .../invertedindex/streaming/SliceBuilder.java   |  81 -------
 .../dict/CreateInvertedIndexDictionaryJob.java  |  70 ------
 .../job/hadoop/invertedindex/IIBulkLoadJob.java |  74 ------
 .../hadoop/invertedindex/IICreateHFileJob.java  |  81 -------
 .../invertedindex/IICreateHFileMapper.java      |  55 -----
 .../hadoop/invertedindex/IICreateHTableJob.java | 148 ------------
 .../invertedindex/IIDeployCoprocessorCLI.java   | 157 -------------
 .../IIDistinctColumnsCombiner.java              |  58 -----
 .../invertedindex/IIDistinctColumnsJob.java     | 136 -----------
 .../invertedindex/IIDistinctColumnsMapper.java  |  66 ------
 .../invertedindex/IIDistinctColumnsReducer.java |  77 -------
 .../hadoop/invertedindex/InvertedIndexJob.java  | 164 -------------
 .../invertedindex/InvertedIndexMapper.java      |  90 --------
 .../invertedindex/InvertedIndexPartitioner.java |  73 ------
 .../invertedindex/InvertedIndexReducer.java     | 100 --------
 .../apache/kylin/job/invertedindex/IIJob.java   |  50 ----
 .../kylin/job/invertedindex/IIJobBuilder.java   | 230 -------------------
 .../java/org/apache/kylin/job/tools/IICLI.java  | 106 ---------
 42 files changed, 1843 insertions(+), 1843 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
index a3b3db2..47415a8 100644
--- a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
@@ -44,8 +44,8 @@ import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.impl.threadpool.DefaultScheduler;
-import org.apache.kylin.job.invertedindex.IIJob;
-import org.apache.kylin.job.invertedindex.IIJobBuilder;
+import org.apache.kylin.engine.mr.invertedindex.IIJob;
+import org.apache.kylin.engine.mr.invertedindex.IIJobBuilder;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.storage.hbase.steps.HBaseMetadataTestCase;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
index 61cf262..4ccc6b4 100644
--- a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
@@ -40,12 +40,7 @@ import java.io.File;
 import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.*;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingDeque;
 
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -66,11 +61,11 @@ import org.apache.kylin.invertedindex.model.IIDesc;
 import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
 import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
 import org.apache.kylin.invertedindex.model.IIRow;
-import org.apache.kylin.invertedindex.streaming.SliceBuilder;
+import org.apache.kylin.engine.streaming.invertedindex.SliceBuilder;
 import org.apache.kylin.job.common.ShellExecutable;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.hadoop.invertedindex.IICreateHTableJob;
+import org.apache.kylin.engine.mr.invertedindex.IICreateHTableJob;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.source.hive.HiveTableReader;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/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
new file mode 100644
index 0000000..7e7be34
--- /dev/null
+++ b/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IICLI.java
@@ -0,0 +1,106 @@
+/*
+ * 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/incubator-kylin/blob/d921f3ca/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
index e113c06..200156a 100644
--- 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
@@ -35,7 +35,7 @@ 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.invertedindex.streaming.SliceBuilder;
+import org.apache.kylin.engine.streaming.invertedindex.SliceBuilder;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/pom.xml
----------------------------------------------------------------------
diff --git a/engine-mr/pom.xml b/engine-mr/pom.xml
index 7a2bfe5..4500555 100644
--- a/engine-mr/pom.xml
+++ b/engine-mr/pom.xml
@@ -34,7 +34,14 @@
     </properties>
 
     <dependencies>
-    
+
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-core-cube</artifactId>
@@ -42,23 +49,21 @@
         </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-job</artifactId>
+            <artifactId>kylin-invertedindex</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-storage</artifactId>
+            <artifactId>kylin-core-job</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-
-        <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-core-common</artifactId>
-            <type>test-jar</type>
-            <scope>test</scope>
+            <artifactId>kylin-core-storage</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
+
+        <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/CreateInvertedIndexDictionaryJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/CreateInvertedIndexDictionaryJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/CreateInvertedIndexDictionaryJob.java
new file mode 100644
index 0000000..39d74b4
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/CreateInvertedIndexDictionaryJob.java
@@ -0,0 +1,70 @@
+/*
+ * 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.commons.cli.Options;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.dict.DistinctColumnValuesProvider;
+import org.apache.kylin.engine.mr.DFSFileTable;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IIManager;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.source.ReadableTable;
+
+/**
+ */
+public class CreateInvertedIndexDictionaryJob extends AbstractHadoopJob {
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_II_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            parseOptions(options, args);
+
+            final String iiname = getOptionValue(OPTION_II_NAME);
+            final String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
+            final KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+            IIManager mgr = IIManager.getInstance(config);
+            IIInstance ii = mgr.getII(iiname);
+
+            mgr.buildInvertedIndexDictionary(ii.getFirstSegment(), new DistinctColumnValuesProvider() {
+                @Override
+                public ReadableTable getDistinctValuesFor(TblColRef col) {
+                    return new DFSFileTable(factColumnsInputPath + "/" + col.getName(), -1);
+                }
+            });
+            return 0;
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new CreateInvertedIndexDictionaryJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIBulkLoadJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIBulkLoadJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIBulkLoadJob.java
new file mode 100644
index 0000000..a0a5ca6
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIBulkLoadJob.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.invertedindex;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.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.invertedindex.model.IIDesc;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+
+/**
+ */
+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);
+            String input = getOptionValue(OPTION_INPUT_PATH);
+            String iiname = getOptionValue(OPTION_II_NAME);
+
+            FileSystem fs = FileSystem.get(getConf());
+            FsPermission permission = new FsPermission((short) 0777);
+            fs.setPermission(new Path(input, IIDesc.HBASE_FAMILY), permission);
+
+            int hbaseExitCode = ToolRunner.run(new LoadIncrementalHFiles(getConf()), new String[] { input, tableName });
+
+            IIManager mgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
+            IIInstance ii = mgr.getII(iiname);
+            IISegment seg = ii.getFirstSegment();
+            seg.setStorageLocationIdentifier(tableName);
+            seg.setStatus(SegmentStatusEnum.READY);
+            mgr.updateII(ii);
+
+            return hbaseExitCode;
+
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileJob.java
new file mode 100644
index 0000000..4ab3051
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileJob.java
@@ -0,0 +1,81 @@
+/*
+ * 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.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.engine.mr.common.AbstractHadoopJob;
+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));
+
+            setJobClasspath(job);
+
+            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;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileMapper.java
new file mode 100644
index 0000000..fdcc138
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileMapper.java
@@ -0,0 +1,55 @@
+/*
+ * 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.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 {
+
+        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, //
+                value.get(), value.getOffset(), value.getLength());
+
+        context.write(key, kv);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHTableJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHTableJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHTableJob.java
new file mode 100644
index 0000000..3ccd701
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHTableJob.java
@@ -0,0 +1,148 @@
+/*
+ * 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.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;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class IICreateHTableJob extends AbstractHadoopJob {
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        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();
+
+            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());
+
+            Configuration conf = HBaseConfiguration.create(getConf());
+            if (User.isHBaseSecurityEnabled(conf)) {
+                // add coprocessor for bulk load
+                tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
+            }
+
+            IIDeployCoprocessorCLI.deployCoprocessor(tableDesc);
+
+            // drop the table first
+            HBaseAdmin admin = new HBaseAdmin(conf);
+            if (admin.tableExists(tableName)) {
+                admin.disableTable(tableName);
+                admin.deleteTable(tableName);
+            }
+
+            // 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++) {
+                    System.out.println("split key " + i + ": " + BytesUtil.toHex(splitKeys[i]));
+                }
+            }
+            System.out.println("create hbase table " + tableName + " done.");
+            admin.close();
+
+            return 0;
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+    }
+
+    //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/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDeployCoprocessorCLI.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDeployCoprocessorCLI.java
new file mode 100644
index 0000000..da4b95b
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDeployCoprocessorCLI.java
@@ -0,0 +1,157 @@
+/*
+ * 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.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/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsCombiner.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsCombiner.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsCombiner.java
new file mode 100644
index 0000000..651ad63
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsCombiner.java
@@ -0,0 +1,58 @@
+/*
+ * 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.util.HashSet;
+
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.engine.mr.KylinReducer;
+
+/**
+ * @author yangli9
+ */
+public class IIDistinctColumnsCombiner extends KylinReducer<ShortWritable, Text, ShortWritable, Text> {
+
+    private Text outputValue = new Text();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+    }
+
+    @Override
+    public void reduce(ShortWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+
+        HashSet<ByteArray> set = new HashSet<ByteArray>();
+        for (Text textValue : values) {
+            ByteArray value = new ByteArray(Bytes.copy(textValue.getBytes(), 0, textValue.getLength()));
+            set.add(value);
+        }
+
+        for (ByteArray value : set) {
+            outputValue.set(value.array(), value.offset(), value.length());
+            context.write(key, outputValue);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
new file mode 100644
index 0000000..c9f5375
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
@@ -0,0 +1,136 @@
+/*
+ * 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.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+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.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.engine.mr.HadoopUtil;
+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.model.IIJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.IntermediateColumnDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author yangli9
+ */
+public class IIDistinctColumnsJob extends AbstractHadoopJob {
+    protected static final Logger logger = LoggerFactory.getLogger(IIDistinctColumnsJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_TABLE_NAME);
+            options.addOption(OPTION_II_NAME);
+            options.addOption(OPTION_OUTPUT_PATH);
+            parseOptions(options, args);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            String tableName = getOptionValue(OPTION_TABLE_NAME).toUpperCase();
+            String iiName = getOptionValue(OPTION_II_NAME);
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+
+            // ----------------------------------------------------------------------------
+
+            logger.info("Starting: " + job.getJobName() + " on table " + tableName);
+
+            IIManager iiMgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
+            IIInstance ii = iiMgr.getII(iiName);
+            job.getConfiguration().set(BatchConstants.TABLE_NAME, tableName);
+            job.getConfiguration().set(BatchConstants.TABLE_COLUMNS, getColumns(ii));
+
+            setJobClasspath(job);
+
+            setupMapper();
+            setupReducer(output);
+
+            return waitForCompletion(job);
+
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+
+    }
+
+    private String getColumns(IIInstance ii) {
+        IIJoinedFlatTableDesc iiflat = new IIJoinedFlatTableDesc(ii.getDescriptor());
+        StringBuilder buf = new StringBuilder();
+        for (IntermediateColumnDesc col : iiflat.getColumnList()) {
+            if (buf.length() > 0)
+                buf.append(",");
+            buf.append(col.getColumnName());
+        }
+        return buf.toString();
+    }
+
+    private void setupMapper() throws IOException {
+
+        String tableName = job.getConfiguration().get(BatchConstants.TABLE_NAME);
+        String[] dbTableNames = HadoopUtil.parseHiveTableName(tableName);
+
+        logger.info("setting hcat input format, db name {} , table name {}", dbTableNames[0], dbTableNames[1]);
+
+        HCatInputFormat.setInput(job, dbTableNames[0], dbTableNames[1]);
+
+        job.setInputFormatClass(HCatInputFormat.class);
+
+        job.setMapperClass(IIDistinctColumnsMapper.class);
+        job.setCombinerClass(IIDistinctColumnsCombiner.class);
+        job.setMapOutputKeyClass(ShortWritable.class);
+        job.setMapOutputValueClass(Text.class);
+    }
+
+    private void setupReducer(Path output) throws IOException {
+        job.setReducerClass(IIDistinctColumnsReducer.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        job.setOutputKeyClass(NullWritable.class);
+        job.setOutputValueClass(Text.class);
+
+        FileOutputFormat.setOutputPath(job, output);
+        job.getConfiguration().set(BatchConstants.OUTPUT_PATH, output.toString());
+
+        job.setNumReduceTasks(1);
+
+        deletePath(job.getConfiguration(), output);
+    }
+
+    public static void main(String[] args) throws Exception {
+        IIDistinctColumnsJob job = new IIDistinctColumnsJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsMapper.java
new file mode 100644
index 0000000..0f0c731
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsMapper.java
@@ -0,0 +1,66 @@
+/*
+ * 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.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hive.hcatalog.data.HCatRecord;
+import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
+import org.apache.hive.hcatalog.data.schema.HCatSchema;
+import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.engine.mr.KylinMapper;
+
+/**
+ * @author yangli9
+ */
+public class IIDistinctColumnsMapper<KEYIN> extends KylinMapper<KEYIN, HCatRecord, ShortWritable, Text> {
+
+    private ShortWritable outputKey = new ShortWritable();
+    private Text outputValue = new Text();
+    private HCatSchema schema = null;
+    private int columnSize = 0;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+        schema = HCatInputFormat.getTableSchema(context.getConfiguration());
+        columnSize = schema.getFields().size();
+    }
+
+    @Override
+    public void map(KEYIN key, HCatRecord record, Context context) throws IOException, InterruptedException {
+
+        HCatFieldSchema fieldSchema = null;
+        for (short i = 0; i < columnSize; i++) {
+            outputKey.set(i);
+            fieldSchema = schema.get(i);
+            Object fieldValue = record.get(fieldSchema.getName(), schema);
+            if (fieldValue == null)
+                continue;
+            byte[] bytes = Bytes.toBytes(fieldValue.toString());
+            outputValue.set(bytes, 0, bytes.length);
+            context.write(outputKey, outputValue);
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsReducer.java
new file mode 100644
index 0000000..d50385f
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsReducer.java
@@ -0,0 +1,77 @@
+/*
+ * 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.util.HashSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.engine.mr.KylinReducer;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+
+/**
+ * @author yangli9
+ */
+public class IIDistinctColumnsReducer extends KylinReducer<ShortWritable, Text, NullWritable, Text> {
+
+    private String[] columns;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+        Configuration conf = context.getConfiguration();
+        this.columns = conf.get(BatchConstants.TABLE_COLUMNS).split(",");
+    }
+
+    @Override
+    public void reduce(ShortWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+        String columnName = columns[key.get()];
+
+        HashSet<ByteArray> set = new HashSet<ByteArray>();
+        for (Text textValue : values) {
+            ByteArray value = new ByteArray(Bytes.copy(textValue.getBytes(), 0, textValue.getLength()));
+            set.add(value);
+        }
+
+        Configuration conf = context.getConfiguration();
+        FileSystem fs = FileSystem.get(conf);
+        String outputPath = conf.get(BatchConstants.OUTPUT_PATH);
+        FSDataOutputStream out = fs.create(new Path(outputPath, columnName));
+
+        try {
+            for (ByteArray value : set) {
+                out.write(value.array(), value.offset(), value.length());
+                out.write('\n');
+            }
+        } finally {
+            out.close();
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/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
new file mode 100644
index 0000000..30e653b
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJob.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.invertedindex;
+
+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);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJobBuilder.java
new file mode 100644
index 0000000..2605e65
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIJobBuilder.java
@@ -0,0 +1,224 @@
+/*
+ * 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.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
+import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
+import org.apache.kylin.source.hive.HiveMRInput.BatchCubingInputSide;
+
+import com.google.common.base.Preconditions;
+
+/**
+ */
+public final class IIJobBuilder {
+
+    final JobEngineConfig engineConfig;
+
+    public IIJobBuilder(JobEngineConfig engineConfig) {
+        this.engineConfig = engineConfig;
+    }
+
+    public IIJob buildJob(IISegment seg, String submitter) {
+        checkPreconditions(seg);
+
+        IIJob result = initialJob(seg, "BUILD", submitter);
+        final String jobId = result.getId();
+        final IIJoinedFlatTableDesc intermediateTableDesc = new IIJoinedFlatTableDesc(seg.getIIDesc());
+        final String intermediateTableIdentity = getIntermediateTableIdentity(intermediateTableDesc);
+        final String factDistinctColumnsPath = getIIDistinctColumnsPath(seg, jobId);
+        final String iiRootPath = getJobWorkingDir(jobId) + "/" + seg.getIIInstance().getName() + "/";
+        final String iiPath = iiRootPath + "*";
+
+        final AbstractExecutable intermediateHiveTableStep = createFlatHiveTableStep(intermediateTableDesc, jobId);
+        result.addTask(intermediateHiveTableStep);
+
+        result.addTask(createFactDistinctColumnsStep(seg, intermediateTableIdentity, jobId, factDistinctColumnsPath));
+
+        result.addTask(createBuildDictionaryStep(seg, factDistinctColumnsPath));
+
+        result.addTask(createInvertedIndexStep(seg, intermediateTableIdentity, iiRootPath));
+
+        // create htable step
+        result.addTask(createCreateHTableStep(seg));
+
+        // generate hfiles step
+        result.addTask(createConvertToHfileStep(seg, iiPath, jobId));
+
+        // bulk load step
+        result.addTask(createBulkLoadStep(seg, jobId));
+
+        return result;
+    }
+
+    private AbstractExecutable createFlatHiveTableStep(IIJoinedFlatTableDesc intermediateTableDesc, String jobId) {
+        return BatchCubingInputSide.createFlatHiveTableStep(engineConfig, intermediateTableDesc, jobId);
+    }
+
+    private IIJob initialJob(IISegment seg, String type, String submitter) {
+        IIJob result = new IIJob();
+        SimpleDateFormat format = new SimpleDateFormat("z yyyy-MM-dd HH:mm:ss");
+        format.setTimeZone(TimeZone.getTimeZone(engineConfig.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;
+    }
+
+    private void checkPreconditions(IISegment seg) {
+        Preconditions.checkNotNull(seg, "segment cannot be null");
+        Preconditions.checkNotNull(engineConfig, "jobEngineConfig cannot be null");
+    }
+
+    private void appendMapReduceParameters(StringBuilder builder, JobEngineConfig engineConfig) {
+        try {
+            String jobConf = engineConfig.getHadoopJobConfFilePath(RealizationCapacity.MEDIUM);
+            if (jobConf != null && jobConf.length() > 0) {
+                builder.append(" -conf ").append(jobConf);
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private String getIIDistinctColumnsPath(IISegment seg, String jobUuid) {
+        return getJobWorkingDir(jobUuid) + "/" + seg.getIIInstance().getName() + "/ii_distinct_columns";
+    }
+
+    private String getHFilePath(IISegment seg, String jobId) {
+        return getJobWorkingDir(jobId) + "/" + seg.getIIInstance().getName() + "/hfile/";
+    }
+
+    private MapReduceExecutable createFactDistinctColumnsStep(IISegment seg, String factTableName, String jobId, String output) {
+        MapReduceExecutable result = new MapReduceExecutable();
+        result.setName(ExecutableConstants.STEP_NAME_FACT_DISTINCT_COLUMNS);
+        result.setMapReduceJobClass(IIDistinctColumnsJob.class);
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd, engineConfig);
+        appendExecCmdParameters(cmd, "tablename", factTableName);
+        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+        appendExecCmdParameters(cmd, "output", output);
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Fact_Distinct_Columns_" + seg.getIIInstance().getName() + "_Step");
+
+        result.setMapReduceParams(cmd.toString());
+        return result;
+    }
+
+    private HadoopShellExecutable createBuildDictionaryStep(IISegment seg, String factDistinctColumnsPath) {
+        // base cuboid job
+        HadoopShellExecutable buildDictionaryStep = new HadoopShellExecutable();
+        buildDictionaryStep.setName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY);
+        StringBuilder cmd = new StringBuilder();
+        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+        appendExecCmdParameters(cmd, "input", factDistinctColumnsPath);
+
+        buildDictionaryStep.setJobParams(cmd.toString());
+        buildDictionaryStep.setJobClass(CreateInvertedIndexDictionaryJob.class);
+        return buildDictionaryStep;
+    }
+
+    private MapReduceExecutable createInvertedIndexStep(IISegment seg, String intermediateHiveTable, String iiOutputTempPath) {
+        // base cuboid job
+        MapReduceExecutable buildIIStep = new MapReduceExecutable();
+
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd, engineConfig);
+
+        buildIIStep.setName(ExecutableConstants.STEP_NAME_BUILD_II);
+
+        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+        appendExecCmdParameters(cmd, "tablename", intermediateHiveTable);
+        appendExecCmdParameters(cmd, "output", iiOutputTempPath);
+        appendExecCmdParameters(cmd, "jobname", ExecutableConstants.STEP_NAME_BUILD_II);
+
+        buildIIStep.setMapReduceParams(cmd.toString());
+        buildIIStep.setMapReduceJobClass(InvertedIndexJob.class);
+        return buildIIStep;
+    }
+
+    private HadoopShellExecutable createCreateHTableStep(IISegment seg) {
+        HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
+        createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
+        StringBuilder cmd = new StringBuilder();
+        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
+
+        createHtableStep.setJobParams(cmd.toString());
+        createHtableStep.setJobClass(IICreateHTableJob.class);
+
+        return createHtableStep;
+    }
+
+    private MapReduceExecutable createConvertToHfileStep(IISegment seg, String inputPath, String jobId) {
+        MapReduceExecutable createHFilesStep = new MapReduceExecutable();
+        createHFilesStep.setName(ExecutableConstants.STEP_NAME_CONVERT_II_TO_HFILE);
+        StringBuilder cmd = new StringBuilder();
+
+        appendMapReduceParameters(cmd, engineConfig);
+        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+        appendExecCmdParameters(cmd, "input", inputPath);
+        appendExecCmdParameters(cmd, "output", getHFilePath(seg, jobId));
+        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
+        appendExecCmdParameters(cmd, "jobname", "Kylin_HFile_Generator_" + seg.getIIInstance().getName() + "_Step");
+
+        createHFilesStep.setMapReduceParams(cmd.toString());
+        createHFilesStep.setMapReduceJobClass(IICreateHFileJob.class);
+
+        return createHFilesStep;
+    }
+
+    private HadoopShellExecutable createBulkLoadStep(IISegment seg, String jobId) {
+        HadoopShellExecutable bulkLoadStep = new HadoopShellExecutable();
+        bulkLoadStep.setName(ExecutableConstants.STEP_NAME_BULK_LOAD_HFILE);
+
+        StringBuilder cmd = new StringBuilder();
+        appendExecCmdParameters(cmd, "input", getHFilePath(seg, jobId));
+        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
+        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
+
+        bulkLoadStep.setJobParams(cmd.toString());
+        bulkLoadStep.setJobClass(IIBulkLoadJob.class);
+
+        return bulkLoadStep;
+
+    }
+
+    private StringBuilder appendExecCmdParameters(StringBuilder buf, String paraName, String paraValue) {
+        return buf.append(" -").append(paraName).append(" ").append(paraValue);
+    }
+
+    private String getJobWorkingDir(String uuid) {
+        return engineConfig.getHdfsWorkingDirectory() + "kylin-" + uuid;
+    }
+
+    private String getIntermediateTableIdentity(IIJoinedFlatTableDesc intermediateTableDesc) {
+        return engineConfig.getConfig().getHiveDatabaseForIntermediateTable() + "." + intermediateTableDesc.getTableName();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/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
new file mode 100644
index 0000000..fc5f939
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexJob.java
@@ -0,0 +1,164 @@
+/*
+ * 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.util.ArrayList;
+
+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.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.engine.mr.HadoopUtil;
+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.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author yangli9
+ */
+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_TABLE_NAME);
+            options.addOption(OPTION_OUTPUT_PATH);
+            parseOptions(options, args);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            String iiname = getOptionValue(OPTION_II_NAME);
+            String intermediateTable = getOptionValue(OPTION_TABLE_NAME);
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+
+            // ----------------------------------------------------------------------------
+
+            System.out.println("Starting: " + job.getJobName());
+
+            IIInstance ii = getII(iiname);
+            short sharding = ii.getDescriptor().getSharding();
+
+            setJobClasspath(job);
+
+            setupMapper(intermediateTable);
+            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());
+    }
+
+    protected void attachKylinPropsAndMetadata(IIInstance ii, Configuration conf) throws IOException {
+        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+        // 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());
+        }
+        for (IISegment segment : ii.getSegments()) {
+            dumpList.addAll(segment.getDictionaryPaths());
+        }
+
+        attachKylinPropsAndMetadata(dumpList, conf);
+    }
+
+    private void setupMapper(String intermediateTable) throws IOException {
+
+        String[] dbTableNames = HadoopUtil.parseHiveTableName(intermediateTable);
+        HCatInputFormat.setInput(job, dbTableNames[0], dbTableNames[1]);
+
+        job.setInputFormatClass(HCatInputFormat.class);
+
+        job.setMapperClass(InvertedIndexMapper.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(ImmutableBytesWritable.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.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/incubator-kylin/blob/d921f3ca/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
new file mode 100644
index 0000000..81921f1
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexMapper.java
@@ -0,0 +1,90 @@
+/*
+ * 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.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hive.hcatalog.data.HCatRecord;
+import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
+import org.apache.hive.hcatalog.data.schema.HCatSchema;
+import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.engine.mr.KylinMapper;
+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 InvertedIndexMapper<KEYIN> extends KylinMapper<KEYIN, HCatRecord, LongWritable, ImmutableBytesWritable> {
+
+    private TableRecordInfo info;
+    private TableRecord rec;
+
+    private LongWritable outputKey;
+    private ImmutableBytesWritable outputValue;
+    private HCatSchema schema = null;
+    private List<HCatFieldSchema> fields;
+
+    @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);
+        this.rec = this.info.createTableRecord();
+
+        outputKey = new LongWritable();
+        outputValue = new ImmutableBytesWritable(rec.getBytes());
+
+        schema = HCatInputFormat.getTableSchema(context.getConfiguration());
+
+        fields = schema.getFields();
+    }
+
+    @Override
+    public void map(KEYIN key, HCatRecord record, Context context) throws IOException, InterruptedException {
+
+        rec.reset();
+        for (int i = 0; i < fields.size(); i++) {
+            Object fieldValue = record.get(i);
+            rec.setValueString(i, fieldValue == null ? null : fieldValue.toString());
+        }
+
+        outputKey.set(rec.getTimestamp());
+        // outputValue's backing bytes array is the same as rec
+
+        context.write(outputKey, outputValue);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/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
new file mode 100644
index 0000000..dcf707f
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexPartitioner.java
@@ -0,0 +1,73 @@
+/*
+ * 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/incubator-kylin/blob/d921f3ca/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
new file mode 100644
index 0000000..7644456
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/InvertedIndexReducer.java
@@ -0,0 +1,100 @@
+/*
+ * 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.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.engine.mr.KylinReducer;
+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.IncrementalSliceMaker;
+import org.apache.kylin.invertedindex.index.Slice;
+import org.apache.kylin.invertedindex.index.TableRecord;
+import org.apache.kylin.invertedindex.index.TableRecordInfo;
+import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
+import org.apache.kylin.invertedindex.model.IIRow;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+
+/**
+ * @author yangli9
+ */
+public class InvertedIndexReducer extends KylinReducer<LongWritable, ImmutableBytesWritable, ImmutableBytesWritable, ImmutableBytesWritable> {
+
+    private TableRecordInfo info;
+    private TableRecord rec;
+    private IncrementalSliceMaker builder;
+    private IIKeyValueCodec kv;
+
+    @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);
+        info = new TableRecordInfo(seg);
+        rec = info.createTableRecord();
+        builder = null;
+        kv = new IIKeyValueCodec(info.getDigest());
+    }
+
+    @Override
+    public void reduce(LongWritable key, Iterable<ImmutableBytesWritable> values, Context context) //
+            throws IOException, InterruptedException {
+        for (ImmutableBytesWritable v : values) {
+            rec.setBytes(v.get(), v.getOffset(), v.getLength());
+
+            if (builder == null) {
+                builder = new IncrementalSliceMaker(info, rec.getShard());
+            }
+
+            //TODO: to delete this log
+            System.out.println(rec.getShard() + " - " + rec);
+
+            Slice slice = builder.append(rec);
+            if (slice != null) {
+                output(slice, context);
+            }
+        }
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        Slice slice = builder.close();
+        if (slice != null) {
+            output(slice, context);
+        }
+    }
+
+    private void output(Slice slice, Context context) throws IOException, InterruptedException {
+        for (IIRow pair : kv.encodeKeyValue(slice)) {
+            context.write(pair.getKey(), pair.getValue());
+        }
+    }
+
+}



[14/19] incubator-kylin git commit: another NPE

Posted by sh...@apache.org.
another NPE


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

Branch: refs/heads/KYLIN-1112-2
Commit: 2d2895ae142762cbbb5d4342d7b3f6fee96eb6ad
Parents: fff4a30
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 13:31:28 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/engine/mr/invertedindex/InvertedIndexMapper.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d2895ae/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
index 8247295..670972a 100644
--- 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
@@ -75,7 +75,8 @@ public class InvertedIndexMapper<KEYIN> extends KylinMapper<KEYIN, Object, LongW
         rec.reset();
         for (int i = 0; i < row.length; i++) {
             Object fieldValue = row[i];
-            rec.setValueString(i, fieldValue == null ? null : fieldValue.toString());
+            if (fieldValue != null)
+                rec.setValueString(i, fieldValue == null ? null : fieldValue.toString());
         }
 
         outputKey.set(rec.getTimestamp());


[07/19] incubator-kylin git commit: KYLIN-1112 make code compile

Posted by sh...@apache.org.
KYLIN-1112 make code compile


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

Branch: refs/heads/KYLIN-1112-2
Commit: aa4944d41daafcf63bbfaf1c757f466b2a3a204d
Parents: d921f3c
Author: shaofengshi <sh...@apache.org>
Authored: Fri Oct 30 22:01:27 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/job/BuildIIWithEngineTest.java |   7 +-
 .../apache/kylin/job/BuildIIWithStreamTest.java |   2 +-
 .../kylin/common/util/ImplementationSwitch.java |   4 +-
 .../java/org/apache/kylin/cube/CubeSegment.java |  18 +-
 .../kylin/job/constant/ExecutableConstants.java |   1 +
 .../kylin/metadata/model/IEngineAware.java      |   1 +
 .../realization/IRealizationSegment.java        |  20 +
 .../kylin/engine/mr/BatchCubingJobBuilder.java  |  23 +-
 .../kylin/engine/mr/BatchCubingJobBuilder2.java |  12 +-
 .../kylin/engine/mr/BatchMergeJobBuilder.java   |  12 +-
 .../kylin/engine/mr/BatchMergeJobBuilder2.java  |  15 +-
 .../org/apache/kylin/engine/mr/IMRInput.java    |   4 +-
 .../org/apache/kylin/engine/mr/IMROutput.java   |  27 ++
 .../kylin/engine/mr/JobBuilderSupport.java      |  46 +-
 .../java/org/apache/kylin/engine/mr/MRUtil.java |  11 +-
 .../mr/invertedindex/BatchIIJobBuilder.java     | 132 ++++++
 .../engine/mr/invertedindex/IIBulkLoadJob.java  |  74 ----
 .../mr/invertedindex/IICreateHFileJob.java      |  81 ----
 .../mr/invertedindex/IICreateHFileMapper.java   |  55 ---
 .../mr/invertedindex/IICreateHTableJob.java     | 148 -------
 .../invertedindex/IIDeployCoprocessorCLI.java   | 157 -------
 .../mr/invertedindex/IIDistinctColumnsJob.java  |  22 +-
 .../invertedindex/IIDistinctColumnsMapper.java  |  41 +-
 .../kylin/engine/mr/invertedindex/IIJob.java    |  23 +
 .../engine/mr/invertedindex/IIJobBuilder.java   | 443 +++++++++----------
 .../mr/invertedindex/InvertedIndexJob.java      |  17 +-
 .../mr/invertedindex/InvertedIndexMapper.java   |  22 +-
 .../UpdateInvertedIndexInfoAfterBuildStep.java  |  93 ++++
 .../engine/spark/SparkCubingJobBuilder.java     |   7 +-
 invertedindex/pom.xml                           |   4 +
 .../apache/kylin/invertedindex/IIInstance.java  |  18 +-
 .../apache/kylin/invertedindex/IISegment.java   |  34 +-
 .../kylin/invertedindex/model/IIDesc.java       |  32 +-
 server/pom.xml                                  |   5 +
 .../apache/kylin/source/hive/HiveMRInput.java   |  13 +-
 .../kylin/storage/hbase/ii/IIBulkLoadJob.java   |  64 +++
 .../storage/hbase/ii/IICreateHFileJob.java      |  81 ++++
 .../storage/hbase/ii/IICreateHFileMapper.java   |  55 +++
 .../storage/hbase/ii/IICreateHTableJob.java     | 149 +++++++
 .../storage/hbase/steps/HBaseMROutput.java      |  18 +
 .../kylin/storage/hbase/steps/HBaseMRSteps.java |  94 +++-
 .../hbase/util/IIDeployCoprocessorCLI.java      | 157 +++++++
 42 files changed, 1362 insertions(+), 880 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
index 47415a8..3c49c74 100644
--- a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.engine.mr.invertedindex.BatchIIJobBuilder;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;
@@ -45,7 +46,6 @@ import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.impl.threadpool.DefaultScheduler;
 import org.apache.kylin.engine.mr.invertedindex.IIJob;
-import org.apache.kylin.engine.mr.invertedindex.IIJobBuilder;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.storage.hbase.steps.HBaseMetadataTestCase;
@@ -225,8 +225,9 @@ public class BuildIIWithEngineTest {
         IISegment segment = iiManager.buildSegment(iiInstance, startDate, endDate);
         iiInstance.getSegments().add(segment);
         iiManager.updateII(iiInstance);
-        IIJobBuilder iiJobBuilder = new IIJobBuilder(jobEngineConfig);
-        IIJob job = iiJobBuilder.buildJob(segment, "TEST");
+
+        BatchIIJobBuilder batchIIJobBuilder = new BatchIIJobBuilder(segment, "SYSTEM");
+        IIJob job = batchIIJobBuilder.build();
         jobService.addJob(job);
         waitForJob(job.getId());
         return job.getId();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
index 4ccc6b4..d97b644 100644
--- a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
@@ -65,7 +65,7 @@ import org.apache.kylin.engine.streaming.invertedindex.SliceBuilder;
 import org.apache.kylin.job.common.ShellExecutable;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.engine.mr.invertedindex.IICreateHTableJob;
+import org.apache.kylin.storage.hbase.ii.IICreateHTableJob;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.source.hive.HiveTableReader;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/core-common/src/main/java/org/apache/kylin/common/util/ImplementationSwitch.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/ImplementationSwitch.java b/core-common/src/main/java/org/apache/kylin/common/util/ImplementationSwitch.java
index f6924a0..4a47b83 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/ImplementationSwitch.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ImplementationSwitch.java
@@ -44,7 +44,7 @@ public class ImplementationSwitch<I> {
             maxId = Math.max(maxId, id);
         }
         if (maxId > 100)
-            throw new IllegalArgumentException("you have more than 100 implentations?");
+            throw new IllegalArgumentException("you have more than 100 implementations?");
 
         Object[] result = new Object[maxId + 1];
 
@@ -65,7 +65,7 @@ public class ImplementationSwitch<I> {
         I result = (I) instances[id];
 
         if (result == null)
-            throw new IllegalArgumentException("Implementations missing, ID " + id + ", interafce " + interfaceClz.getName());
+            throw new IllegalArgumentException("Implementations missing, ID " + id + ", interface " + interfaceClz.getName());
 
         return result;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index 1a44fcf..7d17d30 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -27,9 +27,10 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.ShardingHash;
 import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.IDictionaryAware;
-import org.apache.kylin.metadata.model.IBuildable;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -39,9 +40,11 @@ import com.fasterxml.jackson.annotation.JsonBackReference;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Objects;
 import com.google.common.collect.Maps;
+import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.metadata.realization.IRealizationSegment;
 
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, IBuildable {
+public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, IRealizationSegment {
 
     @JsonBackReference
     private CubeInstance cubeInstance;
@@ -115,6 +118,7 @@ public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, I
         this.uuid = id;
     }
 
+    @Override
     public String getName() {
         return name;
     }
@@ -211,6 +215,7 @@ public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, I
         this.cubeInstance = cubeInstance;
     }
 
+    @Override
     public String getStorageLocationIdentifier() {
 
         return storageLocationIdentifier;
@@ -410,4 +415,13 @@ public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, I
         return ret;
     }
 
+    @Override
+    public IRealization getRealization() {
+        return cubeInstance;
+    }
+
+    @Override
+    public IJoinedFlatTableDesc getJoinedFlatTableDesc() {
+        return new CubeJoinedFlatTableDesc(this.getCubeDesc(), this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java b/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
index 30cefaf..ba50880 100644
--- a/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
+++ b/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
@@ -59,6 +59,7 @@ public final class ExecutableConstants {
 
     public static final String STEP_NAME_BUILD_II = "Build Inverted Index";
     public static final String STEP_NAME_CONVERT_II_TO_HFILE = "Convert Inverted Index Data to HFile";
+    public static final String STEP_NAME_UPDATE_II_INFO = "Update Inverted Index Info";
 
     public static final String PROP_ENGINE_CONTEXT = "jobengineConfig";
     public static final String PROP_JOB_FLOW = "jobFlow";

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/core-metadata/src/main/java/org/apache/kylin/metadata/model/IEngineAware.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IEngineAware.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IEngineAware.java
index 60bd825..882b2e3 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IEngineAware.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IEngineAware.java
@@ -22,6 +22,7 @@ public interface IEngineAware {
 
     public static final int ID_MR_V1 = 0;
     public static final int ID_MR_V2 = 2;
+    public static final int ID_MR_II = 3;
     public static final int ID_SPARK = 5;
 
     int getEngineType();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealizationSegment.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealizationSegment.java b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealizationSegment.java
new file mode 100644
index 0000000..afab86b
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealizationSegment.java
@@ -0,0 +1,20 @@
+package org.apache.kylin.metadata.realization;
+
+import org.apache.kylin.metadata.model.IBuildable;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+
+/**
+ * Created by shaoshi on 10/30/15.
+ */
+public interface IRealizationSegment extends IBuildable {
+
+    public String getUuid();
+    
+    public String getName();
+
+    public String getStorageLocationIdentifier();
+    
+    public IRealization getRealization();
+    
+    public IJoinedFlatTableDesc getJoinedFlatTableDesc();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
index d00f592..dcb887d 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
@@ -19,6 +19,7 @@
 package org.apache.kylin.engine.mr;
 
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.RowKeyDesc;
 import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
 import org.apache.kylin.engine.mr.IMROutput.IMRBatchCubingOutputSide;
 import org.apache.kylin.engine.mr.common.MapReduceExecutable;
@@ -38,13 +39,12 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
     public BatchCubingJobBuilder(CubeSegment newSegment, String submitter) {
         super(newSegment, submitter);
         this.inputSide = MRUtil.getBatchCubingInputSide(seg);
-        this.outputSide = MRUtil.getBatchCubingOutputSide(seg);
+        this.outputSide = MRUtil.getBatchCubingOutputSide((CubeSegment)seg);
     }
 
     public CubingJob build() {
         logger.info("MR_V1 new job to BUILD segment " + seg);
-        
-        final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
+        final CubingJob result = CubingJob.createBuildJob((CubeSegment)seg, submitter, config);
         final String jobId = result.getId();
         final String cuboidRootPath = getCuboidRootPath(jobId);
 
@@ -56,8 +56,9 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
         result.addTask(createBuildDictionaryStep(jobId));
 
         // Phase 3: Build Cube
-        final int groupRowkeyColumnsCount = seg.getCubeDesc().getRowkey().getNCuboidBuildLevels();
-        final int totalRowkeyColumnsCount = seg.getCubeDesc().getRowkey().getRowKeyColumns().length;
+        RowKeyDesc rowKeyDesc = ((CubeSegment)seg).getCubeDesc().getRowkey();
+        final int groupRowkeyColumnsCount = rowKeyDesc.getNCuboidBuildLevels();
+        final int totalRowkeyColumnsCount = rowKeyDesc.getRowKeyColumns().length;
         final String[] cuboidOutputTempPath = getCuboidOutputPaths(cuboidRootPath, totalRowkeyColumnsCount, groupRowkeyColumnsCount);
         // base cuboid step
         result.addTask(createBaseCuboidStep(cuboidOutputTempPath, jobId));
@@ -81,15 +82,15 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
         MapReduceExecutable baseCuboidStep = new MapReduceExecutable();
 
         StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd, seg);
+        appendMapReduceParameters(cmd, ((CubeSegment)seg).getCubeDesc().getModel());
 
         baseCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_BASE_CUBOID);
 
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
         appendExecCmdParameters(cmd, "segmentname", seg.getName());
         appendExecCmdParameters(cmd, "input", "FLAT_TABLE"); // marks flat table input
         appendExecCmdParameters(cmd, "output", cuboidOutputTempPath[0]);
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Base_Cuboid_Builder_" + seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Base_Cuboid_Builder_" + seg.getRealization().getName());
         appendExecCmdParameters(cmd, "level", "0");
 
         baseCuboidStep.setMapReduceParams(cmd.toString());
@@ -105,12 +106,12 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
         ndCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_N_D_CUBOID + " : " + dimNum + "-Dimension");
         StringBuilder cmd = new StringBuilder();
 
-        appendMapReduceParameters(cmd, seg);
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendMapReduceParameters(cmd, ((CubeSegment)seg).getCubeDesc().getModel());
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
         appendExecCmdParameters(cmd, "segmentname", seg.getName());
         appendExecCmdParameters(cmd, "input", cuboidOutputTempPath[totalRowkeyColumnCount - dimNum - 1]);
         appendExecCmdParameters(cmd, "output", cuboidOutputTempPath[totalRowkeyColumnCount - dimNum]);
-        appendExecCmdParameters(cmd, "jobname", "Kylin_ND-Cuboid_Builder_" + seg.getCubeInstance().getName() + "_Step");
+        appendExecCmdParameters(cmd, "jobname", "Kylin_ND-Cuboid_Builder_" + seg.getRealization().getName() + "_Step");
         appendExecCmdParameters(cmd, "level", "" + (totalRowkeyColumnCount - dimNum));
 
         ndCuboidStep.setMapReduceParams(cmd.toString());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
index fe9f1d6..f8fbc33 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
@@ -37,13 +37,13 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
     public BatchCubingJobBuilder2(CubeSegment newSegment, String submitter) {
         super(newSegment, submitter);
         this.inputSide = MRUtil.getBatchCubingInputSide(seg);
-        this.outputSide = MRUtil.getBatchCubingOutputSide2(seg);
+        this.outputSide = MRUtil.getBatchCubingOutputSide2((CubeSegment)seg);
     }
 
     public CubingJob build() {
         logger.info("MR_V2 new job to BUILD segment " + seg);
         
-        final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
+        final CubingJob result = CubingJob.createBuildJob((CubeSegment)seg, submitter, config);
         final String jobId = result.getId();
 
         // Phase 1: Create Flat Table
@@ -70,7 +70,7 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
     private SaveStatisticsStep createSaveStatisticsStep(String jobId) {
         SaveStatisticsStep result = new SaveStatisticsStep();
         result.setName(ExecutableConstants.STEP_NAME_SAVE_STATISTICS);
-        result.setCubeName(seg.getCubeInstance().getName());
+        result.setCubeName(seg.getRealization().getName());
         result.setSegmentId(seg.getUuid());
         result.setStatisticsPath(getStatisticsPath(jobId));
         return result;
@@ -81,13 +81,13 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
         MapReduceExecutable cubeStep = new MapReduceExecutable();
 
         StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd, seg);
+        appendMapReduceParameters(cmd, ((CubeSegment)seg).getCubeDesc().getModel());
 
         cubeStep.setName(ExecutableConstants.STEP_NAME_BUILD_IN_MEM_CUBE);
 
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
         appendExecCmdParameters(cmd, "segmentname", seg.getName());
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Cube_Builder_" + seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Cube_Builder_" + seg.getRealization().getName());
         appendExecCmdParameters(cmd, "jobflowid", jobId);
 
         cubeStep.setMapReduceParams(cmd.toString());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
index bc377ed..4b93b5d 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
@@ -39,17 +39,17 @@ public class BatchMergeJobBuilder extends JobBuilderSupport {
 
     public BatchMergeJobBuilder(CubeSegment mergeSegment, String submitter) {
         super(mergeSegment, submitter);
-        this.outputSide = MRUtil.getBatchMergeOutputSide(seg);
+        this.outputSide = MRUtil.getBatchMergeOutputSide((CubeSegment)seg);
     }
 
     public CubingJob build() {
         logger.info("MR_V1 new job to MERGE segment " + seg);
-        
-        final CubingJob result = CubingJob.createMergeJob(seg, submitter, config);
+        final CubeSegment cubeSegment = (CubeSegment)seg;
+        final CubingJob result = CubingJob.createMergeJob(cubeSegment, submitter, config);
         final String jobId = result.getId();
         final String cuboidRootPath = getCuboidRootPath(jobId);
 
-        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(seg);
+        final List<CubeSegment> mergingSegments = cubeSegment.getCubeInstance().getMergingSegments(cubeSegment);
         Preconditions.checkState(mergingSegments.size() > 1, "there should be more than 2 segments to merge");
         final List<String> mergingSegmentIds = Lists.newArrayList();
         final List<String> mergingCuboidPaths = Lists.newArrayList();
@@ -63,7 +63,7 @@ public class BatchMergeJobBuilder extends JobBuilderSupport {
 
         // Phase 2: Merge Cube Files
         String formattedPath = StringUtil.join(mergingCuboidPaths, ",");
-        result.addTask(createMergeCuboidDataStep(seg, formattedPath, cuboidRootPath));
+        result.addTask(createMergeCuboidDataStep(cubeSegment, formattedPath, cuboidRootPath));
         outputSide.addStepPhase2_BuildCube(result, cuboidRootPath);
 
         // Phase 3: Update Metadata & Cleanup
@@ -78,7 +78,7 @@ public class BatchMergeJobBuilder extends JobBuilderSupport {
         mergeCuboidDataStep.setName(ExecutableConstants.STEP_NAME_MERGE_CUBOID);
         StringBuilder cmd = new StringBuilder();
 
-        appendMapReduceParameters(cmd, seg);
+        appendMapReduceParameters(cmd, seg.getRealization().getDataModelDesc());
         appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
         appendExecCmdParameters(cmd, "segmentname", seg.getName());
         appendExecCmdParameters(cmd, "input", inputPath);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
index 443cf95..48a717f 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
@@ -39,16 +39,17 @@ public class BatchMergeJobBuilder2 extends JobBuilderSupport {
 
     public BatchMergeJobBuilder2(CubeSegment mergeSegment, String submitter) {
         super(mergeSegment, submitter);
-        this.outputSide = MRUtil.getBatchMergeOutputSide2(seg);
+        this.outputSide = MRUtil.getBatchMergeOutputSide2((CubeSegment)seg);
     }
 
     public CubingJob build() {
         logger.info("MR_V2 new job to MERGE segment " + seg);
         
-        final CubingJob result = CubingJob.createMergeJob(seg, submitter, config);
+        final CubeSegment cubeSegment = (CubeSegment)seg;
+        final CubingJob result = CubingJob.createMergeJob(cubeSegment, submitter, config);
         final String jobId = result.getId();
 
-        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(seg);
+        final List<CubeSegment> mergingSegments = cubeSegment.getCubeInstance().getMergingSegments(cubeSegment);
         Preconditions.checkState(mergingSegments.size() > 1, "there should be more than 2 segments to merge");
         final List<String> mergingSegmentIds = Lists.newArrayList();
         final List<String> mergingHTables = Lists.newArrayList();
@@ -59,7 +60,7 @@ public class BatchMergeJobBuilder2 extends JobBuilderSupport {
 
         // Phase 1: Merge Dictionary
         result.addTask(createMergeDictionaryStep(mergingSegmentIds));
-        result.addTask(createMergeStatisticsStep(seg, mergingSegmentIds, getStatisticsPath(jobId)));
+        result.addTask(createMergeStatisticsStep(cubeSegment, mergingSegmentIds, getStatisticsPath(jobId)));
         outputSide.addStepPhase1_MergeDictionary(result);
 
         // Phase 2: Merge Cube
@@ -89,10 +90,10 @@ public class BatchMergeJobBuilder2 extends JobBuilderSupport {
         mergeCuboidDataStep.setName(ExecutableConstants.STEP_NAME_MERGE_CUBOID);
         StringBuilder cmd = new StringBuilder();
 
-        appendMapReduceParameters(cmd, seg);
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendMapReduceParameters(cmd, ((CubeSegment)seg).getCubeDesc().getModel());
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
         appendExecCmdParameters(cmd, "segmentname", seg.getName());
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Merge_Cuboid_" + seg.getCubeInstance().getName() + "_Step");
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Merge_Cuboid_" + seg.getRealization().getName() + "_Step");
         appendExecCmdParameters(cmd, "jobflowid", jobId);
 
         mergeCuboidDataStep.setMapReduceParams(cmd.toString());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
index 336a66f..a61e0dd 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
@@ -19,9 +19,9 @@
 package org.apache.kylin.engine.mr;
 
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.realization.IRealizationSegment;
 
 /**
  * Any ITableSource that wishes to serve as input of MapReduce build engine must adapt to this interface.
@@ -29,7 +29,7 @@ import org.apache.kylin.metadata.model.TableDesc;
 public interface IMRInput {
 
     /** Return a helper to participate in batch cubing job flow. */
-    public IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg);
+    public IMRBatchCubingInputSide getBatchCubingInputSide(IRealizationSegment seg);
 
     /** Return an InputFormat that reads from specified table. */
     public IMRTableInputFormat getTableInputFormat(TableDesc table);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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 577a836..e989042 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,6 +19,7 @@
 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 {
@@ -26,6 +27,10 @@ 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
      * the cuboid output to storage (Phase 3).
@@ -75,4 +80,26 @@ public interface IMROutput {
         /** Add step that does any necessary clean up. */
         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/incubator-kylin/blob/aa4944d4/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 5fba37c..8c770f9 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,15 +25,19 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
 import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.invertedindex.UpdateInvertedIndexInfoAfterBuildStep;
 import org.apache.kylin.engine.mr.steps.CreateDictionaryJob;
 import org.apache.kylin.engine.mr.steps.FactDistinctColumnsJob;
 import org.apache.kylin.engine.mr.steps.MergeDictionaryStep;
 import org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterBuildStep;
 import org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterMergeStep;
+import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
 
 import com.google.common.base.Preconditions;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.realization.IRealizationSegment;
 
 /**
  * Hold reusable steps for builders.
@@ -41,10 +45,10 @@ import com.google.common.base.Preconditions;
 public class JobBuilderSupport {
 
     final protected JobEngineConfig config;
-    final protected CubeSegment seg;
+    final protected IRealizationSegment seg;
     final protected String submitter;
 
-    public JobBuilderSupport(CubeSegment seg, String submitter) {
+    public JobBuilderSupport(IRealizationSegment seg, String submitter) {
         Preconditions.checkNotNull(seg, "segment cannot be null");
         this.config = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
         this.seg = seg;
@@ -64,14 +68,14 @@ public class JobBuilderSupport {
         result.setName(ExecutableConstants.STEP_NAME_FACT_DISTINCT_COLUMNS);
         result.setMapReduceJobClass(FactDistinctColumnsJob.class);
         StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd, seg);
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendMapReduceParameters(cmd, ((CubeSegment)seg).getCubeDesc().getModel());
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
         appendExecCmdParameters(cmd, "output", getFactDistinctColumnsPath(jobId));
         appendExecCmdParameters(cmd, "segmentname", seg.getName());
         appendExecCmdParameters(cmd, "statisticsenabled", String.valueOf(withStats));
         appendExecCmdParameters(cmd, "statisticsoutput", getStatisticsPath(jobId));
         appendExecCmdParameters(cmd, "statisticssamplingpercent", String.valueOf(config.getConfig().getCubingInMemSamplingPercent()));
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Fact_Distinct_Columns_" + seg.getCubeInstance().getName() + "_Step");
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Fact_Distinct_Columns_" + seg.getRealization().getName() + "_Step");
 
         result.setMapReduceParams(cmd.toString());
         return result;
@@ -82,7 +86,7 @@ public class JobBuilderSupport {
         HadoopShellExecutable buildDictionaryStep = new HadoopShellExecutable();
         buildDictionaryStep.setName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY);
         StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "cubename", seg.getRealization().getName());
         appendExecCmdParameters(cmd, "segmentname", seg.getName());
         appendExecCmdParameters(cmd, "input", getFactDistinctColumnsPath(jobId));
 
@@ -94,7 +98,7 @@ public class JobBuilderSupport {
     public UpdateCubeInfoAfterBuildStep createUpdateCubeInfoAfterBuildStep(String jobId) {
         final UpdateCubeInfoAfterBuildStep updateCubeInfoStep = new UpdateCubeInfoAfterBuildStep();
         updateCubeInfoStep.setName(ExecutableConstants.STEP_NAME_UPDATE_CUBE_INFO);
-        updateCubeInfoStep.setCubeName(seg.getCubeInstance().getName());
+        updateCubeInfoStep.setCubeName(seg.getRealization().getName());
         updateCubeInfoStep.setSegmentId(seg.getUuid());
         updateCubeInfoStep.setCubingJobId(jobId);
         return updateCubeInfoStep;
@@ -103,7 +107,7 @@ public class JobBuilderSupport {
     public MergeDictionaryStep createMergeDictionaryStep(List<String> mergingSegmentIds) {
         MergeDictionaryStep result = new MergeDictionaryStep();
         result.setName(ExecutableConstants.STEP_NAME_MERGE_DICTIONARY);
-        result.setCubeName(seg.getCubeInstance().getName());
+        result.setCubeName(seg.getRealization().getName());
         result.setSegmentId(seg.getUuid());
         result.setMergingSegmentIds(mergingSegmentIds);
         return result;
@@ -112,30 +116,44 @@ public class JobBuilderSupport {
     public UpdateCubeInfoAfterMergeStep createUpdateCubeInfoAfterMergeStep(List<String> mergingSegmentIds, String jobId) {
         UpdateCubeInfoAfterMergeStep result = new UpdateCubeInfoAfterMergeStep();
         result.setName(ExecutableConstants.STEP_NAME_UPDATE_CUBE_INFO);
-        result.setCubeName(seg.getCubeInstance().getName());
+        result.setCubeName(seg.getRealization().getName());
         result.setSegmentId(seg.getUuid());
         result.setMergingSegmentIds(mergingSegmentIds);
         result.setCubingJobId(jobId);
         return result;
     }
 
+
+
+    public UpdateInvertedIndexInfoAfterBuildStep createUpdateInvertedIndexInfoAfterBuildStep(String jobId) {
+        final UpdateInvertedIndexInfoAfterBuildStep updateIIInfoStep = new UpdateInvertedIndexInfoAfterBuildStep();
+        updateIIInfoStep.setName(ExecutableConstants.STEP_NAME_UPDATE_II_INFO);
+        updateIIInfoStep.setInvertedIndexName(seg.getRealization().getName());
+        updateIIInfoStep.setSegmentId(seg.getUuid());
+        updateIIInfoStep.setJobId(jobId);
+        return updateIIInfoStep;
+    }
+
     // ============================================================================
 
     public String getJobWorkingDir(String jobId) {
         return getJobWorkingDir(config, jobId);
     }
 
+    public String getRealizationRootPath(String jobId) {
+        return getJobWorkingDir(jobId) + "/" + seg.getRealization().getName();
+    }
     public String getCuboidRootPath(String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/cuboid/";
+        return getRealizationRootPath(jobId) + "/cuboid/";
     }
 
     public String getCuboidRootPath(CubeSegment seg) {
         return getCuboidRootPath(seg.getLastBuildJobID());
     }
 
-    public void appendMapReduceParameters(StringBuilder buf, CubeSegment seg) {
+    public void appendMapReduceParameters(StringBuilder buf, DataModelDesc modelDesc) {
         try {
-            String jobConf = config.getHadoopJobConfFilePath(seg.getCubeDesc().getModel().getCapacity());
+            String jobConf = config.getHadoopJobConfFilePath(modelDesc.getCapacity());
             if (jobConf != null && jobConf.length() > 0) {
                 buf.append(" -conf ").append(jobConf);
             }
@@ -145,11 +163,11 @@ public class JobBuilderSupport {
     }
 
     public String getFactDistinctColumnsPath(String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/fact_distinct_columns";
+        return getRealizationRootPath(jobId) + "/fact_distinct_columns";
     }
 
     public String getStatisticsPath(String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/statistics";
+        return getRealizationRootPath(jobId) + "/statistics";
     }
 
     // ============================================================================

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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 9a1c1f5..55fa9e2 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
@@ -9,17 +9,20 @@ import org.apache.kylin.engine.mr.IMROutput.IMRBatchMergeOutputSide;
 import org.apache.kylin.engine.mr.IMROutput2.IMRBatchCubingOutputSide2;
 import org.apache.kylin.engine.mr.IMROutput2.IMRBatchMergeInputSide2;
 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.IRealization;
+import org.apache.kylin.metadata.realization.IRealizationSegment;
 import org.apache.kylin.source.SourceFactory;
 import org.apache.kylin.storage.StorageFactory;
 
 public class MRUtil {
 
-    public static IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg) {
+    public static IMRBatchCubingInputSide getBatchCubingInputSide(IRealizationSegment seg) {
         return SourceFactory.createEngineAdapter(seg, IMRInput.class).getBatchCubingInputSide(seg);
     }
-
+    
     public static IMRTableInputFormat getTableInputFormat(String tableName) {
         return getTableInputFormat(getTableDesc(tableName));
     }
@@ -52,4 +55,8 @@ 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);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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
new file mode 100644
index 0000000..97e27d0
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/BatchIIJobBuilder.java
@@ -0,0 +1,132 @@
+/*
+ * 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.cube.CubeSegment;
+import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
+import org.apache.kylin.engine.mr.IMROutput;
+import org.apache.kylin.engine.mr.IMROutput.IMRBatchCubingOutputSide;
+import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.engine.mr.MRUtil;
+import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.steps.BaseCuboidJob;
+import org.apache.kylin.engine.mr.steps.NDCuboidJob;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.metadata.realization.IRealizationSegment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+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);
+
+        final String intermediateTableIdentity = seg.getJoinedFlatTableDesc().getTableName();
+        // Phase 2: Build Dictionary
+        result.addTask(createIIFactDistinctColumnsStep(seg, intermediateTableIdentity, getFactDistinctColumnsPath(jobId)));
+        result.addTask(createIIBuildDictionaryStep(seg, getFactDistinctColumnsPath(jobId)));
+
+        // Phase 3: Build Cube
+        result.addTask(createInvertedIndexStep((IISegment)seg, intermediateTableIdentity, iiRootPath));
+        outputSide.addStepPhase3_BuildII(result, iiRootPath);
+
+        // Phase 4: Update Metadata & Cleanup
+        result.addTask(createUpdateInvertedIndexInfoAfterBuildStep(jobId));
+        inputSide.addStepPhase4_Cleanup(result);
+        outputSide.addStepPhase4_Cleanup(result);
+
+        return result;
+    }
+
+    private MapReduceExecutable createIIFactDistinctColumnsStep(IRealizationSegment seg, String factTableName, String output) {
+        MapReduceExecutable result = new MapReduceExecutable();
+        result.setName(ExecutableConstants.STEP_NAME_FACT_DISTINCT_COLUMNS);
+        result.setMapReduceJobClass(IIDistinctColumnsJob.class);
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd, seg.getRealization().getDataModelDesc());
+        appendExecCmdParameters(cmd, "tablename", factTableName);
+        appendExecCmdParameters(cmd, "iiname", seg.getRealization().getName());
+        appendExecCmdParameters(cmd, "output", output);
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Fact_Distinct_Columns_" + seg.getRealization().getName() + "_Step");
+
+        result.setMapReduceParams(cmd.toString());
+        return result;
+    }
+
+    private HadoopShellExecutable createIIBuildDictionaryStep(IRealizationSegment seg, String factDistinctColumnsPath) {
+        // base cuboid job
+        HadoopShellExecutable buildDictionaryStep = new HadoopShellExecutable();
+        buildDictionaryStep.setName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY);
+        StringBuilder cmd = new StringBuilder();
+        appendExecCmdParameters(cmd, "iiname", seg.getRealization().getName());
+        appendExecCmdParameters(cmd, "input", factDistinctColumnsPath);
+
+        buildDictionaryStep.setJobParams(cmd.toString());
+        buildDictionaryStep.setJobClass(CreateInvertedIndexDictionaryJob.class);
+        return buildDictionaryStep;
+    }
+
+    private MapReduceExecutable createInvertedIndexStep(IISegment seg, String intermediateHiveTable, String iiOutputTempPath) {
+        // base cuboid job
+        MapReduceExecutable buildIIStep = new MapReduceExecutable();
+
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd, seg.getRealization().getDataModelDesc());
+
+        buildIIStep.setName(ExecutableConstants.STEP_NAME_BUILD_II);
+
+        appendExecCmdParameters(cmd, "iiname", seg.getRealization().getName());
+        appendExecCmdParameters(cmd, "tablename", intermediateHiveTable);
+        appendExecCmdParameters(cmd, "output", iiOutputTempPath);
+        appendExecCmdParameters(cmd, "jobname", ExecutableConstants.STEP_NAME_BUILD_II);
+
+        buildIIStep.setMapReduceParams(cmd.toString());
+        buildIIStep.setMapReduceJobClass(InvertedIndexJob.class);
+        return buildIIStep;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIBulkLoadJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIBulkLoadJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIBulkLoadJob.java
deleted file mode 100644
index a0a5ca6..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIBulkLoadJob.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.engine.mr.invertedindex;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.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.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-
-/**
- */
-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);
-            String input = getOptionValue(OPTION_INPUT_PATH);
-            String iiname = getOptionValue(OPTION_II_NAME);
-
-            FileSystem fs = FileSystem.get(getConf());
-            FsPermission permission = new FsPermission((short) 0777);
-            fs.setPermission(new Path(input, IIDesc.HBASE_FAMILY), permission);
-
-            int hbaseExitCode = ToolRunner.run(new LoadIncrementalHFiles(getConf()), new String[] { input, tableName });
-
-            IIManager mgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
-            IIInstance ii = mgr.getII(iiname);
-            IISegment seg = ii.getFirstSegment();
-            seg.setStorageLocationIdentifier(tableName);
-            seg.setStatus(SegmentStatusEnum.READY);
-            mgr.updateII(ii);
-
-            return hbaseExitCode;
-
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileJob.java
deleted file mode 100644
index 4ab3051..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileJob.java
+++ /dev/null
@@ -1,81 +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.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.engine.mr.common.AbstractHadoopJob;
-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));
-
-            setJobClasspath(job);
-
-            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;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileMapper.java
deleted file mode 100644
index fdcc138..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHFileMapper.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.engine.mr.invertedindex;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.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 {
-
-        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, //
-                value.get(), value.getOffset(), value.getLength());
-
-        context.write(key, kv);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHTableJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHTableJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHTableJob.java
deleted file mode 100644
index 3ccd701..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IICreateHTableJob.java
+++ /dev/null
@@ -1,148 +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.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;
-
-/**
- * @author George Song (ysong1)
- */
-public class IICreateHTableJob extends AbstractHadoopJob {
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        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();
-
-            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());
-
-            Configuration conf = HBaseConfiguration.create(getConf());
-            if (User.isHBaseSecurityEnabled(conf)) {
-                // add coprocessor for bulk load
-                tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
-            }
-
-            IIDeployCoprocessorCLI.deployCoprocessor(tableDesc);
-
-            // drop the table first
-            HBaseAdmin admin = new HBaseAdmin(conf);
-            if (admin.tableExists(tableName)) {
-                admin.disableTable(tableName);
-                admin.deleteTable(tableName);
-            }
-
-            // 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++) {
-                    System.out.println("split key " + i + ": " + BytesUtil.toHex(splitKeys[i]));
-                }
-            }
-            System.out.println("create hbase table " + tableName + " done.");
-            admin.close();
-
-            return 0;
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    //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/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDeployCoprocessorCLI.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDeployCoprocessorCLI.java
deleted file mode 100644
index da4b95b..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/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.engine.mr.invertedindex;
-
-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/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
index c9f5375..b691dc1 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
@@ -29,15 +29,18 @@ 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.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.engine.mr.HadoopUtil;
+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.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.IntermediateColumnDesc;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -74,7 +77,8 @@ public class IIDistinctColumnsJob extends AbstractHadoopJob {
 
             setJobClasspath(job);
 
-            setupMapper();
+            
+            setupMapper(ii.getFirstSegment());
             setupReducer(output);
 
             return waitForCompletion(job);
@@ -97,16 +101,10 @@ public class IIDistinctColumnsJob extends AbstractHadoopJob {
         return buf.toString();
     }
 
-    private void setupMapper() throws IOException {
-
-        String tableName = job.getConfiguration().get(BatchConstants.TABLE_NAME);
-        String[] dbTableNames = HadoopUtil.parseHiveTableName(tableName);
-
-        logger.info("setting hcat input format, db name {} , table name {}", dbTableNames[0], dbTableNames[1]);
-
-        HCatInputFormat.setInput(job, dbTableNames[0], dbTableNames[1]);
-
-        job.setInputFormatClass(HCatInputFormat.class);
+    private void setupMapper(IISegment segment) throws IOException {
+        
+        IMRInput.IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(segment).getFlatTableInputFormat();
+        flatTableInputFormat.configureJob(job);
 
         job.setMapperClass(IIDistinctColumnsMapper.class);
         job.setCombinerClass(IIDistinctColumnsCombiner.class);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsMapper.java
index 0f0c731..c431ecd 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsMapper.java
@@ -20,43 +20,52 @@ package org.apache.kylin.engine.mr.invertedindex;
 
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hive.hcatalog.data.HCatRecord;
-import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
-import org.apache.hive.hcatalog.data.schema.HCatSchema;
-import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
+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;
 
 /**
  * @author yangli9
  */
-public class IIDistinctColumnsMapper<KEYIN> extends KylinMapper<KEYIN, HCatRecord, ShortWritable, Text> {
+public class IIDistinctColumnsMapper<KEYIN> extends KylinMapper<KEYIN, Object, ShortWritable, Text> {
 
     private ShortWritable outputKey = new ShortWritable();
     private Text outputValue = new Text();
-    private HCatSchema schema = null;
-    private int columnSize = 0;
 
+    protected IMRInput.IMRTableInputFormat flatTableInputFormat;
+    
     @Override
     protected void setup(Context context) throws IOException {
         super.bindCurrentConfiguration(context.getConfiguration());
-        schema = HCatInputFormat.getTableSchema(context.getConfiguration());
-        columnSize = schema.getFields().size();
+        Configuration conf = context.getConfiguration();
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        String iiName = conf.get(BatchConstants.CFG_II_NAME);
+        IIInstance iiInstance = IIManager.getInstance(config).getII(iiName);
+        IISegment seg = iiInstance.getFirstSegment();
+        flatTableInputFormat = MRUtil.getBatchCubingInputSide(seg).getFlatTableInputFormat();
     }
 
     @Override
-    public void map(KEYIN key, HCatRecord record, Context context) throws IOException, InterruptedException {
+    public void map(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
 
-        HCatFieldSchema fieldSchema = null;
-        for (short i = 0; i < columnSize; i++) {
+        String[] row = flatTableInputFormat.parseMapperInput(record);
+        
+        for (short i = 0; i < row.length; i++) {
             outputKey.set(i);
-            fieldSchema = schema.get(i);
-            Object fieldValue = record.get(fieldSchema.getName(), schema);
-            if (fieldValue == null)
+            if (row[i] == null)
                 continue;
-            byte[] bytes = Bytes.toBytes(fieldValue.toString());
+            byte[] bytes = Bytes.toBytes(row[i].toString());
             outputValue.set(bytes, 0, bytes.length);
             context.write(outputKey, outputValue);
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/aa4944d4/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
index 30e653b..86fedf0 100644
--- 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
@@ -18,8 +18,15 @@
 
 package org.apache.kylin.engine.mr.invertedindex;
 
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
 /**
  */
 public class IIJob extends DefaultChainedExecutable {
@@ -47,4 +54,20 @@ public class IIJob extends DefaultChainedExecutable {
         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;
+    }
+
 }


[12/19] incubator-kylin git commit: add ii_name into config in IIDistinctColumnsMapper

Posted by sh...@apache.org.
add ii_name into config in IIDistinctColumnsMapper


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

Branch: refs/heads/KYLIN-1112-2
Commit: 662d16351fa418f6102a26c71df79c40a9e7c075
Parents: 3d652ee
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 10:08:28 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../mr/invertedindex/IIDistinctColumnsJob.java    |  1 +
 jdbc/kylin_jdbc.log.2014-12-22                    | 18 ------------------
 2 files changed, 1 insertion(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/662d1635/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
index e9534f2..fe968b1 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/invertedindex/IIDistinctColumnsJob.java
@@ -83,6 +83,7 @@ public class IIDistinctColumnsJob extends AbstractHadoopJob {
             setupReducer(output);
 
             Configuration conf = job.getConfiguration();
+            conf.set(BatchConstants.CFG_II_NAME, ii.getName());
             attachKylinPropsAndMetadata(ii, conf);
             return waitForCompletion(job);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/662d1635/jdbc/kylin_jdbc.log.2014-12-22
----------------------------------------------------------------------
diff --git a/jdbc/kylin_jdbc.log.2014-12-22 b/jdbc/kylin_jdbc.log.2014-12-22
deleted file mode 100644
index 7c5e001..0000000
--- a/jdbc/kylin_jdbc.log.2014-12-22
+++ /dev/null
@@ -1,18 +0,0 @@
-[main]:[2014-12-22 11:47:05,477][DEBUG][com.kylinolap.jdbc.KylinConnectionImpl.<init>(KylinConnectionImpl.java:68)] - Kylin base url test_url, project name test_db
-[main]:[2014-12-22 11:47:05,486][DEBUG][com.kylinolap.jdbc.Driver$1.onConnectionInit(Driver.java:111)] - Connection inited.
-[main]:[2014-12-22 11:47:05,487][DEBUG][com.kylinolap.jdbc.KylinMetaImpl.getTables(KylinMetaImpl.java:108)] - Get tables with conn KylinConnectionImpl [baseUrl=test_url, project=test_db, metaProject=com.kylinolap.jdbc.KylinMetaImpl$MetaProject@21300853]
-[main]:[2014-12-22 11:47:05,487][DEBUG][com.kylinolap.jdbc.KylinMetaImpl$MetaProject.getMetaTables(KylinMetaImpl.java:489)] - getMetaTables with catalog:null, schema:null, table:null
-[main]:[2014-12-22 11:47:05,534][DEBUG][com.kylinolap.jdbc.KylinConnectionImpl.<init>(KylinConnectionImpl.java:68)] - Kylin base url test_url, project name test_db
-[main]:[2014-12-22 11:47:05,534][DEBUG][com.kylinolap.jdbc.Driver$1.onConnectionInit(Driver.java:111)] - Connection inited.
-[main]:[2014-12-22 15:57:49,077][DEBUG][com.kylinolap.jdbc.KylinConnectionImpl.<init>(KylinConnectionImpl.java:68)] - Kylin base url test_url, project name test_db
-[main]:[2014-12-22 15:57:49,086][DEBUG][com.kylinolap.jdbc.Driver$1.onConnectionInit(Driver.java:111)] - Connection inited.
-[main]:[2014-12-22 15:57:49,087][DEBUG][com.kylinolap.jdbc.KylinMetaImpl.getTables(KylinMetaImpl.java:108)] - Get tables with conn KylinConnectionImpl [baseUrl=test_url, project=test_db, metaProject=com.kylinolap.jdbc.KylinMetaImpl$MetaProject@3add750e]
-[main]:[2014-12-22 15:57:49,087][DEBUG][com.kylinolap.jdbc.KylinMetaImpl$MetaProject.getMetaTables(KylinMetaImpl.java:489)] - getMetaTables with catalog:null, schema:null, table:null
-[main]:[2014-12-22 15:57:49,134][DEBUG][com.kylinolap.jdbc.KylinConnectionImpl.<init>(KylinConnectionImpl.java:68)] - Kylin base url test_url, project name test_db
-[main]:[2014-12-22 15:57:49,135][DEBUG][com.kylinolap.jdbc.Driver$1.onConnectionInit(Driver.java:111)] - Connection inited.
-[main]:[2014-12-22 17:13:45,081][DEBUG][com.kylinolap.jdbc.KylinConnectionImpl.<init>(KylinConnectionImpl.java:68)] - Kylin base url test_url, project name test_db
-[main]:[2014-12-22 17:13:45,090][DEBUG][com.kylinolap.jdbc.Driver$1.onConnectionInit(Driver.java:111)] - Connection inited.
-[main]:[2014-12-22 17:13:45,091][DEBUG][com.kylinolap.jdbc.KylinMetaImpl.getTables(KylinMetaImpl.java:108)] - Get tables with conn KylinConnectionImpl [baseUrl=test_url, project=test_db, metaProject=com.kylinolap.jdbc.KylinMetaImpl$MetaProject@21300853]
-[main]:[2014-12-22 17:13:45,091][DEBUG][com.kylinolap.jdbc.KylinMetaImpl$MetaProject.getMetaTables(KylinMetaImpl.java:489)] - getMetaTables with catalog:null, schema:null, table:null
-[main]:[2014-12-22 17:13:45,141][DEBUG][com.kylinolap.jdbc.KylinConnectionImpl.<init>(KylinConnectionImpl.java:68)] - Kylin base url test_url, project name test_db
-[main]:[2014-12-22 17:13:45,141][DEBUG][com.kylinolap.jdbc.Driver$1.onConnectionInit(Driver.java:111)] - Connection inited.


[16/19] incubator-kylin git commit: compile issue

Posted by sh...@apache.org.
compile issue


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

Branch: refs/heads/KYLIN-1112-2
Commit: 0e5d07c3ada8ee24fb9cf02b9c88bd6dfadb8b91
Parents: ca6f578
Author: shaofengshi <sh...@apache.org>
Authored: Mon Nov 2 16:22:20 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Nov 5 09:24:17 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/storage/hbase/cube/v1/HBaseClientKVIterator.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e5d07c3/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
index 7920a8a..24defef 100644
--- 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
@@ -69,7 +69,7 @@ public class HBaseClientKVIterator implements Iterable<IIRow>, Closeable {
         ImmutableBytesWritable value = new ImmutableBytesWritable();
         ImmutableBytesWritable dict = new ImmutableBytesWritable();
         IIRow pair = new IIRow(key, value, dict);
-        static final byte[] EMPTY_BYTES = new byte[0];
+        final byte[] EMPTY_BYTES = new byte[0];
 
         @Override
         public boolean hasNext() {


[02/19] incubator-kylin git commit: KYLIN-1112 on the way: Reorganize InvertedIndex source codes into plug-in architecture

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/engine-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/engine-streaming/pom.xml b/engine-streaming/pom.xml
index 955124c..46b63b3 100644
--- a/engine-streaming/pom.xml
+++ b/engine-streaming/pom.xml
@@ -26,6 +26,11 @@
         </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/incubator-kylin/blob/d921f3ca/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/invertedindex/SliceBuilder.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/invertedindex/SliceBuilder.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/invertedindex/SliceBuilder.java
new file mode 100644
index 0000000..fa5a0b2
--- /dev/null
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/invertedindex/SliceBuilder.java
@@ -0,0 +1,81 @@
+/*
+ * 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.streaming.invertedindex;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.engine.streaming.StreamingBatch;
+import org.apache.kylin.engine.streaming.StreamingMessage;
+import org.apache.kylin.invertedindex.index.BatchSliceMaker;
+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.util.IIDictionaryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.util.List;
+
+/**
+ */
+public final class SliceBuilder {
+
+    private static Logger logger = LoggerFactory.getLogger(SliceBuilder.class);
+
+    private final BatchSliceMaker sliceMaker;
+    private final IIDesc iiDesc;
+    private final boolean useLocalDict;
+
+    public SliceBuilder(IIDesc desc, short shard, boolean useLocalDict) {
+        this.iiDesc = desc;
+        this.sliceMaker = new BatchSliceMaker(desc, shard);
+        this.useLocalDict = useLocalDict;
+    }
+
+    public Slice buildSlice(StreamingBatch microStreamBatch) {
+        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 = useLocalDict ? IIDictionaryBuilder.buildDictionary(messages, iiDesc) : new Dictionary[iiDesc.listAllColumns().size()];
+        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/incubator-kylin/blob/d921f3ca/invertedindex/pom.xml
----------------------------------------------------------------------
diff --git a/invertedindex/pom.xml b/invertedindex/pom.xml
index 9e8f92e..cd22162 100644
--- a/invertedindex/pom.xml
+++ b/invertedindex/pom.xml
@@ -33,15 +33,20 @@
         <!--Kylin Jar -->
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-engine-streaming</artifactId>
+            <artifactId>kylin-core-common</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-source-hive</artifactId>
+            <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>
@@ -55,12 +60,8 @@
             <scope>test</scope>
             <version>${project.parent.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.hive.hcatalog</groupId>
-            <artifactId>hive-hcatalog-core</artifactId>
-            <version>${hive-hcatalog.version}</version>
-            <scope>provided</scope>
-        </dependency>
+        
+
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-mapreduce-client-jobclient</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/invertedindex/streaming/SliceBuilder.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/streaming/SliceBuilder.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/streaming/SliceBuilder.java
deleted file mode 100644
index ba337c8..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/streaming/SliceBuilder.java
+++ /dev/null
@@ -1,81 +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.streaming;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.engine.streaming.StreamingBatch;
-import org.apache.kylin.engine.streaming.StreamingMessage;
-import org.apache.kylin.invertedindex.index.BatchSliceMaker;
-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.util.IIDictionaryBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-import java.util.List;
-
-/**
- */
-public final class SliceBuilder {
-
-    private static Logger logger = LoggerFactory.getLogger(SliceBuilder.class);
-
-    private final BatchSliceMaker sliceMaker;
-    private final IIDesc iiDesc;
-    private final boolean useLocalDict;
-
-    public SliceBuilder(IIDesc desc, short shard, boolean useLocalDict) {
-        this.iiDesc = desc;
-        this.sliceMaker = new BatchSliceMaker(desc, shard);
-        this.useLocalDict = useLocalDict;
-    }
-
-    public Slice buildSlice(StreamingBatch microStreamBatch) {
-        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 = useLocalDict ? IIDictionaryBuilder.buildDictionary(messages, iiDesc) : new Dictionary[iiDesc.listAllColumns().size()];
-        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/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
deleted file mode 100644
index 87ee70e..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.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.job.hadoop.dict;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.dict.DistinctColumnValuesProvider;
-import org.apache.kylin.engine.mr.DFSFileTable;
-import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.ReadableTable;
-
-/**
- */
-public class CreateInvertedIndexDictionaryJob extends AbstractHadoopJob {
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_II_NAME);
-            options.addOption(OPTION_INPUT_PATH);
-            parseOptions(options, args);
-
-            final String iiname = getOptionValue(OPTION_II_NAME);
-            final String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
-            final KylinConfig config = KylinConfig.getInstanceFromEnv();
-
-            IIManager mgr = IIManager.getInstance(config);
-            IIInstance ii = mgr.getII(iiname);
-
-            mgr.buildInvertedIndexDictionary(ii.getFirstSegment(), new DistinctColumnValuesProvider() {
-                @Override
-                public ReadableTable getDistinctValuesFor(TblColRef col) {
-                    return new DFSFileTable(factColumnsInputPath + "/" + col.getName(), -1);
-                }
-            });
-            return 0;
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new CreateInvertedIndexDictionaryJob(), args);
-        System.exit(exitCode);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
deleted file mode 100644
index 300c89b..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.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.job.hadoop.invertedindex;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.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.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-
-/**
- */
-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);
-            String input = getOptionValue(OPTION_INPUT_PATH);
-            String iiname = getOptionValue(OPTION_II_NAME);
-
-            FileSystem fs = FileSystem.get(getConf());
-            FsPermission permission = new FsPermission((short) 0777);
-            fs.setPermission(new Path(input, IIDesc.HBASE_FAMILY), permission);
-
-            int hbaseExitCode = ToolRunner.run(new LoadIncrementalHFiles(getConf()), new String[] { input, tableName });
-
-            IIManager mgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
-            IIInstance ii = mgr.getII(iiname);
-            IISegment seg = ii.getFirstSegment();
-            seg.setStorageLocationIdentifier(tableName);
-            seg.setStatus(SegmentStatusEnum.READY);
-            mgr.updateII(ii);
-
-            return hbaseExitCode;
-
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
deleted file mode 100644
index 528f06f..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
+++ /dev/null
@@ -1,81 +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 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.engine.mr.common.AbstractHadoopJob;
-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));
-
-            setJobClasspath(job);
-
-            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;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileMapper.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileMapper.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileMapper.java
deleted file mode 100644
index 1adf8d6..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileMapper.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.job.hadoop.invertedindex;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.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 {
-
-        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, //
-                value.get(), value.getOffset(), value.getLength());
-
-        context.write(key, kv);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
deleted file mode 100644
index 0b7cb7a..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
+++ /dev/null
@@ -1,148 +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 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;
-
-/**
- * @author George Song (ysong1)
- */
-public class IICreateHTableJob extends AbstractHadoopJob {
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        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();
-
-            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());
-
-            Configuration conf = HBaseConfiguration.create(getConf());
-            if (User.isHBaseSecurityEnabled(conf)) {
-                // add coprocessor for bulk load
-                tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
-            }
-
-            IIDeployCoprocessorCLI.deployCoprocessor(tableDesc);
-
-            // drop the table first
-            HBaseAdmin admin = new HBaseAdmin(conf);
-            if (admin.tableExists(tableName)) {
-                admin.disableTable(tableName);
-                admin.deleteTable(tableName);
-            }
-
-            // 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++) {
-                    System.out.println("split key " + i + ": " + BytesUtil.toHex(splitKeys[i]));
-                }
-            }
-            System.out.println("create hbase table " + tableName + " done.");
-            admin.close();
-
-            return 0;
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    //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/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDeployCoprocessorCLI.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDeployCoprocessorCLI.java
deleted file mode 100644
index a4c1961..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/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.job.hadoop.invertedindex;
-
-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/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
deleted file mode 100644
index 1f4611b..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
+++ /dev/null
@@ -1,58 +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.HashSet;
-
-import org.apache.hadoop.io.ShortWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.engine.mr.KylinReducer;
-
-/**
- * @author yangli9
- */
-public class IIDistinctColumnsCombiner extends KylinReducer<ShortWritable, Text, ShortWritable, Text> {
-
-    private Text outputValue = new Text();
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-    }
-
-    @Override
-    public void reduce(ShortWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
-
-        HashSet<ByteArray> set = new HashSet<ByteArray>();
-        for (Text textValue : values) {
-            ByteArray value = new ByteArray(Bytes.copy(textValue.getBytes(), 0, textValue.getLength()));
-            set.add(value);
-        }
-
-        for (ByteArray value : set) {
-            outputValue.set(value.array(), value.offset(), value.length());
-            context.write(key, outputValue);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
deleted file mode 100644
index 042678e..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
+++ /dev/null
@@ -1,136 +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 org.apache.commons.cli.Options;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.ShortWritable;
-import org.apache.hadoop.io.Text;
-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.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.engine.mr.HadoopUtil;
-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.model.IIJoinedFlatTableDesc;
-import org.apache.kylin.metadata.model.IntermediateColumnDesc;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author yangli9
- */
-public class IIDistinctColumnsJob extends AbstractHadoopJob {
-    protected static final Logger logger = LoggerFactory.getLogger(IIDistinctColumnsJob.class);
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_TABLE_NAME);
-            options.addOption(OPTION_II_NAME);
-            options.addOption(OPTION_OUTPUT_PATH);
-            parseOptions(options, args);
-
-            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
-            String tableName = getOptionValue(OPTION_TABLE_NAME).toUpperCase();
-            String iiName = getOptionValue(OPTION_II_NAME);
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-
-            // ----------------------------------------------------------------------------
-
-            logger.info("Starting: " + job.getJobName() + " on table " + tableName);
-
-            IIManager iiMgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
-            IIInstance ii = iiMgr.getII(iiName);
-            job.getConfiguration().set(BatchConstants.TABLE_NAME, tableName);
-            job.getConfiguration().set(BatchConstants.TABLE_COLUMNS, getColumns(ii));
-
-            setJobClasspath(job);
-
-            setupMapper();
-            setupReducer(output);
-
-            return waitForCompletion(job);
-
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-
-    }
-
-    private String getColumns(IIInstance ii) {
-        IIJoinedFlatTableDesc iiflat = new IIJoinedFlatTableDesc(ii.getDescriptor());
-        StringBuilder buf = new StringBuilder();
-        for (IntermediateColumnDesc col : iiflat.getColumnList()) {
-            if (buf.length() > 0)
-                buf.append(",");
-            buf.append(col.getColumnName());
-        }
-        return buf.toString();
-    }
-
-    private void setupMapper() throws IOException {
-
-        String tableName = job.getConfiguration().get(BatchConstants.TABLE_NAME);
-        String[] dbTableNames = HadoopUtil.parseHiveTableName(tableName);
-
-        logger.info("setting hcat input format, db name {} , table name {}", dbTableNames[0], dbTableNames[1]);
-
-        HCatInputFormat.setInput(job, dbTableNames[0], dbTableNames[1]);
-
-        job.setInputFormatClass(HCatInputFormat.class);
-
-        job.setMapperClass(IIDistinctColumnsMapper.class);
-        job.setCombinerClass(IIDistinctColumnsCombiner.class);
-        job.setMapOutputKeyClass(ShortWritable.class);
-        job.setMapOutputValueClass(Text.class);
-    }
-
-    private void setupReducer(Path output) throws IOException {
-        job.setReducerClass(IIDistinctColumnsReducer.class);
-        job.setOutputFormatClass(SequenceFileOutputFormat.class);
-        job.setOutputKeyClass(NullWritable.class);
-        job.setOutputValueClass(Text.class);
-
-        FileOutputFormat.setOutputPath(job, output);
-        job.getConfiguration().set(BatchConstants.OUTPUT_PATH, output.toString());
-
-        job.setNumReduceTasks(1);
-
-        deletePath(job.getConfiguration(), output);
-    }
-
-    public static void main(String[] args) throws Exception {
-        IIDistinctColumnsJob job = new IIDistinctColumnsJob();
-        int exitCode = ToolRunner.run(job, args);
-        System.exit(exitCode);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsMapper.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsMapper.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsMapper.java
deleted file mode 100644
index 3418a57..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsMapper.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.job.hadoop.invertedindex;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.ShortWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hive.hcatalog.data.HCatRecord;
-import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
-import org.apache.hive.hcatalog.data.schema.HCatSchema;
-import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.engine.mr.KylinMapper;
-
-/**
- * @author yangli9
- */
-public class IIDistinctColumnsMapper<KEYIN> extends KylinMapper<KEYIN, HCatRecord, ShortWritable, Text> {
-
-    private ShortWritable outputKey = new ShortWritable();
-    private Text outputValue = new Text();
-    private HCatSchema schema = null;
-    private int columnSize = 0;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-        schema = HCatInputFormat.getTableSchema(context.getConfiguration());
-        columnSize = schema.getFields().size();
-    }
-
-    @Override
-    public void map(KEYIN key, HCatRecord record, Context context) throws IOException, InterruptedException {
-
-        HCatFieldSchema fieldSchema = null;
-        for (short i = 0; i < columnSize; i++) {
-            outputKey.set(i);
-            fieldSchema = schema.get(i);
-            Object fieldValue = record.get(fieldSchema.getName(), schema);
-            if (fieldValue == null)
-                continue;
-            byte[] bytes = Bytes.toBytes(fieldValue.toString());
-            outputValue.set(bytes, 0, bytes.length);
-            context.write(outputKey, outputValue);
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
deleted file mode 100644
index fcb4dd5..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.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.job.hadoop.invertedindex;
-
-import java.io.IOException;
-import java.util.HashSet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.ShortWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.engine.mr.KylinReducer;
-import org.apache.kylin.engine.mr.common.BatchConstants;
-
-/**
- * @author yangli9
- */
-public class IIDistinctColumnsReducer extends KylinReducer<ShortWritable, Text, NullWritable, Text> {
-
-    private String[] columns;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        Configuration conf = context.getConfiguration();
-        this.columns = conf.get(BatchConstants.TABLE_COLUMNS).split(",");
-    }
-
-    @Override
-    public void reduce(ShortWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
-        String columnName = columns[key.get()];
-
-        HashSet<ByteArray> set = new HashSet<ByteArray>();
-        for (Text textValue : values) {
-            ByteArray value = new ByteArray(Bytes.copy(textValue.getBytes(), 0, textValue.getLength()));
-            set.add(value);
-        }
-
-        Configuration conf = context.getConfiguration();
-        FileSystem fs = FileSystem.get(conf);
-        String outputPath = conf.get(BatchConstants.OUTPUT_PATH);
-        FSDataOutputStream out = fs.create(new Path(outputPath, columnName));
-
-        try {
-            for (ByteArray value : set) {
-                out.write(value.array(), value.offset(), value.length());
-                out.write('\n');
-            }
-        } finally {
-            out.close();
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
deleted file mode 100644
index c9ad448..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
+++ /dev/null
@@ -1,164 +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.ArrayList;
-
-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.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.engine.mr.HadoopUtil;
-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.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author yangli9
- */
-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_TABLE_NAME);
-            options.addOption(OPTION_OUTPUT_PATH);
-            parseOptions(options, args);
-
-            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
-            String iiname = getOptionValue(OPTION_II_NAME);
-            String intermediateTable = getOptionValue(OPTION_TABLE_NAME);
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-
-            // ----------------------------------------------------------------------------
-
-            System.out.println("Starting: " + job.getJobName());
-
-            IIInstance ii = getII(iiname);
-            short sharding = ii.getDescriptor().getSharding();
-
-            setJobClasspath(job);
-
-            setupMapper(intermediateTable);
-            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());
-    }
-
-    protected void attachKylinPropsAndMetadata(IIInstance ii, Configuration conf) throws IOException {
-        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
-
-        // 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());
-        }
-        for (IISegment segment : ii.getSegments()) {
-            dumpList.addAll(segment.getDictionaryPaths());
-        }
-
-        attachKylinPropsAndMetadata(dumpList, conf);
-    }
-
-    private void setupMapper(String intermediateTable) throws IOException {
-
-        String[] dbTableNames = HadoopUtil.parseHiveTableName(intermediateTable);
-        HCatInputFormat.setInput(job, dbTableNames[0], dbTableNames[1]);
-
-        job.setInputFormatClass(HCatInputFormat.class);
-
-        job.setMapperClass(InvertedIndexMapper.class);
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setMapOutputValueClass(ImmutableBytesWritable.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.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/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
deleted file mode 100644
index bc43b65..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.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.job.hadoop.invertedindex;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hive.hcatalog.data.HCatRecord;
-import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
-import org.apache.hive.hcatalog.data.schema.HCatSchema;
-import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.engine.mr.KylinMapper;
-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 InvertedIndexMapper<KEYIN> extends KylinMapper<KEYIN, HCatRecord, LongWritable, ImmutableBytesWritable> {
-
-    private TableRecordInfo info;
-    private TableRecord rec;
-
-    private LongWritable outputKey;
-    private ImmutableBytesWritable outputValue;
-    private HCatSchema schema = null;
-    private List<HCatFieldSchema> fields;
-
-    @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);
-        this.rec = this.info.createTableRecord();
-
-        outputKey = new LongWritable();
-        outputValue = new ImmutableBytesWritable(rec.getBytes());
-
-        schema = HCatInputFormat.getTableSchema(context.getConfiguration());
-
-        fields = schema.getFields();
-    }
-
-    @Override
-    public void map(KEYIN key, HCatRecord record, Context context) throws IOException, InterruptedException {
-
-        rec.reset();
-        for (int i = 0; i < fields.size(); i++) {
-            Object fieldValue = record.get(i);
-            rec.setValueString(i, fieldValue == null ? null : fieldValue.toString());
-        }
-
-        outputKey.set(rec.getTimestamp());
-        // outputValue's backing bytes array is the same as rec
-
-        context.write(outputKey, outputValue);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexPartitioner.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexPartitioner.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexPartitioner.java
deleted file mode 100644
index 396c221..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/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.job.hadoop.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/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java b/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
deleted file mode 100644
index 5a69eec..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.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.job.hadoop.invertedindex;
-
-import java.io.IOException;
-
-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.engine.mr.KylinReducer;
-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.IncrementalSliceMaker;
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.invertedindex.index.TableRecord;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.apache.kylin.invertedindex.model.IIRow;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-
-/**
- * @author yangli9
- */
-public class InvertedIndexReducer extends KylinReducer<LongWritable, ImmutableBytesWritable, ImmutableBytesWritable, ImmutableBytesWritable> {
-
-    private TableRecordInfo info;
-    private TableRecord rec;
-    private IncrementalSliceMaker builder;
-    private IIKeyValueCodec kv;
-
-    @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);
-        info = new TableRecordInfo(seg);
-        rec = info.createTableRecord();
-        builder = null;
-        kv = new IIKeyValueCodec(info.getDigest());
-    }
-
-    @Override
-    public void reduce(LongWritable key, Iterable<ImmutableBytesWritable> values, Context context) //
-            throws IOException, InterruptedException {
-        for (ImmutableBytesWritable v : values) {
-            rec.setBytes(v.get(), v.getOffset(), v.getLength());
-
-            if (builder == null) {
-                builder = new IncrementalSliceMaker(info, rec.getShard());
-            }
-
-            //TODO: to delete this log
-            System.out.println(rec.getShard() + " - " + rec);
-
-            Slice slice = builder.append(rec);
-            if (slice != null) {
-                output(slice, context);
-            }
-        }
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-        Slice slice = builder.close();
-        if (slice != null) {
-            output(slice, context);
-        }
-    }
-
-    private void output(Slice slice, Context context) throws IOException, InterruptedException {
-        for (IIRow pair : kv.encodeKeyValue(slice)) {
-            context.write(pair.getKey(), pair.getValue());
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/invertedindex/IIJob.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/invertedindex/IIJob.java b/invertedindex/src/main/java/org/apache/kylin/job/invertedindex/IIJob.java
deleted file mode 100644
index 0af846b..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/invertedindex/IIJob.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.job.invertedindex;
-
-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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d921f3ca/invertedindex/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java b/invertedindex/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
deleted file mode 100644
index 4bd06c5..0000000
--- a/invertedindex/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
+++ /dev/null
@@ -1,230 +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.invertedindex;
-
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.TimeZone;
-
-import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
-import org.apache.kylin.engine.mr.common.MapReduceExecutable;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.hadoop.dict.CreateInvertedIndexDictionaryJob;
-import org.apache.kylin.job.hadoop.invertedindex.IIBulkLoadJob;
-import org.apache.kylin.job.hadoop.invertedindex.IICreateHFileJob;
-import org.apache.kylin.job.hadoop.invertedindex.IICreateHTableJob;
-import org.apache.kylin.job.hadoop.invertedindex.IIDistinctColumnsJob;
-import org.apache.kylin.job.hadoop.invertedindex.InvertedIndexJob;
-import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
-import org.apache.kylin.source.hive.HiveMRInput.BatchCubingInputSide;
-
-import com.google.common.base.Preconditions;
-
-/**
- */
-public final class IIJobBuilder {
-
-    final JobEngineConfig engineConfig;
-
-    public IIJobBuilder(JobEngineConfig engineConfig) {
-        this.engineConfig = engineConfig;
-    }
-
-    public IIJob buildJob(IISegment seg, String submitter) {
-        checkPreconditions(seg);
-
-        IIJob result = initialJob(seg, "BUILD", submitter);
-        final String jobId = result.getId();
-        final IIJoinedFlatTableDesc intermediateTableDesc = new IIJoinedFlatTableDesc(seg.getIIDesc());
-        final String intermediateTableIdentity = getIntermediateTableIdentity(intermediateTableDesc);
-        final String factDistinctColumnsPath = getIIDistinctColumnsPath(seg, jobId);
-        final String iiRootPath = getJobWorkingDir(jobId) + "/" + seg.getIIInstance().getName() + "/";
-        final String iiPath = iiRootPath + "*";
-
-        final AbstractExecutable intermediateHiveTableStep = createFlatHiveTableStep(intermediateTableDesc, jobId);
-        result.addTask(intermediateHiveTableStep);
-
-        result.addTask(createFactDistinctColumnsStep(seg, intermediateTableIdentity, jobId, factDistinctColumnsPath));
-
-        result.addTask(createBuildDictionaryStep(seg, factDistinctColumnsPath));
-
-        result.addTask(createInvertedIndexStep(seg, intermediateTableIdentity, iiRootPath));
-
-        // create htable step
-        result.addTask(createCreateHTableStep(seg));
-
-        // generate hfiles step
-        result.addTask(createConvertToHfileStep(seg, iiPath, jobId));
-
-        // bulk load step
-        result.addTask(createBulkLoadStep(seg, jobId));
-
-        return result;
-    }
-
-    private AbstractExecutable createFlatHiveTableStep(IIJoinedFlatTableDesc intermediateTableDesc, String jobId) {
-        return BatchCubingInputSide.createFlatHiveTableStep(engineConfig, intermediateTableDesc, jobId);
-    }
-
-    private IIJob initialJob(IISegment seg, String type, String submitter) {
-        IIJob result = new IIJob();
-        SimpleDateFormat format = new SimpleDateFormat("z yyyy-MM-dd HH:mm:ss");
-        format.setTimeZone(TimeZone.getTimeZone(engineConfig.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;
-    }
-
-    private void checkPreconditions(IISegment seg) {
-        Preconditions.checkNotNull(seg, "segment cannot be null");
-        Preconditions.checkNotNull(engineConfig, "jobEngineConfig cannot be null");
-    }
-
-    private void appendMapReduceParameters(StringBuilder builder, JobEngineConfig engineConfig) {
-        try {
-            String jobConf = engineConfig.getHadoopJobConfFilePath(RealizationCapacity.MEDIUM);
-            if (jobConf != null && jobConf.length() > 0) {
-                builder.append(" -conf ").append(jobConf);
-            }
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private String getIIDistinctColumnsPath(IISegment seg, String jobUuid) {
-        return getJobWorkingDir(jobUuid) + "/" + seg.getIIInstance().getName() + "/ii_distinct_columns";
-    }
-
-    private String getHFilePath(IISegment seg, String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getIIInstance().getName() + "/hfile/";
-    }
-
-    private MapReduceExecutable createFactDistinctColumnsStep(IISegment seg, String factTableName, String jobId, String output) {
-        MapReduceExecutable result = new MapReduceExecutable();
-        result.setName(ExecutableConstants.STEP_NAME_FACT_DISTINCT_COLUMNS);
-        result.setMapReduceJobClass(IIDistinctColumnsJob.class);
-        StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd, engineConfig);
-        appendExecCmdParameters(cmd, "tablename", factTableName);
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "output", output);
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Fact_Distinct_Columns_" + seg.getIIInstance().getName() + "_Step");
-
-        result.setMapReduceParams(cmd.toString());
-        return result;
-    }
-
-    private HadoopShellExecutable createBuildDictionaryStep(IISegment seg, String factDistinctColumnsPath) {
-        // base cuboid job
-        HadoopShellExecutable buildDictionaryStep = new HadoopShellExecutable();
-        buildDictionaryStep.setName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY);
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "input", factDistinctColumnsPath);
-
-        buildDictionaryStep.setJobParams(cmd.toString());
-        buildDictionaryStep.setJobClass(CreateInvertedIndexDictionaryJob.class);
-        return buildDictionaryStep;
-    }
-
-    private MapReduceExecutable createInvertedIndexStep(IISegment seg, String intermediateHiveTable, String iiOutputTempPath) {
-        // base cuboid job
-        MapReduceExecutable buildIIStep = new MapReduceExecutable();
-
-        StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd, engineConfig);
-
-        buildIIStep.setName(ExecutableConstants.STEP_NAME_BUILD_II);
-
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "tablename", intermediateHiveTable);
-        appendExecCmdParameters(cmd, "output", iiOutputTempPath);
-        appendExecCmdParameters(cmd, "jobname", ExecutableConstants.STEP_NAME_BUILD_II);
-
-        buildIIStep.setMapReduceParams(cmd.toString());
-        buildIIStep.setMapReduceJobClass(InvertedIndexJob.class);
-        return buildIIStep;
-    }
-
-    private HadoopShellExecutable createCreateHTableStep(IISegment seg) {
-        HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
-        createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
-
-        createHtableStep.setJobParams(cmd.toString());
-        createHtableStep.setJobClass(IICreateHTableJob.class);
-
-        return createHtableStep;
-    }
-
-    private MapReduceExecutable createConvertToHfileStep(IISegment seg, String inputPath, String jobId) {
-        MapReduceExecutable createHFilesStep = new MapReduceExecutable();
-        createHFilesStep.setName(ExecutableConstants.STEP_NAME_CONVERT_II_TO_HFILE);
-        StringBuilder cmd = new StringBuilder();
-
-        appendMapReduceParameters(cmd, engineConfig);
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-        appendExecCmdParameters(cmd, "input", inputPath);
-        appendExecCmdParameters(cmd, "output", getHFilePath(seg, jobId));
-        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
-        appendExecCmdParameters(cmd, "jobname", "Kylin_HFile_Generator_" + seg.getIIInstance().getName() + "_Step");
-
-        createHFilesStep.setMapReduceParams(cmd.toString());
-        createHFilesStep.setMapReduceJobClass(IICreateHFileJob.class);
-
-        return createHFilesStep;
-    }
-
-    private HadoopShellExecutable createBulkLoadStep(IISegment seg, String jobId) {
-        HadoopShellExecutable bulkLoadStep = new HadoopShellExecutable();
-        bulkLoadStep.setName(ExecutableConstants.STEP_NAME_BULK_LOAD_HFILE);
-
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "input", getHFilePath(seg, jobId));
-        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
-        appendExecCmdParameters(cmd, "iiname", seg.getIIInstance().getName());
-
-        bulkLoadStep.setJobParams(cmd.toString());
-        bulkLoadStep.setJobClass(IIBulkLoadJob.class);
-
-        return bulkLoadStep;
-
-    }
-
-    private StringBuilder appendExecCmdParameters(StringBuilder buf, String paraName, String paraValue) {
-        return buf.append(" -").append(paraName).append(" ").append(paraValue);
-    }
-
-    private String getJobWorkingDir(String uuid) {
-        return engineConfig.getHdfsWorkingDirectory() + "kylin-" + uuid;
-    }
-
-    private String getIntermediateTableIdentity(IIJoinedFlatTableDesc intermediateTableDesc) {
-        return engineConfig.getConfig().getHiveDatabaseForIntermediateTable() + "." + intermediateTableDesc.getTableName();
-    }
-}