You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2015/07/24 01:20:09 UTC

[01/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Repository: incubator-kylin
Updated Branches:
  refs/heads/0.8 ccdc4155e -> 993e0645e


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/memstore/GTSimpleMemStore.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/memstore/GTSimpleMemStore.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/memstore/GTSimpleMemStore.java
deleted file mode 100644
index b3d77de..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/memstore/GTSimpleMemStore.java
+++ /dev/null
@@ -1,112 +0,0 @@
-package org.apache.kylin.storage.gridtable.memstore;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTRowBlock;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.IGTStore;
-
-public class GTSimpleMemStore implements IGTStore {
-
-    final GTInfo info;
-    final List<GTRowBlock> rowBlockList;
-
-    public GTSimpleMemStore(GTInfo info) {
-        this.info = info;
-        this.rowBlockList = new ArrayList<GTRowBlock>();
-
-        if (info.isShardingEnabled())
-            throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public GTInfo getInfo() {
-        return info;
-    }
-
-    public long memoryUsage() {
-        if (rowBlockList.size() == 0) {
-            return 0;
-        } else {
-            return rowBlockList.get(0).exportLength() * Long.valueOf(rowBlockList.size());
-        }
-    }
-
-    @Override
-    public IGTStoreWriter rebuild(int shard) {
-        rowBlockList.clear();
-        return new Writer(rowBlockList);
-    }
-
-    @Override
-    public IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) {
-        if (rowBlockList.size() > 0) {
-            GTRowBlock last = rowBlockList.get(rowBlockList.size() - 1);
-            fillLast.copyFrom(last);
-        }
-        return new Writer(rowBlockList);
-    }
-
-    private static class Writer implements IGTStoreWriter {
-
-        private final List<GTRowBlock> rowBlockList;
-
-        Writer(List<GTRowBlock> rowBlockList) {
-            this.rowBlockList = rowBlockList;
-        }
-        @Override
-        public void close() throws IOException {
-        }
-
-        @Override
-        public void write(GTRowBlock block) throws IOException {
-            GTRowBlock copy = block.copy();
-            int id = block.getSequenceId();
-            if (id < rowBlockList.size()) {
-                rowBlockList.set(id, copy);
-            } else {
-                assert id == rowBlockList.size();
-                rowBlockList.add(copy);
-            }
-        }
-    }
-
-    @Override
-    public IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) {
-
-        return new IGTStoreScanner() {
-            Iterator<GTRowBlock> it = rowBlockList.iterator();
-
-            @Override
-            public boolean hasNext() {
-                return it.hasNext();
-            }
-
-            @Override
-            public GTRowBlock next() {
-                return it.next();
-            }
-
-            @Override
-            public void remove() {
-                throw new UnsupportedOperationException();
-            }
-
-            @Override
-            public void close() throws IOException {
-            }
-        };
-    }
-
-    public void drop() throws IOException {
-        //will there be any concurrent issue? If yes, ArrayList should be replaced
-        rowBlockList.clear();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
index 63623ca..1a804cd 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
@@ -36,6 +36,7 @@ import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITupleIterator;
 import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.hbase.coprocessor.observer.ObserverEnabler;
+import org.apache.kylin.storage.hbase.steps.RowValueDecoder;
 import org.apache.kylin.storage.tuple.Tuple;
 import org.apache.kylin.storage.tuple.TupleInfo;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java
index bd5f21a..4c3920b 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageQuery.java
@@ -50,6 +50,7 @@ import org.apache.kylin.metadata.tuple.ITupleIterator;
 import org.apache.kylin.storage.ICachableStorageQuery;
 import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.hbase.coprocessor.observer.ObserverEnabler;
+import org.apache.kylin.storage.hbase.steps.RowValueDecoder;
 import org.apache.kylin.storage.tuple.TupleInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/hbase/CubeTupleConverter.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeTupleConverter.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeTupleConverter.java
index 8d0af58..08c3784 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeTupleConverter.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeTupleConverter.java
@@ -17,6 +17,7 @@ import org.apache.kylin.dict.lookup.LookupStringTable;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.storage.hbase.steps.RowValueDecoder;
 import org.apache.kylin.storage.tuple.Tuple;
 import org.apache.kylin.storage.tuple.TupleInfo;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
index 5e6d9a0..96e4df6 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
@@ -30,6 +30,7 @@ import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
 import org.apache.kylin.metadata.tuple.ITupleIterator;
 import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.hbase.steps.RowValueDecoder;
 import org.apache.kylin.storage.tuple.TupleInfo;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java
index 8e0e8f7..a81dcfc 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java
@@ -24,8 +24,8 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.List;
 
-import org.apache.kylin.storage.hbase.RowValueDecoder;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorConstants;
+import org.apache.kylin.storage.hbase.steps.RowValueDecoder;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesSerializer;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
index 3bbf43a..2aeca8f 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
@@ -35,11 +35,11 @@ import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.hbase.RegionScannerAdapter;
 import org.apache.kylin.storage.hbase.ResultScannerAdapter;
-import org.apache.kylin.storage.hbase.RowValueDecoder;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorFilter;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorProjector;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorRowType;
 import org.apache.kylin.storage.hbase.coprocessor.FilterDecorator;
+import org.apache.kylin.storage.hbase.steps.RowValueDecoder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java b/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
deleted file mode 100644
index c6b1a18..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage.tuple;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.List;
-
-import net.sf.ehcache.pool.sizeof.annotations.IgnoreSizeOf;
-
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.metadata.measure.DoubleMutable;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.tuple.ITuple;
-
-/**
- * @author xjiang
- */
-public class Tuple implements ITuple {
-
-    @IgnoreSizeOf
-    private final TupleInfo info;
-    private final Object[] values;
-
-    public Tuple(TupleInfo info) {
-        this.info = info;
-        this.values = new Object[info.size()];
-    }
-
-    public List<String> getAllFields() {
-        return info.getAllFields();
-    }
-
-    public List<TblColRef> getAllColumns() {
-        return info.getAllColumns();
-    }
-
-    public Object[] getAllValues() {
-        return values;
-    }
-
-    @Override
-    public ITuple makeCopy() {
-        Tuple ret = new Tuple(this.info);
-        for (int i = 0; i < this.values.length; ++i) {
-            ret.values[i] = this.values[i];
-        }
-        return ret;
-    }
-
-    public TupleInfo getInfo() {
-        return info;
-    }
-
-    public String getFieldName(TblColRef col) {
-        return info.getFieldName(col);
-    }
-
-    public TblColRef getFieldColumn(String fieldName) {
-        return info.getColumn(fieldName);
-    }
-
-    public Object getValue(String fieldName) {
-        int index = info.getFieldIndex(fieldName);
-        return values[index];
-    }
-
-    public Object getValue(TblColRef col) {
-        int index = info.getColumnIndex(col);
-        return values[index];
-    }
-
-    public String getDataTypeName(int idx) {
-        return info.getDataTypeName(idx);
-    }
-
-    public void setDimensionValue(String fieldName, String fieldValue) {
-        setDimensionValue(info.getFieldIndex(fieldName), fieldValue);
-    }
-
-    public void setDimensionValue(int idx, String fieldValue) {
-        Object objectValue = convertOptiqCellValue(fieldValue, getDataTypeName(idx));
-        values[idx] = objectValue;
-    }
-
-    public void setMeasureValue(String fieldName, Object fieldValue) {
-        setMeasureValue(info.getFieldIndex(fieldName), fieldValue);
-    }
-
-    public void setMeasureValue(int idx, Object fieldValue) {
-        fieldValue = convertWritableToJava(fieldValue);
-
-        String dataType = getDataTypeName(idx);
-        // special handling for BigDecimal, allow double be aggregated as
-        // BigDecimal during cube build for best precision
-        if ("double".equals(dataType) && fieldValue instanceof BigDecimal) {
-            fieldValue = ((BigDecimal) fieldValue).doubleValue();
-        } else if ("integer".equals(dataType) && !(fieldValue instanceof Integer)) {
-            fieldValue = ((Number) fieldValue).intValue();
-        } else if ("float".equals(dataType) && fieldValue instanceof BigDecimal) {
-            fieldValue = ((BigDecimal) fieldValue).floatValue();
-        }
-        values[idx] = fieldValue;
-    }
-
-    private Object convertWritableToJava(Object o) {
-        if (o instanceof LongMutable)
-            o = ((LongMutable) o).get();
-        else if (o instanceof DoubleMutable)
-            o = ((DoubleMutable) o).get();
-        return o;
-    }
-
-    public boolean hasColumn(TblColRef column) {
-        return info.hasColumn(column);
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        for (String field : info.getAllFields()) {
-            sb.append(field);
-            sb.append("=");
-            sb.append(getValue(field));
-            sb.append(",");
-        }
-        return sb.toString();
-    }
-
-    public static long epicDaysToMillis(int days) {
-        return 1L * days * (1000 * 3600 * 24);
-    }
-
-    public static int dateToEpicDays(String strValue) {
-        Date dateValue = DateFormat.stringToDate(strValue); // NOTE: forces GMT timezone
-        long millis = dateValue.getTime();
-        return (int) (millis / (1000 * 3600 * 24));
-    }
-
-    public static long getTs(ITuple row, TblColRef partitionCol) {
-        //ts column type differentiate
-        if (partitionCol.getDatatype().equals("date")) {
-            return Tuple.epicDaysToMillis(Integer.valueOf(row.getValue(partitionCol).toString()));
-        } else {
-            return Long.valueOf(row.getValue(partitionCol).toString());
-        }
-    }
-
-    public static Object convertOptiqCellValue(String strValue, String dataTypeName) {
-        if (strValue == null)
-            return null;
-
-        if ((strValue.equals("") || strValue.equals("\\N")) && !dataTypeName.equals("string"))
-            return null;
-
-        // TODO use data type enum instead of string comparison
-        if ("date".equals(dataTypeName)) {
-            // convert epoch time
-            return dateToEpicDays(strValue);// Optiq expects Integer instead of Long. by honma
-        } else if ("timestamp".equals(dataTypeName) || "datetime".equals(dataTypeName)) {
-            return Long.valueOf(DateFormat.stringToMillis(strValue));
-        } else if ("tinyint".equals(dataTypeName)) {
-            return Byte.valueOf(strValue);
-        } else if ("short".equals(dataTypeName) || "smallint".equals(dataTypeName)) {
-            return Short.valueOf(strValue);
-        } else if ("integer".equals(dataTypeName)) {
-            return Integer.valueOf(strValue);
-        } else if ("long".equals(dataTypeName) || "bigint".equals(dataTypeName)) {
-            return Long.valueOf(strValue);
-        } else if ("double".equals(dataTypeName)) {
-            return Double.valueOf(strValue);
-        } else if ("decimal".equals(dataTypeName)) {
-            return new BigDecimal(strValue);
-        } else if ("float".equals(dataTypeName)){
-            return Float.valueOf(strValue);
-        } else {
-            return strValue;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/tuple/TupleInfo.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/tuple/TupleInfo.java b/storage/src/main/java/org/apache/kylin/storage/tuple/TupleInfo.java
deleted file mode 100644
index 735cc64..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/tuple/TupleInfo.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage.tuple;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.kylin.metadata.model.TblColRef;
-
-/**
- * 
- * @author xjiang
- * 
- */
-public class TupleInfo {
-
-    private final Map<String, Integer> fieldMap;
-    private final Map<TblColRef, Integer> columnMap;
-    
-    private final List<String> fields;
-    private final List<TblColRef> columns;
-    private final List<String> dataTypeNames;
-
-    public TupleInfo() {
-        fieldMap = new HashMap<String, Integer>();
-        columnMap = new HashMap<TblColRef, Integer>();
-        fields = new ArrayList<String>();
-        columns = new ArrayList<TblColRef>();
-        dataTypeNames = new ArrayList<String>();
-    }
-
-    public TblColRef getColumn(String fieldName) {
-        int idx = getFieldIndex(fieldName);
-        return columns.get(idx);
-    }
-
-    public int getColumnIndex(TblColRef col) {
-        return columnMap.get(col);
-    }
-
-    public String getDataTypeName(int index) {
-        return dataTypeNames.get(index);
-    }
-
-    public int getFieldIndex(String fieldName) {
-        return fieldMap.get(fieldName);
-    }
-    
-    public boolean hasField(String fieldName) {
-        return fieldMap.containsKey(fieldName);
-    }
-
-    public String getFieldName(TblColRef col) {
-        int idx = columnMap.get(col);
-        return fields.get(idx);
-    }
-
-    public boolean hasColumn(TblColRef col) {
-        return columnMap.containsKey(col);
-    }
-
-    public void setField(String fieldName, TblColRef col, int index) {
-        fieldMap.put(fieldName, index);
-
-        if (col != null)
-            columnMap.put(col, index);
-
-        if (fields.size() > index)
-            fields.set(index, fieldName);
-        else
-            fields.add(index, fieldName);
-
-        if (columns.size() > index)
-            columns.set(index, col);
-        else
-            columns.add(index, col);
-        
-        if (dataTypeNames.size() > index)
-            dataTypeNames.set(index, col.getType().getName());
-        else
-            dataTypeNames.add(index, col.getType().getName());
-    }
-
-    public List<String> getAllFields() {
-        return fields;
-    }
-
-    public List<TblColRef> getAllColumns() {
-        return columns;
-    }
-
-    public int size() {
-        return fields.size();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/test/java/org/apache/kylin/storage/gridtable/AggregationCacheMemSizeTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/gridtable/AggregationCacheMemSizeTest.java b/storage/src/test/java/org/apache/kylin/storage/gridtable/AggregationCacheMemSizeTest.java
deleted file mode 100644
index 649fd5a..0000000
--- a/storage/src/test/java/org/apache/kylin/storage/gridtable/AggregationCacheMemSizeTest.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements. See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License. You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.kylin.storage.gridtable;
-
-import java.math.BigDecimal;
-import java.util.Comparator;
-import java.util.Random;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.metadata.measure.BigDecimalSumAggregator;
-import org.apache.kylin.metadata.measure.DoubleSumAggregator;
-import org.apache.kylin.metadata.measure.DoubleMutable;
-import org.apache.kylin.metadata.measure.HLLCAggregator;
-import org.apache.kylin.metadata.measure.LongSumAggregator;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
-import org.junit.Test;
-
-public class AggregationCacheMemSizeTest {
-
-    public static final int NUM_OF_OBJS = 1000000 / 2;
-
-    interface CreateAnObject {
-        Object create();
-    }
-
-    @Test
-    public void testHLLCAggregatorSize() throws InterruptedException {
-        int est = estimateObjectSize(new CreateAnObject() {
-            @Override
-            public Object create() {
-                HLLCAggregator aggr = new HLLCAggregator(10);
-                aggr.aggregate(new HyperLogLogPlusCounter(10));
-                return aggr;
-            }
-        });
-        System.out.println("HLLC: " + est);
-    }
-
-    @Test
-    public void testBigDecimalAggregatorSize() throws InterruptedException {
-        int est = estimateObjectSize(new CreateAnObject() {
-            @Override
-            public Object create() {
-                return newBigDecimalAggr();
-            }
-
-        });
-        System.out.println("BigDecimal: " + est);
-    }
-
-    private BigDecimalSumAggregator newBigDecimalAggr() {
-        BigDecimalSumAggregator aggr = new BigDecimalSumAggregator();
-        aggr.aggregate(new BigDecimal("12345678901234567890.123456789"));
-        return aggr;
-    }
-
-    @Test
-    public void testLongAggregatorSize() throws InterruptedException {
-        int est = estimateObjectSize(new CreateAnObject() {
-            @Override
-            public Object create() {
-                return newLongAggr();
-            }
-        });
-        System.out.println("Long: " + est);
-    }
-
-    private LongSumAggregator newLongAggr() {
-        LongSumAggregator aggr = new LongSumAggregator();
-        aggr.aggregate(new LongMutable(10));
-        return aggr;
-    }
-
-    @Test
-    public void testDoubleAggregatorSize() throws InterruptedException {
-        int est = estimateObjectSize(new CreateAnObject() {
-            @Override
-            public Object create() {
-                return newDoubleAggr();
-            }
-        });
-        System.out.println("Double: " + est);
-    }
-
-    private DoubleSumAggregator newDoubleAggr() {
-        DoubleSumAggregator aggr = new DoubleSumAggregator();
-        aggr.aggregate(new DoubleMutable(10));
-        return aggr;
-    }
-
-    @Test
-    public void testByteArraySize() throws InterruptedException {
-        int est = estimateObjectSize(new CreateAnObject() {
-            @Override
-            public Object create() {
-                return new byte[10];
-            }
-        });
-        System.out.println("byte[10]: " + est);
-    }
-
-    @Test
-    public void testAggregatorArraySize() throws InterruptedException {
-        int est = estimateObjectSize(new CreateAnObject() {
-            @Override
-            public Object create() {
-                return new MeasureAggregator[7];
-            }
-        });
-        System.out.println("MeasureAggregator[7]: " + est);
-    }
-
-    @Test
-    public void testTreeMapSize() throws InterruptedException {
-        final SortedMap<byte[], Object> map = new TreeMap<byte[], Object>(new Comparator<byte[]>() {
-            @Override
-            public int compare(byte[] o1, byte[] o2) {
-                return Bytes.compareTo(o1, o2);
-            }
-        });
-        final Random rand = new Random();
-        int est = estimateObjectSize(new CreateAnObject() {
-            @Override
-            public Object create() {
-                byte[] key = new byte[10];
-                rand.nextBytes(key);
-                map.put(key, null);
-                return null;
-            }
-        });
-        System.out.println("TreeMap entry: " + (est - 20)); // -20 is to exclude byte[10]
-    }
-
-    @Test
-    public void testAggregationCacheSize() throws InterruptedException {
-        final SortedMap<byte[], Object> map = new TreeMap<byte[], Object>(new Comparator<byte[]>() {
-            @Override
-            public int compare(byte[] o1, byte[] o2) {
-                return Bytes.compareTo(o1, o2);
-            }
-        });
-        final Random rand = new Random();
-
-        long bytesBefore = memLeft();
-        byte[] key = null;
-        MeasureAggregator<?>[] aggrs = null;
-        for (int i = 0; i < NUM_OF_OBJS; i++) {
-            key = new byte[10];
-            rand.nextBytes(key);
-            aggrs = new MeasureAggregator[4];
-            aggrs[0] = newBigDecimalAggr();
-            aggrs[1] = newLongAggr();
-            aggrs[2] = newDoubleAggr();
-            aggrs[3] = newDoubleAggr();
-            map.put(key, aggrs);
-        }
-
-        long bytesAfter = memLeft();
-        
-        long mapActualSize = bytesBefore - bytesAfter;
-        long mapExpectSize = GTAggregateScanner.estimateSizeOfAggrCache(key, aggrs, map.size());
-        System.out.println("Actual cache size: " + mapActualSize);
-        System.out.println("Expect cache size: " + mapExpectSize);
-    }
-
-    private int estimateObjectSize(CreateAnObject factory) throws InterruptedException {
-        Object[] hold = new Object[NUM_OF_OBJS];
-        long bytesBefore = memLeft();
-
-        for (int i = 0; i < hold.length; i++) {
-            hold[i] = factory.create();
-        }
-
-        long bytesAfter = memLeft();
-        return (int) ((bytesBefore - bytesAfter) / hold.length);
-    }
-
-    private long memLeft() throws InterruptedException {
-        Runtime.getRuntime().gc();
-        Thread.sleep(500);
-        return getSystemAvailBytes();
-    }
-
-    private long getSystemAvailBytes() {
-        Runtime runtime = Runtime.getRuntime();
-        long totalMemory = runtime.totalMemory(); // current heap allocated to the VM process
-        long freeMemory = runtime.freeMemory(); // out of the current heap, how much is free
-        long maxMemory = runtime.maxMemory(); // Max heap VM can use e.g. Xmx setting
-        long usedMemory = totalMemory - freeMemory; // how much of the current heap the VM is using
-        long availableMemory = maxMemory - usedMemory; // available memory i.e. Maximum heap size minus the current amount used
-        return availableMemory;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/test/java/org/apache/kylin/storage/gridtable/DictGridTableTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/gridtable/DictGridTableTest.java b/storage/src/test/java/org/apache/kylin/storage/gridtable/DictGridTableTest.java
deleted file mode 100644
index 6092956..0000000
--- a/storage/src/test/java/org/apache/kylin/storage/gridtable/DictGridTableTest.java
+++ /dev/null
@@ -1,381 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements. See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License. You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.kylin.storage.gridtable;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.dict.NumberDictionaryBuilder;
-import org.apache.kylin.dict.StringBytesConverter;
-import org.apache.kylin.dict.TrieDictionaryBuilder;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.ExtractTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.cube.CubeCodeSystem;
-import org.apache.kylin.storage.gridtable.GTInfo.Builder;
-import org.apache.kylin.storage.gridtable.memstore.GTSimpleMemStore;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-public class DictGridTableTest {
-
-    @Test
-    public void test() throws IOException {
-        GridTable table = newTestTable();
-        verifyScanRangePlanner(table);
-        verifyFirstRow(table);
-        verifyScanWithUnevaluatableFilter(table);
-        verifyScanWithEvaluatableFilter(table);
-        verifyConvertFilterConstants1(table);
-        verifyConvertFilterConstants2(table);
-        verifyConvertFilterConstants3(table);
-        verifyConvertFilterConstants4(table);
-    }
-
-    private void verifyScanRangePlanner(GridTable table) {
-        GTInfo info = table.getInfo();
-        GTScanRangePlanner planner = new GTScanRangePlanner(info);
-        
-        CompareTupleFilter timeComp1 = compare(info.colRef(0), FilterOperatorEnum.GT, enc(info, 0, "2015-01-14"));
-        CompareTupleFilter timeComp2 = compare(info.colRef(0), FilterOperatorEnum.LT, enc(info, 0, "2015-01-13"));
-        CompareTupleFilter timeComp3 = compare(info.colRef(0), FilterOperatorEnum.LT, enc(info, 0, "2015-01-15"));
-        CompareTupleFilter timeComp4 = compare(info.colRef(0), FilterOperatorEnum.EQ, enc(info, 0, "2015-01-15"));
-        CompareTupleFilter ageComp1 = compare(info.colRef(1), FilterOperatorEnum.EQ, enc(info, 1, "10"));
-        CompareTupleFilter ageComp2 = compare(info.colRef(1), FilterOperatorEnum.EQ, enc(info, 1, "20"));
-        CompareTupleFilter ageComp3 = compare(info.colRef(1), FilterOperatorEnum.EQ, enc(info, 1, "30"));
-        CompareTupleFilter ageComp4 = compare(info.colRef(1), FilterOperatorEnum.NEQ, enc(info, 1, "30"));
-        
-        // flatten or-and & hbase fuzzy value
-        {
-            LogicalTupleFilter filter = and(timeComp1, or(ageComp1, ageComp2));
-            List<GTScanRange> r = planner.planScanRanges(filter);
-            assertEquals(1, r.size());
-            assertEquals("[1421193600000, 10]-[null, null]", r.get(0).toString());
-            assertEquals("[[10], [20]]", r.get(0).hbaseFuzzyKeys.toString());
-        }
-        
-        // pre-evaluate ever false
-        {
-            LogicalTupleFilter filter = and(timeComp1, timeComp2);
-            List<GTScanRange> r = planner.planScanRanges(filter);
-            assertEquals(0, r.size());
-        }
-        
-        // pre-evaluate ever true
-        {
-            LogicalTupleFilter filter = or(timeComp1, ageComp4);
-            List<GTScanRange> r = planner.planScanRanges(filter);
-            assertEquals("[[null, null]-[null, null]]", r.toString());
-        }
-        
-        // merge overlap range
-        {
-            LogicalTupleFilter filter = or(timeComp1, timeComp3);
-            List<GTScanRange> r = planner.planScanRanges(filter);
-            assertEquals("[[null, null]-[null, null]]", r.toString());
-        }
-        
-        // merge too many ranges
-        {
-            LogicalTupleFilter filter = or(and(timeComp4, ageComp1), and(timeComp4, ageComp2), and(timeComp4, ageComp3));
-            List<GTScanRange> r = planner.planScanRanges(filter);
-            assertEquals(3, r.size());
-            assertEquals("[1421280000000, 10]-[1421280000000, 10]", r.get(0).toString());
-            assertEquals("[1421280000000, 20]-[1421280000000, 20]", r.get(1).toString());
-            assertEquals("[1421280000000, 30]-[1421280000000, 30]", r.get(2).toString());
-            List<GTScanRange> r2 = planner.planScanRanges(filter, 2);
-            assertEquals("[[1421280000000, 10]-[1421280000000, 30]]", r2.toString());
-        }
-    }
-
-    private void verifyFirstRow(GridTable table) throws IOException {
-        doScanAndVerify(table, new GTScanRequest(table.getInfo()), "[1421193600000, 30, Yang, 10, 10.5]");
-    }
-
-    private void verifyScanWithUnevaluatableFilter(GridTable table) throws IOException {
-        GTInfo info = table.getInfo();
-
-        CompareTupleFilter fComp = compare(info.colRef(0), FilterOperatorEnum.GT, enc(info, 0, "2015-01-14"));
-        ExtractTupleFilter fUnevaluatable = unevaluatable(info.colRef(1));
-        LogicalTupleFilter fNotPlusUnevaluatable = not(unevaluatable(info.colRef(1)));
-        LogicalTupleFilter filter = and(fComp, fUnevaluatable, fNotPlusUnevaluatable);
-
-        GTScanRequest req = new GTScanRequest(info, null, setOf(0), setOf(3), new String[] { "sum" }, filter);
-
-        // note the unEvaluatable column 1 in filter is added to group by
-        assertEquals("GTScanRequest [range=[null, null]-[null, null], columns={0, 1, 3}, filterPushDown=AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], [null], [null]], aggrGroupBy={0, 1}, aggrMetrics={3}, aggrMetricsFuncs=[sum]]", req.toString());
-        
-        doScanAndVerify(table, req, "[1421280000000, 20, null, 20, null]");
-    }
-    
-    private void verifyScanWithEvaluatableFilter(GridTable table) throws IOException {
-        GTInfo info = table.getInfo();
-
-        CompareTupleFilter fComp1 = compare(info.colRef(0), FilterOperatorEnum.GT, enc(info, 0, "2015-01-14"));
-        CompareTupleFilter fComp2 = compare(info.colRef(1), FilterOperatorEnum.GT, enc(info, 1, "10"));
-        LogicalTupleFilter filter = and(fComp1, fComp2);
-
-        GTScanRequest req = new GTScanRequest(info, null, setOf(0), setOf(3), new String[] { "sum" }, filter);
-        
-        // note the evaluatable column 1 in filter is added to returned columns but not in group by
-        assertEquals("GTScanRequest [range=[null, null]-[null, null], columns={0, 1, 3}, filterPushDown=AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], NULL.GT_MOCKUP_TABLE.1 GT [\\x00]], aggrGroupBy={0}, aggrMetrics={3}, aggrMetricsFuncs=[sum]]", req.toString());
-        
-        doScanAndVerify(table, req, "[1421280000000, 20, null, 30, null]", "[1421366400000, 20, null, 40, null]");
-    }
-
-    private void verifyConvertFilterConstants1(GridTable table) {
-        GTInfo info = table.getInfo();
-        
-        TableDesc extTable = TableDesc.mockup("ext");
-        TblColRef extColA = new TblColRef(ColumnDesc.mockup(extTable, 1, "A", "timestamp"));
-        TblColRef extColB = new TblColRef(ColumnDesc.mockup(extTable, 2, "B", "integer"));
-
-        CompareTupleFilter fComp1 = compare(extColA, FilterOperatorEnum.GT, "2015-01-14");
-        CompareTupleFilter fComp2 = compare(extColB, FilterOperatorEnum.EQ, "10");
-        LogicalTupleFilter filter = and(fComp1, fComp2);
-        
-        List<TblColRef> colMapping = Lists.newArrayList();
-        colMapping.add(extColA);
-        colMapping.add(extColB);
-        
-        TupleFilter newFilter = GTUtil.convertFilterColumnsAndConstants(filter, info, colMapping, null);
-        assertEquals("AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], NULL.GT_MOCKUP_TABLE.1 EQ [\\x00]]", newFilter.toString());
-    }
-
-    private void verifyConvertFilterConstants2(GridTable table) {
-        GTInfo info = table.getInfo();
-        
-        TableDesc extTable = TableDesc.mockup("ext");
-        TblColRef extColA = new TblColRef(ColumnDesc.mockup(extTable, 1, "A", "timestamp"));
-        TblColRef extColB = new TblColRef(ColumnDesc.mockup(extTable, 2, "B", "integer"));
-        
-        CompareTupleFilter fComp1 = compare(extColA, FilterOperatorEnum.GT, "2015-01-14");
-        CompareTupleFilter fComp2 = compare(extColB, FilterOperatorEnum.LT, "9");
-        LogicalTupleFilter filter = and(fComp1, fComp2);
-        
-        List<TblColRef> colMapping = Lists.newArrayList();
-        colMapping.add(extColA);
-        colMapping.add(extColB);
-        
-        // $1<"9" round up to $1<"10"
-        TupleFilter newFilter = GTUtil.convertFilterColumnsAndConstants(filter, info, colMapping, null);
-        assertEquals("AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], NULL.GT_MOCKUP_TABLE.1 LT [\\x00]]", newFilter.toString());
-    }
-    
-    private void verifyConvertFilterConstants3(GridTable table) {
-        GTInfo info = table.getInfo();
-        
-        TableDesc extTable = TableDesc.mockup("ext");
-        TblColRef extColA = new TblColRef(ColumnDesc.mockup(extTable, 1, "A", "timestamp"));
-        TblColRef extColB = new TblColRef(ColumnDesc.mockup(extTable, 2, "B", "integer"));
-        
-        CompareTupleFilter fComp1 = compare(extColA, FilterOperatorEnum.GT, "2015-01-14");
-        CompareTupleFilter fComp2 = compare(extColB, FilterOperatorEnum.LTE, "9");
-        LogicalTupleFilter filter = and(fComp1, fComp2);
-        
-        List<TblColRef> colMapping = Lists.newArrayList();
-        colMapping.add(extColA);
-        colMapping.add(extColB);
-        
-        // $1<="9" round down to FALSE
-        TupleFilter newFilter = GTUtil.convertFilterColumnsAndConstants(filter, info, colMapping, null);
-        assertEquals("AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], []]", newFilter.toString());
-    }
-    
-    private void verifyConvertFilterConstants4(GridTable table) {
-        GTInfo info = table.getInfo();
-        
-        TableDesc extTable = TableDesc.mockup("ext");
-        TblColRef extColA = new TblColRef(ColumnDesc.mockup(extTable, 1, "A", "timestamp"));
-        TblColRef extColB = new TblColRef(ColumnDesc.mockup(extTable, 2, "B", "integer"));
-        
-        CompareTupleFilter fComp1 = compare(extColA, FilterOperatorEnum.GT, "2015-01-14");
-        CompareTupleFilter fComp2 = compare(extColB, FilterOperatorEnum.IN, "9", "10", "15");
-        LogicalTupleFilter filter = and(fComp1, fComp2);
-        
-        List<TblColRef> colMapping = Lists.newArrayList();
-        colMapping.add(extColA);
-        colMapping.add(extColB);
-        
-        // $1 in ("9", "10", "15") has only "10" left
-        TupleFilter newFilter = GTUtil.convertFilterColumnsAndConstants(filter, info, colMapping, null);
-        assertEquals("AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], NULL.GT_MOCKUP_TABLE.1 IN [\\x00]]", newFilter.toString());
-    }
-    
-    private void doScanAndVerify(GridTable table, GTScanRequest req, String... verifyRows) throws IOException {
-        System.out.println(req);
-        IGTScanner scanner = table.scan(req);
-        int i = 0;
-        for (GTRecord r : scanner) {
-            System.out.println(r);
-            if (verifyRows != null && i < verifyRows.length) {
-                assertEquals(verifyRows[i], r.toString());
-            }
-            i++;
-        }
-        scanner.close();
-    }
-
-    private Object enc(GTInfo info, int col, String value) {
-        ByteBuffer buf = ByteBuffer.allocate(info.getMaxColumnLength());
-        info.codeSystem.encodeColumnValue(col, value, buf);
-        return ByteArray.copyOf(buf.array(), buf.arrayOffset(), buf.position());
-    }
-
-    private ExtractTupleFilter unevaluatable(TblColRef col) {
-        ExtractTupleFilter r = new ExtractTupleFilter(FilterOperatorEnum.EXTRACT);
-        r.addChild(new ColumnTupleFilter(col));
-        return r;
-    }
-
-    private CompareTupleFilter compare(TblColRef col, FilterOperatorEnum op, Object... value) {
-        CompareTupleFilter result = new CompareTupleFilter(op);
-        result.addChild(new ColumnTupleFilter(col));
-        result.addChild(new ConstantTupleFilter(Arrays.asList(value)));
-        return result;
-    }
-
-    private LogicalTupleFilter and(TupleFilter... children) {
-        return logic(FilterOperatorEnum.AND, children);
-    }
-
-    private LogicalTupleFilter or(TupleFilter... children) {
-        return logic(FilterOperatorEnum.OR, children);
-    }
-
-    private LogicalTupleFilter not(TupleFilter child) {
-        return logic(FilterOperatorEnum.NOT, child);
-    }
-
-    private LogicalTupleFilter logic(FilterOperatorEnum op, TupleFilter... children) {
-        LogicalTupleFilter result = new LogicalTupleFilter(op);
-        for (TupleFilter c : children) {
-            result.addChild(c);
-        }
-        return result;
-    }
-
-    static GridTable newTestTable() throws IOException {
-        GTInfo info = newInfo();
-        GTSimpleMemStore store = new GTSimpleMemStore(info);
-        GridTable table = new GridTable(info, store);
-
-        GTRecord r = new GTRecord(table.getInfo());
-        GTBuilder builder = table.rebuild();
-
-        builder.write(r.setValues("2015-01-14", "30", "Yang", new LongMutable(10), new BigDecimal("10.5")));
-        builder.write(r.setValues("2015-01-14", "30", "Luke", new LongMutable(10), new BigDecimal("10.5")));
-        builder.write(r.setValues("2015-01-15", "20", "Dong", new LongMutable(10), new BigDecimal("10.5")));
-        builder.write(r.setValues("2015-01-15", "20", "Jason", new LongMutable(10), new BigDecimal("10.5")));
-        builder.write(r.setValues("2015-01-15", "30", "Xu", new LongMutable(10), new BigDecimal("10.5")));
-        builder.write(r.setValues("2015-01-16", "20", "Mahone", new LongMutable(10), new BigDecimal("10.5")));
-        builder.write(r.setValues("2015-01-16", "20", "Qianhao", new LongMutable(10), new BigDecimal("10.5")));
-        builder.write(r.setValues("2015-01-16", "30", "George", new LongMutable(10), new BigDecimal("10.5")));
-        builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new LongMutable(10), new BigDecimal("10.5")));
-        builder.write(r.setValues("2015-01-17", "10", "Kejia", new LongMutable(10), new BigDecimal("10.5")));
-        builder.close();
-
-        return table;
-    }
-
-    static GTInfo newInfo() {
-        Builder builder = GTInfo.builder();
-        builder.setCodeSystem(newDictCodeSystem());
-        builder.setColumns( //
-                DataType.getInstance("timestamp"), //
-                DataType.getInstance("integer"), //
-                DataType.getInstance("varchar(10)"), //
-                DataType.getInstance("bigint"), //
-                DataType.getInstance("decimal") //
-        );
-        builder.setPrimaryKey(setOf(0, 1));
-        builder.setColumnPreferIndex(setOf(0));
-        builder.enableColumnBlock(new ImmutableBitSet[] { setOf(0, 1), setOf(2), setOf(3, 4) });
-        builder.enableRowBlock(4);
-        GTInfo info = builder.build();
-        return info;
-    }
-
-    @SuppressWarnings("rawtypes")
-    private static CubeCodeSystem newDictCodeSystem() {
-        Map<Integer, Dictionary> dictionaryMap = Maps.newHashMap();
-        dictionaryMap.put(1, newDictionaryOfInteger());
-        dictionaryMap.put(2, newDictionaryOfString());
-        return new CubeCodeSystem(dictionaryMap);
-    }
-
-    @SuppressWarnings("rawtypes")
-    private static Dictionary newDictionaryOfString() {
-        TrieDictionaryBuilder<String> builder = new TrieDictionaryBuilder<>(new StringBytesConverter());
-        builder.addValue("Dong");
-        builder.addValue("George");
-        builder.addValue("Jason");
-        builder.addValue("Kejia");
-        builder.addValue("Luke");
-        builder.addValue("Mahone");
-        builder.addValue("Qianhao");
-        builder.addValue("Shaofeng");
-        builder.addValue("Xu");
-        builder.addValue("Yang");
-        return builder.build(0);
-    }
-
-    @SuppressWarnings("rawtypes")
-    private static Dictionary newDictionaryOfInteger() {
-        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<>(new StringBytesConverter());
-        builder.addValue("10");
-        builder.addValue("20");
-        builder.addValue("30");
-        builder.addValue("40");
-        builder.addValue("50");
-        builder.addValue("60");
-        builder.addValue("70");
-        builder.addValue("80");
-        builder.addValue("90");
-        builder.addValue("100");
-        return builder.build(0);
-    }
-
-    private static ImmutableBitSet setOf(int... values) {
-        BitSet set = new BitSet();
-        for (int i : values)
-            set.set(i);
-        return new ImmutableBitSet(set);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/test/java/org/apache/kylin/storage/gridtable/SimpleGridTableTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/gridtable/SimpleGridTableTest.java b/storage/src/test/java/org/apache/kylin/storage/gridtable/SimpleGridTableTest.java
deleted file mode 100644
index be0d13e..0000000
--- a/storage/src/test/java/org/apache/kylin/storage/gridtable/SimpleGridTableTest.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements. See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License. You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.kylin.storage.gridtable;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.BitSet;
-import java.util.List;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.storage.gridtable.memstore.GTSimpleMemStore;
-import org.junit.Test;
-
-public class SimpleGridTableTest {
-
-    @Test
-    public void testBasics() throws IOException {
-        GTInfo info = UnitTestSupport.basicInfo();
-        GTSimpleMemStore store = new GTSimpleMemStore(info);
-        GridTable table = new GridTable(info, store);
-
-        GTBuilder builder = rebuild(table);
-        IGTScanner scanner = scan(table);
-        assertEquals(builder.getWrittenRowBlockCount(), scanner.getScannedRowBlockCount());
-        assertEquals(builder.getWrittenRowCount(), scanner.getScannedRowCount());
-    }
-
-    @Test
-    public void testAdvanced() throws IOException {
-        GTInfo info = UnitTestSupport.advancedInfo();
-        GTSimpleMemStore store = new GTSimpleMemStore(info);
-        GridTable table = new GridTable(info, store);
-
-        GTBuilder builder = rebuild(table);
-        IGTScanner scanner = scan(table);
-        assertEquals(builder.getWrittenRowBlockCount(), scanner.getScannedRowBlockCount());
-        assertEquals(builder.getWrittenRowCount(), scanner.getScannedRowCount());
-    }
-
-    @Test
-    public void testAggregate() throws IOException {
-        GTInfo info = UnitTestSupport.advancedInfo();
-        GTSimpleMemStore store = new GTSimpleMemStore(info);
-        GridTable table = new GridTable(info, store);
-
-        GTBuilder builder = rebuild(table);
-        IGTScanner scanner = scanAndAggregate(table);
-        assertEquals(builder.getWrittenRowBlockCount(), scanner.getScannedRowBlockCount());
-        assertEquals(builder.getWrittenRowCount(), scanner.getScannedRowCount());
-    }
-
-    @Test
-    public void testAppend() throws IOException {
-        GTInfo info = UnitTestSupport.advancedInfo();
-        GTSimpleMemStore store = new GTSimpleMemStore(info);
-        GridTable table = new GridTable(info, store);
-
-        rebuildViaAppend(table);
-        IGTScanner scanner = scan(table);
-        assertEquals(3, scanner.getScannedRowBlockCount());
-        assertEquals(10, scanner.getScannedRowCount());
-    }
-
-    private IGTScanner scan(GridTable table) throws IOException {
-        GTScanRequest req = new GTScanRequest(table.getInfo());
-        IGTScanner scanner = table.scan(req);
-        for (GTRecord r : scanner) {
-            Object[] v = r.getValues();
-            assertTrue(((String) v[0]).startsWith("2015-"));
-            assertTrue(((String) v[2]).equals("Food"));
-            assertTrue(((LongMutable) v[3]).get() == 10);
-            assertTrue(((BigDecimal) v[4]).doubleValue() == 10.5);
-            System.out.println(r);
-        }
-        scanner.close();
-        System.out.println("Scanned Row Block Count: " + scanner.getScannedRowBlockCount());
-        System.out.println("Scanned Row Count: " + scanner.getScannedRowCount());
-        return scanner;
-    }
-
-    private IGTScanner scanAndAggregate(GridTable table) throws IOException {
-        GTScanRequest req = new GTScanRequest(table.getInfo(), null, setOf(0, 2), setOf(3, 4), new String[] { "count", "sum" }, null);
-        IGTScanner scanner = table.scan(req);
-        int i = 0;
-        for (GTRecord r : scanner) {
-            Object[] v = r.getValues();
-            switch (i) {
-            case 0:
-                assertTrue(((LongMutable) v[3]).get() == 20);
-                assertTrue(((BigDecimal) v[4]).doubleValue() == 21.0);
-                break;
-            case 1:
-                assertTrue(((LongMutable) v[3]).get() == 30);
-                assertTrue(((BigDecimal) v[4]).doubleValue() == 31.5);
-                break;
-            case 2:
-                assertTrue(((LongMutable) v[3]).get() == 40);
-                assertTrue(((BigDecimal) v[4]).doubleValue() == 42.0);
-                break;
-            case 3:
-                assertTrue(((LongMutable) v[3]).get() == 10);
-                assertTrue(((BigDecimal) v[4]).doubleValue() == 10.5);
-                break;
-            default:
-                fail();
-            }
-            i++;
-            System.out.println(r);
-        }
-        scanner.close();
-        System.out.println("Scanned Row Block Count: " + scanner.getScannedRowBlockCount());
-        System.out.println("Scanned Row Count: " + scanner.getScannedRowCount());
-        return scanner;
-    }
-
-    static GTBuilder rebuild(GridTable table) throws IOException {
-        GTBuilder builder = table.rebuild();
-        for (GTRecord rec : UnitTestSupport.mockupData(table.getInfo(), 10)) {
-            builder.write(rec);
-        }
-        builder.close();
-
-        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
-        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
-        return builder;
-    }
-    
-    static void rebuildViaAppend(GridTable table) throws IOException {
-        List<GTRecord> data = UnitTestSupport.mockupData(table.getInfo(), 10);
-        GTBuilder builder;
-        int i = 0;
-
-        builder = table.append();
-        builder.write(data.get(i++));
-        builder.write(data.get(i++));
-        builder.write(data.get(i++));
-        builder.write(data.get(i++));
-        builder.close();
-        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
-        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
-
-        builder = table.append();
-        builder.write(data.get(i++));
-        builder.write(data.get(i++));
-        builder.write(data.get(i++));
-        builder.close();
-        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
-        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
-
-        builder = table.append();
-        builder.write(data.get(i++));
-        builder.write(data.get(i++));
-        builder.close();
-        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
-        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
-
-        builder = table.append();
-        builder.write(data.get(i++));
-        builder.close();
-        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
-        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
-    }
-
-    private static ImmutableBitSet setOf(int... values) {
-        BitSet set = new BitSet();
-        for (int i : values)
-            set.set(i);
-        return new ImmutableBitSet(set);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/test/java/org/apache/kylin/storage/gridtable/SimpleInvertedIndexTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/gridtable/SimpleInvertedIndexTest.java b/storage/src/test/java/org/apache/kylin/storage/gridtable/SimpleInvertedIndexTest.java
deleted file mode 100644
index a9ab61c..0000000
--- a/storage/src/test/java/org/apache/kylin/storage/gridtable/SimpleInvertedIndexTest.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements. See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License. You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.kylin.storage.gridtable;
-
-import static org.junit.Assert.*;
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.serializer.StringSerializer;
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-public class SimpleInvertedIndexTest {
-
-    GTInfo info;
-    GTInvertedIndex index;
-    ArrayList<CompareTupleFilter> basicFilters = Lists.newArrayList();
-    ArrayList<ConciseSet> basicResults = Lists.newArrayList();
-
-    public SimpleInvertedIndexTest() {
-        
-        info = UnitTestSupport.advancedInfo();
-        TblColRef colA = info.colRef(0);
-        
-        // block i contains value "i", the last is NULL
-        index = new GTInvertedIndex(info);
-        GTRowBlock mockBlock = GTRowBlock.allocate(info);
-        GTRowBlock.Writer writer = mockBlock.getWriter();
-        GTRecord record = new GTRecord(info);
-        for (int i = 0; i < 10; i++) {
-            record.setValues(i < 9 ? "" + i : null, "", "", new LongMutable(0), new BigDecimal(0));
-            for (int j = 0; j < info.getRowBlockSize(); j++) {
-                writer.append(record);
-            }
-            writer.readyForFlush();
-            index.add(mockBlock);
-            
-            writer.clearForNext();
-        }
-        
-        basicFilters.add(compare(colA, FilterOperatorEnum.ISNULL));
-        basicResults.add(set(9));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.ISNOTNULL));
-        basicResults.add(set(0, 1, 2, 3, 4, 5, 6, 7, 8, 9));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.EQ, 0));
-        basicResults.add(set(0));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.NEQ, 0));
-        basicResults.add(set(0, 1, 2, 3, 4, 5, 6, 7, 8, 9));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.IN, 0, 5));
-        basicResults.add(set(0, 5));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.NOTIN, 0, 5));
-        basicResults.add(set(0, 1, 2, 3, 4, 5, 6, 7, 8, 9));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.LT, 3));
-        basicResults.add(set(0, 1, 2));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.LTE, 3));
-        basicResults.add(set(0, 1, 2, 3));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.GT, 3));
-        basicResults.add(set(4, 5, 6, 7, 8));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.GTE, 3));
-        basicResults.add(set(3, 4, 5, 6, 7, 8));
-    }
-
-    @Test
-    public void testBasics() {
-        for (int i = 0; i < basicFilters.size(); i++) {
-            assertEquals(basicResults.get(i), index.filter(basicFilters.get(i)));
-        }
-    }
-
-    @Test
-    public void testLogicalAnd() {
-        for (int i = 0; i < basicFilters.size(); i++) {
-            for (int j = 0; j < basicFilters.size(); j++) {
-                LogicalTupleFilter f = logical(FilterOperatorEnum.AND, basicFilters.get(i), basicFilters.get(j));
-                ConciseSet r = basicResults.get(i).clone();
-                r.retainAll(basicResults.get(j));
-                assertEquals(r, index.filter(f));
-            }
-        }
-    }
-
-    @Test
-    public void testLogicalOr() {
-        for (int i = 0; i < basicFilters.size(); i++) {
-            for (int j = 0; j < basicFilters.size(); j++) {
-                LogicalTupleFilter f = logical(FilterOperatorEnum.OR, basicFilters.get(i), basicFilters.get(j));
-                ConciseSet r = basicResults.get(i).clone();
-                r.addAll(basicResults.get(j));
-                assertEquals(r, index.filter(f));
-            }
-        }
-    }
-
-    @Test
-    public void testNotEvaluable() {
-        ConciseSet all = set(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
-        
-        CompareTupleFilter notEvaluable = compare(info.colRef(1), FilterOperatorEnum.EQ, 0);
-        assertEquals(all, index.filter(notEvaluable));
-
-        LogicalTupleFilter or = logical(FilterOperatorEnum.OR, basicFilters.get(0), notEvaluable);
-        assertEquals(all, index.filter(or));
-
-        LogicalTupleFilter and = logical(FilterOperatorEnum.AND, basicFilters.get(0), notEvaluable);
-        assertEquals(basicResults.get(0), index.filter(and));
-    }
-
-    public static CompareTupleFilter compare(TblColRef col, TupleFilter.FilterOperatorEnum op, int... ids) {
-        CompareTupleFilter filter = new CompareTupleFilter(op);
-        filter.addChild(columnFilter(col));
-        for (int i : ids) {
-            filter.addChild(constFilter(i));
-        }
-        return filter;
-    }
-
-    public static LogicalTupleFilter logical(TupleFilter.FilterOperatorEnum op, TupleFilter... filters) {
-        LogicalTupleFilter filter = new LogicalTupleFilter(op);
-        for (TupleFilter f : filters)
-            filter.addChild(f);
-        return filter;
-    }
-
-    public static ColumnTupleFilter columnFilter(TblColRef col) {
-        return new ColumnTupleFilter(col);
-    }
-
-    public static ConstantTupleFilter constFilter(int id) {
-        byte[] space = new byte[10];
-        ByteBuffer buf = ByteBuffer.wrap(space);
-        StringSerializer stringSerializer = new StringSerializer(DataType.getInstance("string"));
-        stringSerializer.serialize("" + id, buf);
-        ByteArray data = new ByteArray(buf.array(), buf.arrayOffset(), buf.position());
-        return new ConstantTupleFilter(data);
-    }
-
-    public static ConciseSet set(int... ints) {
-        ConciseSet set = new ConciseSet();
-        for (int i : ints)
-            set.add(i);
-        return set;
-    }
-
-
-}


[11/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJob.java
deleted file mode 100644
index eded7c7..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJob.java
+++ /dev/null
@@ -1,101 +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.cube;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.job.constant.BatchConstants;
-
-/**
- * @author ysong1
- */
-public class MergeCuboidJob extends CuboidJob {
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_CUBE_NAME);
-            options.addOption(OPTION_SEGMENT_NAME);
-            options.addOption(OPTION_INPUT_PATH);
-            options.addOption(OPTION_OUTPUT_PATH);
-            parseOptions(options, args);
-
-            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
-            String segmentName = getOptionValue(OPTION_SEGMENT_NAME).toUpperCase();
-            KylinConfig config = KylinConfig.getInstanceFromEnv();
-            CubeManager cubeMgr = CubeManager.getInstance(config);
-            CubeInstance cube = cubeMgr.getCube(cubeName);
-
-            // start job
-            String jobName = getOptionValue(OPTION_JOB_NAME);
-            System.out.println("Starting: " + jobName);
-            job = Job.getInstance(getConf(), jobName);
-
-            setJobClasspath(job);
-
-            // set inputs
-            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
-
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-            FileOutputFormat.setOutputPath(job, output);
-
-            // Mapper
-            job.setInputFormatClass(SequenceFileInputFormat.class);
-            job.setMapperClass(MergeCuboidMapper.class);
-            job.setMapOutputKeyClass(Text.class);
-            job.setMapOutputValueClass(Text.class);
-
-            // Reducer - only one
-            job.setReducerClass(CuboidReducer.class);
-            job.setOutputFormatClass(SequenceFileOutputFormat.class);
-            job.setOutputKeyClass(Text.class);
-            job.setOutputValueClass(Text.class);
-
-            // set job configuration
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
-
-            // add metadata to distributed cache
-            attachKylinPropsAndMetadata(cube, job.getConfiguration());
-
-            setReduceTaskNum(job, config, cubeName, 0);
-
-            this.deletePath(job.getConfiguration(), output);
-
-            return waitForCompletion(job);
-        } catch (Exception e) {
-            logger.error("error in MergeCuboidJob", e);
-            printUsage(options);
-            throw e;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapper.java
deleted file mode 100644
index 0e7a89f..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapper.java
+++ /dev/null
@@ -1,192 +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.cube;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.common.RowKeySplitter;
-import org.apache.kylin.common.util.SplittedBytes;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.dict.DictionaryManager;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.kylin.metadata.model.TblColRef;
-
-/**
- * @author ysong1, honma
- */
-public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
-
-    private KylinConfig config;
-    private String cubeName;
-    private String segmentName;
-    private CubeManager cubeManager;
-    private CubeInstance cube;
-    private CubeDesc cubeDesc;
-    private CubeSegment mergedCubeSegment;
-    private CubeSegment sourceCubeSegment;// Must be unique during a mapper's
-    // life cycle
-
-    private Text outputKey = new Text();
-
-    private byte[] newKeyBuf;
-    private RowKeySplitter rowKeySplitter;
-
-    private HashMap<TblColRef, Boolean> dictsNeedMerging = new HashMap<TblColRef, Boolean>();
-
-    private static final Pattern JOB_NAME_PATTERN = Pattern.compile("kylin-([0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12})");
-
-    private Boolean checkNeedMerging(TblColRef col) throws IOException {
-        Boolean ret = dictsNeedMerging.get(col);
-        if (ret != null)
-            return ret;
-        else {
-            ret = cubeDesc.getRowkey().isUseDictionary(col);
-            if (ret) {
-                String dictTable = (String) DictionaryManager.getInstance(config).decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0];
-                ret = cubeDesc.getFactTable().equalsIgnoreCase(dictTable);
-            }
-            dictsNeedMerging.put(col, ret);
-            return ret;
-        }
-    }
-
-    private String extractJobIDFromPath(String path) {
-        Matcher matcher = JOB_NAME_PATTERN.matcher(path);
-        // check the first occurance
-        if (matcher.find()) {
-            return matcher.group(1);
-        } else {
-            throw new IllegalStateException("Can not extract job ID from file path : " + path);
-        }
-    }
-
-    private CubeSegment findSegmentWithUuid(String jobID, CubeInstance cubeInstance) {
-        for (CubeSegment segment : cubeInstance.getSegments()) {
-            String lastBuildJobID = segment.getLastBuildJobID();
-            if (lastBuildJobID != null && lastBuildJobID.equalsIgnoreCase(jobID)) {
-                return segment;
-            }
-        }
-
-        throw new IllegalStateException("No merging segment's last build job ID equals " + jobID);
-
-    }
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
-        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME).toUpperCase();
-
-        config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-
-        cubeManager = CubeManager.getInstance(config);
-        cube = cubeManager.getCube(cubeName);
-        cubeDesc = cube.getDescriptor();
-        mergedCubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
-
-        // int colCount = cubeDesc.getRowkey().getRowKeyColumns().length;
-        newKeyBuf = new byte[256];// size will auto-grow
-
-        // decide which source segment
-        InputSplit inputSplit = context.getInputSplit();
-        String filePath = ((FileSplit) inputSplit).getPath().toString();
-        System.out.println("filePath:" + filePath);
-        String jobID = extractJobIDFromPath(filePath);
-        System.out.println("jobID:" + jobID);
-        sourceCubeSegment = findSegmentWithUuid(jobID, cube);
-        System.out.println(sourceCubeSegment);
-
-        this.rowKeySplitter = new RowKeySplitter(sourceCubeSegment, 65, 255);
-    }
-
-    @Override
-    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
-        long cuboidID = rowKeySplitter.split(key.getBytes(), key.getBytes().length);
-        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidID);
-
-        SplittedBytes[] splittedByteses = rowKeySplitter.getSplitBuffers();
-        int bufOffset = 0;
-        BytesUtil.writeLong(cuboidID, newKeyBuf, bufOffset, RowConstants.ROWKEY_CUBOIDID_LEN);
-        bufOffset += RowConstants.ROWKEY_CUBOIDID_LEN;
-
-        for (int i = 0; i < cuboid.getColumns().size(); ++i) {
-            TblColRef col = cuboid.getColumns().get(i);
-
-            if (this.checkNeedMerging(col)) {
-                // if dictionary on fact table column, needs rewrite
-                DictionaryManager dictMgr = DictionaryManager.getInstance(config);
-                Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(col));
-                Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(col));
-
-                while (sourceDict.getSizeOfValue() > newKeyBuf.length - bufOffset || mergedDict.getSizeOfValue() > newKeyBuf.length - bufOffset) {
-                    byte[] oldBuf = newKeyBuf;
-                    newKeyBuf = new byte[2 * newKeyBuf.length];
-                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
-                }
-
-                int idInSourceDict = BytesUtil.readUnsigned(splittedByteses[i + 1].value, 0, splittedByteses[i + 1].length);
-                int idInMergedDict;
-
-                int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBuf, bufOffset);
-                if (size < 0) {
-                    idInMergedDict = mergedDict.nullId();
-                } else {
-                    idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBuf, bufOffset, size);
-                }
-
-                BytesUtil.writeUnsigned(idInMergedDict, newKeyBuf, bufOffset, mergedDict.getSizeOfId());
-                bufOffset += mergedDict.getSizeOfId();
-            } else {
-                // keep as it is
-                while (splittedByteses[i + 1].length > newKeyBuf.length - bufOffset) {
-                    byte[] oldBuf = newKeyBuf;
-                    newKeyBuf = new byte[2 * newKeyBuf.length];
-                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
-                }
-
-                System.arraycopy(splittedByteses[i + 1].value, 0, newKeyBuf, bufOffset, splittedByteses[i + 1].length);
-                bufOffset += splittedByteses[i + 1].length;
-            }
-        }
-        byte[] newKey = Arrays.copyOf(newKeyBuf, bufOffset);
-        outputKey.set(newKey, 0, newKey.length);
-
-        context.write(outputKey, value);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
deleted file mode 100644
index 005254b..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/MetadataCleanupJob.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-/**
- */
-public class MetadataCleanupJob extends AbstractHadoopJob {
-
-    @SuppressWarnings("static-access")
-    private static final Option OPTION_DELETE = OptionBuilder.withArgName("delete").hasArg().isRequired(false).withDescription("Delete the unused metadata").create("delete");
-
-    protected static final Logger log = LoggerFactory.getLogger(MetadataCleanupJob.class);
-
-    boolean delete = false;
-
-    private KylinConfig config = null;
-
-    public static final long TIME_THREADSHOLD = 2 * 26 * 3600 * 1000l; // 2 days
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
-     */
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        log.info("----- jobs args: " + Arrays.toString(args));
-        try {
-            options.addOption(OPTION_DELETE);
-            parseOptions(options, args);
-
-            log.info("options: '" + getOptionsAsString() + "'");
-            log.info("delete option value: '" + getOptionValue(OPTION_DELETE) + "'");
-            delete = Boolean.parseBoolean(getOptionValue(OPTION_DELETE));
-
-            config = KylinConfig.getInstanceFromEnv();
-
-            cleanup();
-
-            return 0;
-        } catch (Exception e) {
-            e.printStackTrace(System.err);
-            throw e;
-        }
-    }
-
-    private ResourceStore getStore() {
-        return ResourceStore.getStore(config);
-    }
-
-    private boolean isOlderThanThreshold(long resourceTime) {
-        long currentTime = System.currentTimeMillis();
-
-        if (currentTime - resourceTime > TIME_THREADSHOLD)
-            return true;
-        return false;
-    }
-
-    public void cleanup() throws Exception {
-        CubeManager cubeManager = CubeManager.getInstance(config);
-
-        List<String> activeResourceList = Lists.newArrayList();
-        for (org.apache.kylin.cube.CubeInstance cube : cubeManager.listAllCubes()) {
-            for (org.apache.kylin.cube.CubeSegment segment : cube.getSegments()) {
-                activeResourceList.addAll(segment.getSnapshotPaths());
-                activeResourceList.addAll(segment.getDictionaryPaths());
-                activeResourceList.add(segment.getStatisticsResourcePath());
-            }
-        }
-
-        List<String> toDeleteResource = Lists.newArrayList();
-
-        // two level resources, snapshot tables and cube statistics
-        for (String resourceRoot : new String[]{ResourceStore.SNAPSHOT_RESOURCE_ROOT, ResourceStore.CUBE_STATISTICS_ROOT}) {
-            ArrayList<String> snapshotTables = getStore().listResources(resourceRoot);
-
-            for (String snapshotTable : snapshotTables) {
-                ArrayList<String> snapshotNames = getStore().listResources(snapshotTable);
-                if (snapshotNames != null)
-                    for (String snapshot : snapshotNames) {
-                        if (!activeResourceList.contains(snapshot)) {
-                            if (isOlderThanThreshold(getStore().getResourceTimestamp(snapshot)))
-                                toDeleteResource.add(snapshot);
-                        }
-                    }
-            }
-        }
-
-        // three level resources, only dictionaries
-        ArrayList<String> dictTables = getStore().listResources(ResourceStore.DICT_RESOURCE_ROOT);
-
-        for (String table : dictTables) {
-            ArrayList<String> tableColNames = getStore().listResources(table);
-            if (tableColNames != null)
-                for (String tableCol : tableColNames) {
-                    ArrayList<String> dictionaries = getStore().listResources(tableCol);
-                    if (dictionaries != null)
-                        for (String dict : dictionaries)
-                            if (!activeResourceList.contains(dict)) {
-                                if (isOlderThanThreshold(getStore().getResourceTimestamp(dict)))
-                                    toDeleteResource.add(dict);
-                            }
-                }
-        }
-
-
-        if (toDeleteResource.size() > 0) {
-            logger.info("The following resources have no reference, will be cleaned from metadata store: \n");
-
-            for (String s : toDeleteResource) {
-                logger.info(s);
-                if (delete == true) {
-                    getStore().deleteResource(s);
-                }
-            }
-        } else {
-            logger.info("No resource to be cleaned up from metadata store;");
-        }
-
-    }
-
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new MetadataCleanupJob(), args);
-        System.exit(exitCode);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidJob.java
deleted file mode 100644
index a391979..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidJob.java
+++ /dev/null
@@ -1,39 +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.cube;
-
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-
-public class NDCuboidJob extends CuboidJob {
-
-    public NDCuboidJob() {
-        this.setMapperClass(NDCuboidMapper.class);
-    }
-
-    public static void main(String[] args) throws Exception {
-        CuboidJob job = new NDCuboidJob();
-        int exitCode = ToolRunner.run(job, args);
-        System.exit(exitCode);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapper.java
deleted file mode 100644
index c4978bf..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapper.java
+++ /dev/null
@@ -1,142 +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.cube;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.common.RowKeySplitter;
-import org.apache.kylin.common.util.SplittedBytes;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.cuboid.CuboidScheduler;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.job.constant.BatchConstants;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
-
-    private static final Logger logger = LoggerFactory.getLogger(NDCuboidMapper.class);
-
-    private Text outputKey = new Text();
-    private String cubeName;
-    private String segmentName;
-    private CubeDesc cubeDesc;
-    private CuboidScheduler cuboidScheduler;
-
-    private int handleCounter;
-    private int skipCounter;
-
-    private byte[] keyBuf = new byte[4096];
-    private RowKeySplitter rowKeySplitter;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-
-        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
-        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME).toUpperCase();
-
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-
-        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
-        CubeSegment cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
-        cubeDesc = cube.getDescriptor();
-
-        // initialize CubiodScheduler
-        cuboidScheduler = new CuboidScheduler(cubeDesc);
-
-        rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 256);
-    }
-
-    private int buildKey(Cuboid parentCuboid, Cuboid childCuboid, SplittedBytes[] splitBuffers) {
-        int offset = 0;
-
-        // cuboid id
-        System.arraycopy(childCuboid.getBytes(), 0, keyBuf, offset, childCuboid.getBytes().length);
-        offset += childCuboid.getBytes().length;
-
-        // rowkey columns
-        long mask = Long.highestOneBit(parentCuboid.getId());
-        long parentCuboidId = parentCuboid.getId();
-        long childCuboidId = childCuboid.getId();
-        long parentCuboidIdActualLength = Long.SIZE - Long.numberOfLeadingZeros(parentCuboid.getId());
-        int index = 1; // skip cuboidId
-        for (int i = 0; i < parentCuboidIdActualLength; i++) {
-            if ((mask & parentCuboidId) > 0) {// if the this bit position equals
-                                              // 1
-                if ((mask & childCuboidId) > 0) {// if the child cuboid has this
-                                                 // column
-                    System.arraycopy(splitBuffers[index].value, 0, keyBuf, offset, splitBuffers[index].length);
-                    offset += splitBuffers[index].length;
-                }
-                index++;
-            }
-            mask = mask >> 1;
-        }
-
-        return offset;
-    }
-
-    @Override
-    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
-        long cuboidId = rowKeySplitter.split(key.getBytes(), key.getLength());
-        Cuboid parentCuboid = Cuboid.findById(cubeDesc, cuboidId);
-
-        Collection<Long> myChildren = cuboidScheduler.getSpanningCuboid(cuboidId);
-
-        // if still empty or null
-        if (myChildren == null || myChildren.size() == 0) {
-            context.getCounter(BatchConstants.MAPREDUCE_COUTNER_GROUP_NAME, "Skipped records").increment(1L);
-            skipCounter++;
-            if (skipCounter % BatchConstants.COUNTER_MAX == 0) {
-                logger.info("Skipped " + skipCounter + " records!");
-            }
-            return;
-        }
-
-        context.getCounter(BatchConstants.MAPREDUCE_COUTNER_GROUP_NAME, "Processed records").increment(1L);
-
-        handleCounter++;
-        if (handleCounter % BatchConstants.COUNTER_MAX == 0) {
-            logger.info("Handled " + handleCounter + " records!");
-        }
-
-        for (Long child : myChildren) {
-            Cuboid childCuboid = Cuboid.findById(cubeDesc, child);
-            int keyLength = buildKey(parentCuboid, childCuboid, rowKeySplitter.getSplitBuffers());
-            outputKey.set(keyBuf, 0, keyLength);
-            context.write(outputKey, value);
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
index 97b5de8..2acace4 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
@@ -45,8 +45,8 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.DimensionDesc;
 import org.apache.kylin.dict.lookup.LookupBytesTable;
 import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.FunctionDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java
index 277624a..b259e1a 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
deleted file mode 100644
index 7c5e5db..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
+++ /dev/null
@@ -1,110 +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.cube;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.apache.hadoop.util.ToolRunner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
-
-/**
- * @author xjiang, ysong1
- * 
- */
-
-public class RangeKeyDistributionJob extends AbstractHadoopJob {
-    protected static final Logger log = LoggerFactory.getLogger(RangeKeyDistributionJob.class);
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
-     */
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_INPUT_PATH);
-            options.addOption(OPTION_OUTPUT_PATH);
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_CUBE_NAME);
-
-            parseOptions(options, args);
-
-            // start job
-            String jobName = getOptionValue(OPTION_JOB_NAME);
-            job = Job.getInstance(getConf(), jobName);
-
-            setJobClasspath(job);
-
-            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
-
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-            FileOutputFormat.setOutputPath(job, output);
-            // job.getConfiguration().set("dfs.block.size", "67108864");
-
-            // Mapper
-            job.setInputFormatClass(SequenceFileInputFormat.class);
-            job.setMapperClass(RangeKeyDistributionMapper.class);
-            job.setMapOutputKeyClass(Text.class);
-            job.setMapOutputValueClass(LongWritable.class);
-
-            // Reducer - only one
-            job.setReducerClass(RangeKeyDistributionReducer.class);
-            job.setOutputFormatClass(SequenceFileOutputFormat.class);
-            job.setOutputKeyClass(Text.class);
-            job.setOutputValueClass(LongWritable.class);
-            job.setNumReduceTasks(1);
-
-            this.deletePath(job.getConfiguration(), output);
-
-            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
-            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
-            CubeInstance cube = cubeMgr.getCube(cubeName);
-            RealizationCapacity realizationCapacity = cube.getDescriptor().getModel().getCapacity();
-            job.getConfiguration().set(BatchConstants.CUBE_CAPACITY, realizationCapacity.toString());
-
-            return waitForCompletion(job);
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new RangeKeyDistributionJob(), args);
-        System.exit(exitCode);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapper.java
deleted file mode 100644
index e49f090..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapper.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import java.io.IOException;
-
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-
-/**
- * @author ysong1
- * 
- */
-public class RangeKeyDistributionMapper extends KylinMapper<Text, Text, Text, LongWritable> {
-
-    private static final long ONE_MEGA_BYTES = 1L * 1024L * 1024L;
-
-    private LongWritable outputValue = new LongWritable(0);
-
-    private long bytesRead = 0;
-
-    private Text lastKey;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-    }
-
-    @Override
-    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
-        lastKey = key;
-
-        int bytesLength = key.getLength() + value.getLength();
-        bytesRead += bytesLength;
-
-        if (bytesRead >= ONE_MEGA_BYTES) {
-            outputValue.set(bytesRead);
-            context.write(key, outputValue);
-
-            // reset bytesRead
-            bytesRead = 0;
-        }
-
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-        if (lastKey != null) {
-            outputValue.set(bytesRead);
-            context.write(lastKey, outputValue);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
deleted file mode 100644
index a580a9e..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.kylin.cube.model.v1.CubeDesc.CubeCapacity;
-import org.apache.kylin.engine.mr.KylinReducer;
-import org.apache.kylin.job.constant.BatchConstants;
-
-/**
- * @author ysong1
- * 
- */
-public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable, Text, LongWritable> {
-
-    public static final long ONE_GIGA_BYTES = 1024L * 1024L * 1024L;
-    public static final int SMALL_CUT = 5;  //  5 GB per region
-    public static final int MEDIUM_CUT = 10; //  10 GB per region
-    public static final int LARGE_CUT = 50; // 50 GB per region
-    
-    public static final int MAX_REGION = 1000;
-
-    private static final Logger logger = LoggerFactory.getLogger(RangeKeyDistributionReducer.class);
-
-    private LongWritable outputValue = new LongWritable(0);
-
-    private int cut;
-    private long bytesRead = 0;
-    private List<Text> gbPoints = new ArrayList<Text>();
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        CubeCapacity cubeCapacity = CubeCapacity.valueOf(context.getConfiguration().get(BatchConstants.CUBE_CAPACITY));
-        switch (cubeCapacity) {
-        case SMALL:
-            cut = SMALL_CUT;
-            break;
-        case MEDIUM:
-            cut = MEDIUM_CUT;
-            break;
-        case LARGE:
-            cut = LARGE_CUT;
-            break;
-        }
-
-        logger.info("Chosen cut for htable is " + cut);
-    }
-
-    @Override
-    public void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
-        for (LongWritable v : values) {
-            bytesRead += v.get();
-        }
-        
-        if (bytesRead >= ONE_GIGA_BYTES) {
-            gbPoints.add(new Text(key));
-            bytesRead = 0; // reset bytesRead
-        }
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-        int nRegion = Math.round((float) gbPoints.size() / (float) cut);
-        nRegion = Math.max(1,  nRegion);
-        nRegion = Math.min(MAX_REGION, nRegion);
-        
-        int gbPerRegion = gbPoints.size() / nRegion;
-        gbPerRegion = Math.max(1, gbPerRegion);
-        
-        System.out.println(nRegion + " regions");
-        System.out.println(gbPerRegion + " GB per region");
-        
-        for (int i = gbPerRegion; i < gbPoints.size(); i += gbPerRegion) {
-            Text key = gbPoints.get(i);
-            outputValue.set(i);
-            System.out.println(StringUtils.byteToHexString(key.getBytes()) + "\t" + outputValue.get());
-            context.write(key, outputValue);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerJob.java
deleted file mode 100644
index faf6675..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerJob.java
+++ /dev/null
@@ -1,98 +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.cube;
-
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.apache.hadoop.util.ToolRunner;
-
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-
-/**
- * @author ysong1
- * 
- */
-public class RowKeyDistributionCheckerJob extends AbstractHadoopJob {
-
-    @SuppressWarnings("static-access")
-    protected static final Option rowKeyStatsFilePath = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("rowKeyStatsFilePath").create("rowKeyStatsFilePath");
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_INPUT_PATH);
-            options.addOption(OPTION_OUTPUT_PATH);
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(rowKeyStatsFilePath);
-
-            parseOptions(options, args);
-
-            String statsFilePath = getOptionValue(rowKeyStatsFilePath);
-
-            // start job
-            String jobName = getOptionValue(OPTION_JOB_NAME);
-            job = Job.getInstance(getConf(), jobName);
-
-            setJobClasspath(job);
-
-            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
-
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-            FileOutputFormat.setOutputPath(job, output);
-
-            // Mapper
-            job.setInputFormatClass(SequenceFileInputFormat.class);
-            job.setMapperClass(RowKeyDistributionCheckerMapper.class);
-            job.setMapOutputKeyClass(Text.class);
-            job.setMapOutputValueClass(LongWritable.class);
-
-            // Reducer - only one
-            job.setReducerClass(RowKeyDistributionCheckerReducer.class);
-            job.setOutputFormatClass(SequenceFileOutputFormat.class);
-            job.setOutputKeyClass(Text.class);
-            job.setOutputValueClass(LongWritable.class);
-            job.setNumReduceTasks(1);
-
-            job.getConfiguration().set("rowKeyStatsFilePath", statsFilePath);
-
-            this.deletePath(job.getConfiguration(), output);
-
-            return waitForCompletion(job);
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new RowKeyDistributionCheckerJob(), args);
-        System.exit(exitCode);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
deleted file mode 100644
index de57562..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
+++ /dev/null
@@ -1,107 +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.cube;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.kylin.engine.mr.KylinMapper;
-
-/**
- * @author ysong1
- * 
- */
-public class RowKeyDistributionCheckerMapper extends KylinMapper<Text, Text, Text, LongWritable> {
-
-    String rowKeyStatsFilePath;
-    byte[][] splitKeys;
-    Map<Text, Long> resultMap;
-    List<Text> keyList;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        rowKeyStatsFilePath = context.getConfiguration().get("rowKeyStatsFilePath");
-        splitKeys = this.getSplits(context.getConfiguration(), new Path(rowKeyStatsFilePath));
-
-        resultMap = new HashMap<Text, Long>();
-        keyList = new ArrayList<Text>();
-        for (int i = 0; i < splitKeys.length; i++) {
-            Text key = new Text(splitKeys[i]);
-            resultMap.put(key, 0L);
-            keyList.add(new Text(splitKeys[i]));
-        }
-    }
-
-    @Override
-    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
-        for (Text t : keyList) {
-            if (key.compareTo(t) < 0) {
-                Long v = resultMap.get(t);
-                long length = key.getLength() + value.getLength();
-                v += length;
-                resultMap.put(t, v);
-                break;
-            }
-        }
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-        LongWritable outputValue = new LongWritable();
-        for (Entry<Text, Long> kv : resultMap.entrySet()) {
-            outputValue.set(kv.getValue());
-            context.write(kv.getKey(), outputValue);
-        }
-    }
-
-    @SuppressWarnings("deprecation")
-    public byte[][] getSplits(Configuration conf, Path path) {
-        List<byte[]> rowkeyList = new ArrayList<byte[]>();
-        SequenceFile.Reader reader = null;
-        try {
-            reader = new SequenceFile.Reader(path.getFileSystem(conf), path, conf);
-            Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
-            Writable value = (Writable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
-            while (reader.next(key, value)) {
-                byte[] tmp = ((Text) key).copyBytes();
-                if (rowkeyList.contains(tmp) == false) {
-                    rowkeyList.add(tmp);
-                }
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-        } finally {
-            IOUtils.closeStream(reader);
-        }
-
-        byte[][] retValue = rowkeyList.toArray(new byte[rowkeyList.size()][]);
-
-        return retValue;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
deleted file mode 100644
index c010a33..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
+++ /dev/null
@@ -1,51 +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.cube;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.kylin.engine.mr.KylinReducer;
-
-/**
- * @author ysong1
- * 
- */
-public class RowKeyDistributionCheckerReducer extends KylinReducer<Text, LongWritable, Text, LongWritable> {
-
-    LongWritable outputKey = new LongWritable(0L);
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-    }
-
-    @Override
-    public void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
-
-        long length = 0;
-        for (LongWritable v : values) {
-            length += v.get();
-        }
-
-        outputKey.set(length);
-        context.write(key, outputKey);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
index d88b116..1247cd3 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
@@ -35,13 +35,13 @@ 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.common.AbstractHadoopJob;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.job.JobInstance;
 import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.execution.ExecutableState;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateDictionaryJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateDictionaryJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateDictionaryJob.java
deleted file mode 100644
index 1a5e690..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateDictionaryJob.java
+++ /dev/null
@@ -1,75 +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.cube.cli.DictionaryGeneratorCLI;
-import org.apache.kylin.dict.DistinctColumnValuesProvider;
-import org.apache.kylin.engine.mr.DFSFileTable;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.ReadableTable;
-
-/**
- * @author ysong1
- * 
- */
-
-public class CreateDictionaryJob extends AbstractHadoopJob {
-
-    private int returnCode = 0;
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_CUBE_NAME);
-            options.addOption(OPTION_SEGMENT_NAME);
-            options.addOption(OPTION_INPUT_PATH);
-            parseOptions(options, args);
-
-            final String cubeName = getOptionValue(OPTION_CUBE_NAME);
-            final String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
-            final String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
-
-            KylinConfig config = KylinConfig.getInstanceFromEnv();
-
-            DictionaryGeneratorCLI.processSegment(config, cubeName, segmentName, new DistinctColumnValuesProvider() {
-                @Override
-                public ReadableTable getDistinctValuesFor(TblColRef col) {
-                    return new DFSFileTable(factColumnsInputPath + "/" + col.getName(), -1);
-                }
-            });
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-
-        return returnCode;
-    }
-    
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new CreateDictionaryJob(), args);
-        System.exit(exitCode);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
index 66a1106..87ee70e 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
@@ -23,9 +23,9 @@ 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.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.source.ReadableTable;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
deleted file mode 100644
index 7a4702e..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
+++ /dev/null
@@ -1,101 +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.hbase;
-
-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.HBaseConfiguration;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.util.ToolRunner;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author ysong1
- * 
- */
-public class BulkLoadJob extends AbstractHadoopJob {
-
-    protected static final Logger log = LoggerFactory.getLogger(BulkLoadJob.class);
-
-    @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_CUBE_NAME);
-            parseOptions(options, args);
-
-            String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
-            // e.g
-            // /tmp/kylin-3f150b00-3332-41ca-9d3d-652f67f044d7/test_kylin_cube_with_slr_ready_2_segments/hfile/
-            // end with "/"
-            String input = getOptionValue(OPTION_INPUT_PATH);
-
-            Configuration conf = HBaseConfiguration.create(getConf());
-            FileSystem fs = FileSystem.get(conf);
-
-            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
-            KylinConfig config = KylinConfig.getInstanceFromEnv();
-            CubeManager cubeMgr = CubeManager.getInstance(config);
-            CubeInstance cube = cubeMgr.getCube(cubeName);
-            CubeDesc cubeDesc = cube.getDescriptor();
-            FsPermission permission = new FsPermission((short) 0777);
-            for (HBaseColumnFamilyDesc cf : cubeDesc.getHBaseMapping().getColumnFamily()) {
-                String cfName = cf.getName();
-                Path columnFamilyPath = new Path(input + cfName);
-
-                // File may have already been auto-loaded (in the case of MapR DB)
-                if(fs.exists(columnFamilyPath)) {
-                    fs.setPermission(columnFamilyPath, permission);
-                }
-            }
-
-            String[] newArgs = new String[2];
-            newArgs[0] = input;
-            newArgs[1] = tableName;
-
-            log.debug("Start to run LoadIncrementalHFiles");
-            int ret = ToolRunner.run(new LoadIncrementalHFiles(conf), newArgs);
-            log.debug("End to run LoadIncrementalHFiles");
-            return ret;
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new BulkLoadJob(), args);
-        System.exit(exitCode);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
deleted file mode 100644
index 304fa04..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
+++ /dev/null
@@ -1,314 +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.hbase;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.DataModelDesc;
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-/**
- */
-public class CreateHTableJob extends AbstractHadoopJob {
-
-    protected static final Logger logger = LoggerFactory.getLogger(CreateHTableJob.class);
-
-    CubeInstance cube = null;
-    CubeDesc cubeDesc = null;
-    String segmentName = null;
-    KylinConfig kylinConfig;
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        options.addOption(OPTION_CUBE_NAME);
-        options.addOption(OPTION_SEGMENT_NAME);
-        options.addOption(OPTION_PARTITION_FILE_PATH);
-        options.addOption(OPTION_HTABLE_NAME);
-        options.addOption(OPTION_STATISTICS_ENABLED);
-        parseOptions(options, args);
-
-        Path partitionFilePath = new Path(getOptionValue(OPTION_PARTITION_FILE_PATH));
-        boolean statistics_enabled = Boolean.parseBoolean(getOptionValue(OPTION_STATISTICS_ENABLED));
-
-        String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
-        kylinConfig = KylinConfig.getInstanceFromEnv();
-        CubeManager cubeMgr = CubeManager.getInstance(kylinConfig);
-        cube = cubeMgr.getCube(cubeName);
-        cubeDesc = cube.getDescriptor();
-        segmentName = getOptionValue(OPTION_SEGMENT_NAME);
-        CubeSegment cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
-
-        String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
-        Configuration conf = HBaseConfiguration.create(getConf());
-
-        try {
-
-            byte[][] splitKeys;
-            if (statistics_enabled) {
-                List<Integer> rowkeyColumnSize = Lists.newArrayList();
-                long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-                Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
-                List<TblColRef> columnList = baseCuboid.getColumns();
-
-                for (int i = 0; i < columnList.size(); i++) {
-                    logger.info("Rowkey column " + i + " length " + cubeSegment.getColumnLength(columnList.get(i)));
-                    rowkeyColumnSize.add(cubeSegment.getColumnLength(columnList.get(i)));
-                }
-
-                splitKeys = getSplitsFromCuboidStatistics(conf, kylinConfig, rowkeyColumnSize, cubeSegment);
-            } else {
-                splitKeys = getSplits(conf, partitionFilePath);
-            }
-
-            CubeHTableUtil.createHTable(cubeDesc, tableName, splitKeys);
-            return 0;
-        } catch (Exception e) {
-            printUsage(options);
-            e.printStackTrace(System.err);
-            logger.error(e.getLocalizedMessage(), e);
-            return 2;
-        }
-    }
-
-    @SuppressWarnings("deprecation")
-    public byte[][] getSplits(Configuration conf, Path path) throws Exception {
-        FileSystem fs = path.getFileSystem(conf);
-        if (fs.exists(path) == false) {
-            System.err.println("Path " + path + " not found, no region split, HTable will be one region");
-            return null;
-        }
-
-        List<byte[]> rowkeyList = new ArrayList<byte[]>();
-        SequenceFile.Reader reader = null;
-        try {
-            reader = new SequenceFile.Reader(fs, path, conf);
-            Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
-            Writable value = (Writable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
-            while (reader.next(key, value)) {
-                rowkeyList.add(((Text) key).copyBytes());
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw e;
-        } finally {
-            IOUtils.closeStream(reader);
-        }
-
-        logger.info((rowkeyList.size() + 1) + " regions");
-        logger.info(rowkeyList.size() + " splits");
-        for (byte[] split : rowkeyList) {
-            logger.info(StringUtils.byteToHexString(split));
-        }
-
-        byte[][] retValue = rowkeyList.toArray(new byte[rowkeyList.size()][]);
-        return retValue.length == 0 ? null : retValue;
-    }
-
-
-    @SuppressWarnings("deprecation")
-    public static byte[][] getSplitsFromCuboidStatistics(Configuration conf, KylinConfig kylinConfig,  List<Integer> rowkeyColumnSize, CubeSegment cubeSegment) throws IOException {
-
-        CubeDesc cubeDesc = cubeSegment.getCubeDesc();
-        DataModelDesc.RealizationCapacity cubeCapacity = cubeDesc.getModel().getCapacity();
-        int cut = kylinConfig.getHBaseRegionCut(cubeCapacity.toString());
-
-        logger.info("Cube capacity " + cubeCapacity.toString() + ", chosen cut for HTable is " + cut + "GB");
-
-        Map<Long, Long> cuboidSizeMap = Maps.newHashMap();
-        long totalSizeInM = 0;
-
-        ResourceStore rs = ResourceStore.getStore(kylinConfig);
-        String fileKey = cubeSegment.getStatisticsResourcePath();
-        InputStream is = rs.getResource(fileKey);
-        File tempFile = null;
-        FileOutputStream tempFileStream = null;
-        try {
-            tempFile = File.createTempFile(cubeSegment.getUuid(), ".seq");
-            tempFileStream = new FileOutputStream(tempFile);
-            org.apache.commons.io.IOUtils.copy(is, tempFileStream);
-        } finally {
-            IOUtils.closeStream(is);
-            IOUtils.closeStream(tempFileStream);
-        }
-
-        FileSystem fs = HadoopUtil.getFileSystem("file:///" + tempFile.getAbsolutePath());
-        SequenceFile.Reader reader = null;
-        try {
-            reader = new SequenceFile.Reader(fs, new Path(tempFile.getAbsolutePath()), conf);
-            LongWritable key = (LongWritable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
-            BytesWritable value = (BytesWritable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
-            int samplingPercentage = 25;
-            while (reader.next(key, value)) {
-                if (key.get() == 0l) {
-                    samplingPercentage = Bytes.toInt(value.getBytes());
-                } else {
-                    HyperLogLogPlusCounter hll = new HyperLogLogPlusCounter(14);
-                    ByteArray byteArray = new ByteArray(value.getBytes());
-                    hll.readRegisters(byteArray.asBuffer());
-
-                    cuboidSizeMap.put(key.get(), hll.getCountEstimate() * 100 / samplingPercentage);
-                }
-
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw e;
-        } finally {
-            IOUtils.closeStream(reader);
-        }
-
-        List<Long> allCuboids = Lists.newArrayList();
-        allCuboids.addAll(cuboidSizeMap.keySet());
-        Collections.sort(allCuboids);
-
-        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        for (long cuboidId : allCuboids) {
-            long cuboidSize = estimateCuboidStorageSize(cubeDesc, cuboidId, cuboidSizeMap.get(cuboidId), baseCuboidId, rowkeyColumnSize);
-            cuboidSizeMap.put(cuboidId, cuboidSize);
-            totalSizeInM += cuboidSize;
-        }
-
-        int nRegion = Math.round((float) totalSizeInM / ((float) cut * 1024l));
-        nRegion = Math.max(kylinConfig.getHBaseRegionCutMin(), nRegion);
-        nRegion = Math.min(kylinConfig.getHBaseRegionCutMax(), nRegion);
-
-        int mbPerRegion = (int) (totalSizeInM / (nRegion));
-        mbPerRegion = Math.max(1, mbPerRegion);
-
-        logger.info("Total size " + totalSizeInM + "M (estimated)");
-        logger.info(nRegion + " regions (estimated)");
-        logger.info(mbPerRegion + " MB per region (estimated)");
-
-        List<Long> regionSplit = Lists.newArrayList();
-
-
-        long size = 0;
-        int regionIndex = 0;
-        int cuboidCount = 0;
-        for (int i = 0; i < allCuboids.size(); i++) {
-            long cuboidId = allCuboids.get(i);
-            if (size >= mbPerRegion || (size + cuboidSizeMap.get(cuboidId)) >= mbPerRegion * 1.2) {
-                // if the size already bigger than threshold, or it will exceed by 20%, cut for next region
-                regionSplit.add(cuboidId);
-                logger.info("Region " + regionIndex + " will be " + size + " MB, contains cuboids < " + cuboidId + " (" + cuboidCount + ") cuboids");
-                size = 0;
-                cuboidCount = 0;
-                regionIndex++;
-            }
-            size += cuboidSizeMap.get(cuboidId);
-            cuboidCount++;
-        }
-
-
-        byte[][] result = new byte[regionSplit.size()][];
-        for (int i = 0; i < regionSplit.size(); i++) {
-            result[i] = Bytes.toBytes(regionSplit.get(i));
-        }
-
-        return result;
-    }
-
-    /**
-     * Estimate the cuboid's size
-     *
-     * @param cubeDesc
-     * @param cuboidId
-     * @param rowCount
-     * @return the cuboid size in M bytes
-     */
-    private static long estimateCuboidStorageSize(CubeDesc cubeDesc, long cuboidId, long rowCount, long baseCuboidId, List<Integer> rowKeyColumnLength) {
-
-        int bytesLength = RowConstants.ROWKEY_CUBOIDID_LEN;
-
-        long mask = Long.highestOneBit(baseCuboidId);
-        long parentCuboidIdActualLength = Long.SIZE - Long.numberOfLeadingZeros(baseCuboidId);
-        for (int i = 0; i < parentCuboidIdActualLength; i++) {
-            if ((mask & cuboidId) > 0) {
-                bytesLength += rowKeyColumnLength.get(i); //colIO.getColumnLength(columnList.get(i));
-            }
-            mask = mask >> 1;
-        }
-
-        // add the measure length
-        int space = 0;
-        for (MeasureDesc measureDesc : cubeDesc.getMeasures()) {
-            DataType returnType = measureDesc.getFunction().getReturnDataType();
-            if (returnType.isHLLC()) {
-                // for HLL, it will be compressed when export to bytes
-                space += returnType.getSpaceEstimate() * 0.75;
-            } else {
-                space += returnType.getSpaceEstimate();
-            }
-        }
-        bytesLength += space;
-
-        logger.info("Cuboid " + cuboidId + " has " + rowCount + " rows, each row size is " + bytesLength + " bytes.");
-        logger.info("Cuboid " + cuboidId + " total size is " + (bytesLength * rowCount / (1024L * 1024L)) + "M.");
-        return bytesLength * rowCount / (1024L * 1024L);
-    }
-
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new CreateHTableJob(), args);
-        System.exit(exitCode);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CubeHTableUtil.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CubeHTableUtil.java b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CubeHTableUtil.java
deleted file mode 100644
index 88b345b..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CubeHTableUtil.java
+++ /dev/null
@@ -1,84 +0,0 @@
-package org.apache.kylin.job.hadoop.hbase;
-
-import com.google.common.base.Preconditions;
-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.ConstantSizeRegionSplitPolicy;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
-import org.apache.kylin.job.tools.DeployCoprocessorCLI;
-import org.apache.kylin.job.tools.LZOSupportnessChecker;
-import org.apache.kylin.metadata.realization.IRealizationConstants;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- */
-public class CubeHTableUtil {
-
-    private static final Logger logger = LoggerFactory.getLogger(CubeHTableUtil.class);
-
-    public static void createHTable(CubeDesc cubeDesc, String tableName, byte[][] splitKeys) throws IOException {
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-
-        HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(tableName));
-        // https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.html
-        tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName());
-        tableDesc.setValue(IRealizationConstants.HTableTag, kylinConfig.getMetadataUrlPrefix());
-
-        Configuration conf = HBaseConfiguration.create();
-        HBaseAdmin admin = new HBaseAdmin(conf);
-
-        try {
-            if (User.isHBaseSecurityEnabled(conf)) {
-                // add coprocessor for bulk load
-                tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
-            }
-
-            for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
-                HColumnDescriptor cf = new HColumnDescriptor(cfDesc.getName());
-                cf.setMaxVersions(1);
-
-                if (LZOSupportnessChecker.getSupportness()) {
-                    logger.info("hbase will use lzo to compress cube data");
-                    cf.setCompressionType(Compression.Algorithm.LZO);
-                } else {
-                    logger.info("hbase will not use lzo to compress cube data");
-                }
-
-                cf.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
-                cf.setInMemory(false);
-                cf.setBlocksize(4 * 1024 * 1024); // set to 4MB
-                tableDesc.addFamily(cf);
-            }
-
-            if (admin.tableExists(tableName)) {
-                // admin.disableTable(tableName);
-                // admin.deleteTable(tableName);
-                throw new RuntimeException("HBase table " + tableName + " exists!");
-            }
-
-            DeployCoprocessorCLI.deployCoprocessor(tableDesc);
-
-            admin.createTable(tableDesc, splitKeys);
-            Preconditions.checkArgument(admin.isTableAvailable(tableName), "table " + tableName + " created, but is not available due to some reasons");
-            logger.info("create hbase table " + tableName + " done.");
-        } catch (Exception e) {
-            logger.error("Failed to create HTable", e);
-            throw e;
-        } finally {
-            admin.close();
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/hive/SqlHiveDataTypeMapping.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/hive/SqlHiveDataTypeMapping.java b/job/src/main/java/org/apache/kylin/job/hadoop/hive/SqlHiveDataTypeMapping.java
deleted file mode 100644
index f710d34..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/hive/SqlHiveDataTypeMapping.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.hive;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-public class SqlHiveDataTypeMapping {
-
-    public static String getHiveDataType(String javaDataType) {
-        String hiveDataType = javaDataType.toLowerCase().startsWith("varchar") ? "string" : javaDataType;
-        hiveDataType = javaDataType.toLowerCase().startsWith("integer") ? "int" : hiveDataType;
-
-        return hiveDataType.toLowerCase();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
index 8575f89..d3f6d68 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIBulkLoadJob.java
@@ -25,11 +25,11 @@ 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.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
index d55900b..08b7a46 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
index 3690e48..6149d27 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
@@ -32,15 +32,15 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.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.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.job.tools.DeployCoprocessorCLI;
-import org.apache.kylin.job.tools.LZOSupportnessChecker;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.storage.hbase.HBaseConnection;
+import org.apache.kylin.storage.hbase.util.DeployCoprocessorCLI;
+import org.apache.kylin.storage.hbase.util.LZOSupportnessChecker;
 
 /**
  * @author George Song (ysong1)

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
index b1ff6f1..1ed6b89 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
@@ -32,10 +32,10 @@ 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.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.IntermediateColumnDesc;
 import org.slf4j.Logger;



[10/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
index 1676429..2fb28c0 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.engine.mr.KylinReducer;
-import org.apache.kylin.job.constant.BatchConstants;
+import org.apache.kylin.engine.mr.common.BatchConstants;
 
 /**
  * @author yangli9

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
index 16b8ca1..741dd62 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
@@ -19,6 +19,7 @@
 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;
@@ -32,11 +33,13 @@ 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.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.TableDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -102,6 +105,26 @@ 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(String intermediateTable) throws IOException {
 
         String[] dbTableNames = HadoopUtil.parseHiveTableName(intermediateTable);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
index 1d30ee7..0efd585 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexMapper.java
@@ -30,13 +30,13 @@ 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.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexPartitioner.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexPartitioner.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexPartitioner.java
index fa4dccf..584c96b 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexPartitioner.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexPartitioner.java
@@ -27,13 +27,13 @@ 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.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 
 /**
  * @author yangli9

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
index d9b5aee..9476428 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexReducer.java
@@ -23,6 +23,8 @@ 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;
@@ -32,8 +34,6 @@ 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.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionJob.java
deleted file mode 100644
index 0f94d32..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionJob.java
+++ /dev/null
@@ -1,116 +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.Option;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.kylin.job.constant.BatchConstants;
-
-/**
- * @author ysong1
- * 
- */
-@SuppressWarnings("static-access")
-public class RandomKeyDistributionJob extends AbstractHadoopJob {
-
-    protected static final Logger log = LoggerFactory.getLogger(RandomKeyDistributionJob.class);
-
-    static final Option OPTION_KEY_CLASS = OptionBuilder.withArgName("keyclass").hasArg().isRequired(true).withDescription("Key Class").create("keyclass");
-    static final Option OPTION_REGION_MB = OptionBuilder.withArgName("regionmb").hasArg().isRequired(true).withDescription("MB per Region").create("regionmb");
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_INPUT_PATH);
-            options.addOption(OPTION_OUTPUT_PATH);
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_KEY_CLASS);
-            options.addOption(OPTION_REGION_MB);
-
-            parseOptions(options, args);
-
-            // start job
-            String jobName = getOptionValue(OPTION_JOB_NAME);
-            job = Job.getInstance(getConf(), jobName);
-
-            setJobClasspath(job);
-            
-            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
-
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-            FileOutputFormat.setOutputPath(job, output);
-
-            String keyClass = getOptionValue(OPTION_KEY_CLASS);
-            Class<?> keyClz = Class.forName(keyClass);
-
-            int regionMB = Integer.parseInt(getOptionValue(OPTION_REGION_MB));
-
-            // Mapper
-            job.setInputFormatClass(SequenceFileInputFormat.class);
-            job.setMapperClass(RandomKeyDistributionMapper.class);
-            job.setMapOutputKeyClass(keyClz);
-            job.setMapOutputValueClass(NullWritable.class);
-
-            // Reducer - only one
-            job.setReducerClass(RandomKeyDistributionReducer.class);
-            job.setOutputFormatClass(SequenceFileOutputFormat.class);
-            job.setOutputKeyClass(keyClz);
-            job.setOutputValueClass(NullWritable.class);
-            job.setNumReduceTasks(1);
-
-            this.deletePath(job.getConfiguration(), output);
-
-            // total map input MB
-            double totalMapInputMB = this.getTotalMapInputMB();
-            int regionCount = Math.max(1, (int) (totalMapInputMB / regionMB));
-            int mapSampleNumber = 1000;
-            System.out.println("Total Map Input MB: " + totalMapInputMB);
-            System.out.println("Region Count: " + regionCount);
-
-            // set job configuration
-            job.getConfiguration().set(BatchConstants.MAPPER_SAMPLE_NUMBER, String.valueOf(mapSampleNumber));
-            job.getConfiguration().set(BatchConstants.REGION_NUMBER, String.valueOf(regionCount));
-
-            return waitForCompletion(job);
-        } catch (Exception e) {
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new RandomKeyDistributionJob(), args);
-        System.exit(exitCode);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
deleted file mode 100644
index e3e743e..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.invertedindex;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.kylin.common.util.RandomSampler;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.job.constant.BatchConstants;
-
-/**
- * @author ysong1
- * 
- */
-public class RandomKeyDistributionMapper<KEY extends Writable, VALUE> extends KylinMapper<KEY, VALUE, KEY, NullWritable> {
-
-    private Configuration conf;
-    private int sampleNumber;
-    private List<KEY> allKeys;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        conf = context.getConfiguration();
-        allKeys = new ArrayList<KEY>();
-        sampleNumber = Integer.parseInt(conf.get(BatchConstants.MAPPER_SAMPLE_NUMBER));
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public void map(KEY key, VALUE value, Context context) throws IOException, InterruptedException {
-        KEY keyCopy = (KEY) ReflectionUtils.newInstance(key.getClass(), conf);
-        ReflectionUtils.copy(conf, key, keyCopy);
-        allKeys.add(keyCopy);
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-        RandomSampler<KEY> sampler = new RandomSampler<KEY>();
-        List<KEY> sampleResult = sampler.sample(allKeys, sampleNumber);
-        for (KEY k : sampleResult) {
-            context.write(k, NullWritable.get());
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
deleted file mode 100644
index bc6d379..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.invertedindex;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.kylin.engine.mr.KylinReducer;
-import org.apache.kylin.job.constant.BatchConstants;
-
-/**
- * @author ysong1
- * 
- */
-public class RandomKeyDistributionReducer<KEY extends Writable> extends KylinReducer<KEY, NullWritable, KEY, NullWritable> {
-
-
-    private Configuration conf;
-    private int regionNumber;
-    private List<KEY> allSplits;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        conf = context.getConfiguration();
-        allSplits = new ArrayList<KEY>();
-        regionNumber = Integer.parseInt(context.getConfiguration().get(BatchConstants.REGION_NUMBER));
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public void reduce(KEY key, Iterable<NullWritable> values, Context context) throws IOException, InterruptedException {
-        KEY keyCopy = (KEY) ReflectionUtils.newInstance(key.getClass(), conf);
-        ReflectionUtils.copy(conf, key, keyCopy);
-        allSplits.add(keyCopy);
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-        int stepLength = allSplits.size() / regionNumber;
-        for (int i = stepLength; i < allSplits.size(); i += stepLength) {
-            context.write(allSplits.get(i), NullWritable.get());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultContext.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultContext.java b/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultContext.java
deleted file mode 100644
index e959ae2..0000000
--- a/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultContext.java
+++ /dev/null
@@ -1,61 +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.impl.threadpool;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.job.execution.Executable;
-import org.apache.kylin.job.execution.ExecutableContext;
-
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- */
-public class DefaultContext implements ExecutableContext {
-
-    private final ConcurrentMap<String, Executable> runningJobs;
-    private final KylinConfig kylinConfig;
-
-    public DefaultContext(ConcurrentMap<String, Executable> runningJobs, KylinConfig kylinConfig) {
-        this.runningJobs = runningJobs;
-        this.kylinConfig = kylinConfig;
-    }
-    @Override
-    public Object getSchedulerContext() {
-        return null;
-    }
-
-    @Override
-    public KylinConfig getConfig() {
-        return kylinConfig;
-    }
-
-    void addRunningJob(Executable executable) {
-        runningJobs.put(executable.getId(), executable);
-    }
-
-    void removeRunningJob(Executable executable) {
-        runningJobs.remove(executable.getId());
-    }
-
-    public Map<String, Executable> getRunningJobs() {
-        return Collections.unmodifiableMap(runningJobs);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java b/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
deleted file mode 100644
index 8a83870..0000000
--- a/job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
+++ /dev/null
@@ -1,209 +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.impl.threadpool;
-
-import com.google.common.collect.Maps;
-
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.state.ConnectionState;
-import org.apache.curator.framework.state.ConnectionStateListener;
-import org.apache.kylin.job.Scheduler;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.exception.SchedulerException;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.Executable;
-import org.apache.kylin.job.execution.ExecutableState;
-import org.apache.kylin.job.execution.Output;
-import org.apache.kylin.job.lock.JobLock;
-import org.apache.kylin.job.manager.ExecutableManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-import java.util.concurrent.*;
-
-/**
- */
-public class DefaultScheduler implements Scheduler<AbstractExecutable>, ConnectionStateListener {
-
-
-    private ExecutableManager executableManager;
-    private FetcherRunner fetcher;
-    private ScheduledExecutorService fetcherPool;
-    private ExecutorService jobPool;
-    private DefaultContext context;
-
-    private Logger logger = LoggerFactory.getLogger(DefaultScheduler.class);
-    private volatile boolean initialized = false;
-    private volatile boolean hasStarted = false;
-    private JobEngineConfig jobEngineConfig;
-
-    private static final DefaultScheduler INSTANCE = new DefaultScheduler();
-
-    private DefaultScheduler() {
-    }
-
-    private class FetcherRunner implements Runnable {
-
-        @Override
-        synchronized public void run() {
-            // logger.debug("Job Fetcher is running...");
-            Map<String, Executable> runningJobs = context.getRunningJobs();
-            if (runningJobs.size() >= jobEngineConfig.getMaxConcurrentJobLimit()) {
-                logger.warn("There are too many jobs running, Job Fetch will wait until next schedule time");
-                return;
-            }
-
-            int nRunning = 0, nReady = 0, nOthers = 0;
-            for (final String id : executableManager.getAllJobIds()) {
-                if (runningJobs.containsKey(id)) {
-                    // logger.debug("Job id:" + id + " is already running");
-                    nRunning++;
-                    continue;
-                }
-                final Output output = executableManager.getOutput(id);
-                if ((output.getState() != ExecutableState.READY)) {
-                    // logger.debug("Job id:" + id + " not runnable");
-                    nOthers++;
-                    continue;
-                }
-                nReady++;
-                AbstractExecutable executable = executableManager.getJob(id);
-                String jobDesc = executable.toString();
-                logger.info(jobDesc + " prepare to schedule");
-                try {
-                    context.addRunningJob(executable);
-                    jobPool.execute(new JobRunner(executable));
-                    logger.info(jobDesc + " scheduled");
-                } catch (Exception ex) {
-                    context.removeRunningJob(executable);
-                    logger.warn(jobDesc + " fail to schedule", ex);
-                }
-            }
-            logger.info("Job Fetcher: " + nRunning + " running, " + runningJobs.size() + " actual running, " + nReady + " ready, " + nOthers + " others");
-        }
-    }
-
-    private class JobRunner implements Runnable {
-
-        private final AbstractExecutable executable;
-
-        public JobRunner(AbstractExecutable executable) {
-            this.executable = executable;
-        }
-
-        @Override
-        public void run() {
-            try {
-                executable.execute(context);
-                // trigger the next step asap
-                fetcherPool.schedule(fetcher, 0, TimeUnit.SECONDS);
-            } catch (ExecuteException e) {
-                logger.error("ExecuteException job:" + executable.getId(), e);
-            } catch (Exception e) {
-                logger.error("unknown error execute job:" + executable.getId(), e);
-            } finally {
-                context.removeRunningJob(executable);
-            }
-        }
-    }
-
-    public static DefaultScheduler getInstance() {
-        return INSTANCE;
-    }
-
-    @Override
-    public void stateChanged(CuratorFramework client, ConnectionState newState) {
-        if ((newState == ConnectionState.SUSPENDED) || (newState == ConnectionState.LOST)) {
-            try {
-                shutdown();
-            } catch (SchedulerException e) {
-                throw new RuntimeException("failed to shutdown scheduler", e);
-            }
-        }
-    }
-
-    @Override
-    public synchronized void init(JobEngineConfig jobEngineConfig, final JobLock jobLock) throws SchedulerException {
-        if (!initialized) {
-            initialized = true;
-        } else {
-            return;
-        }
-
-        this.jobEngineConfig = jobEngineConfig;
-
-        if (jobLock.lock() == false) {
-            throw new IllegalStateException("Cannot start job scheduler due to lack of job lock");
-        }
-
-        executableManager = ExecutableManager.getInstance(jobEngineConfig.getConfig());
-        //load all executable, set them to a consistent status
-        fetcherPool = Executors.newScheduledThreadPool(1);
-        int corePoolSize = jobEngineConfig.getMaxConcurrentJobLimit();
-        jobPool = new ThreadPoolExecutor(corePoolSize, corePoolSize, Long.MAX_VALUE, TimeUnit.DAYS, new SynchronousQueue<Runnable>());
-        context = new DefaultContext(Maps.<String, Executable> newConcurrentMap(), jobEngineConfig.getConfig());
-
-        for (AbstractExecutable executable : executableManager.getAllExecutables()) {
-            if (executable.getStatus() == ExecutableState.READY) {
-                executableManager.updateJobOutput(executable.getId(), ExecutableState.ERROR, null, "scheduler initializing work to reset job to ERROR status");
-            }
-        }
-        executableManager.updateAllRunningJobsToError();
-
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            public void run() {
-                logger.debug("Closing zk connection");
-                try {
-                    shutdown();
-                    jobLock.unlock();
-                } catch (SchedulerException e) {
-                    logger.error("error shutdown scheduler", e);
-                }
-            }
-        });
-
-        fetcher = new FetcherRunner();
-        fetcherPool.scheduleAtFixedRate(fetcher, 10, ExecutableConstants.DEFAULT_SCHEDULER_INTERVAL_SECONDS, TimeUnit.SECONDS);
-        hasStarted = true;
-    }
-
-    @Override
-    public void shutdown() throws SchedulerException {
-        fetcherPool.shutdown();
-        jobPool.shutdown();
-    }
-
-    @Override
-    public boolean stop(AbstractExecutable executable) throws SchedulerException {
-        if (hasStarted) {
-            return true;
-        } else {
-            //TODO should try to stop this executable
-            return true;
-        }
-    }
-
-    public boolean hasStarted() {
-        return this.hasStarted;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/inmemcubing/AbstractInMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/inmemcubing/AbstractInMemCubeBuilder.java b/job/src/main/java/org/apache/kylin/job/inmemcubing/AbstractInMemCubeBuilder.java
deleted file mode 100644
index 2a72064..0000000
--- a/job/src/main/java/org/apache/kylin/job/inmemcubing/AbstractInMemCubeBuilder.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements. See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License. You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.kylin.job.inmemcubing;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.BlockingQueue;
-
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.GridTable;
-import org.apache.kylin.storage.gridtable.IGTScanner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An interface alike abstract class. Hold common tunable parameters and nothing more.
- */
-abstract public class AbstractInMemCubeBuilder {
-
-    private static Logger logger = LoggerFactory.getLogger(AbstractInMemCubeBuilder.class);
-
-    final protected CubeDesc cubeDesc;
-    final protected Map<TblColRef, Dictionary<?>> dictionaryMap;
-    
-    protected int taskThreadCount = 4;
-    protected int reserveMemoryMB = 100;
-
-    public AbstractInMemCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<?>> dictionaryMap) {
-        if(cubeDesc == null)
-            throw new NullPointerException();
-        if (dictionaryMap == null)
-            throw new IllegalArgumentException("dictionary cannot be null");
-        
-        this.cubeDesc = cubeDesc;
-        this.dictionaryMap = dictionaryMap;
-    }
-    
-    public void setConcurrentThreads(int n) {
-        this.taskThreadCount = n;
-    }
-
-    public void setReserveMemoryMB(int mb) {
-        this.reserveMemoryMB = mb;
-    }
-
-    public Runnable buildAsRunnable(final BlockingQueue<List<String>> input, final ICuboidWriter output) {
-        return new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    build(input, output);
-                } catch (IOException e) {
-                    throw new RuntimeException(e);
-                }
-            }
-        };
-    }
-    
-    abstract public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException;
-
-    protected void outputCuboid(long cuboidId, GridTable gridTable, ICuboidWriter output) throws IOException {
-        long startTime = System.currentTimeMillis();
-        GTScanRequest req = new GTScanRequest(gridTable.getInfo(), null, null, null);
-        IGTScanner scanner = gridTable.scan(req);
-        for (GTRecord record : scanner) {
-            output.write(cuboidId, record);
-        }
-        scanner.close();
-        logger.info("Cuboid " + cuboidId + " output takes " + (System.currentTimeMillis() - startTime) + "ms");
-    }
-    
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/inmemcubing/ConcurrentDiskStore.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/inmemcubing/ConcurrentDiskStore.java b/job/src/main/java/org/apache/kylin/job/inmemcubing/ConcurrentDiskStore.java
deleted file mode 100644
index 1ba7e4d..0000000
--- a/job/src/main/java/org/apache/kylin/job/inmemcubing/ConcurrentDiskStore.java
+++ /dev/null
@@ -1,342 +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.inmemcubing;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.Closeable;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.file.StandardOpenOption;
-import java.util.HashSet;
-import java.util.NoSuchElementException;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTRowBlock;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.IGTStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A disk store that allows concurrent read and exclusive write.
- */
-public class ConcurrentDiskStore implements IGTStore, Closeable {
-
-    private static final Logger logger = LoggerFactory.getLogger(MemDiskStore.class);
-    private static final boolean debug = true;
-
-    private static final int STREAM_BUFFER_SIZE = 8192;
-
-    final private GTInfo info;
-    final private Object lock;
-
-    final private File diskFile;
-    final private boolean delOnClose;
-
-    private Writer activeWriter;
-    private HashSet<Reader> activeReaders = new HashSet<Reader>();
-    private FileChannel writeChannel;
-    private FileChannel readChannel; // sharable across multi-threads
-
-    public ConcurrentDiskStore(GTInfo info) throws IOException {
-        this(info, File.createTempFile("ConcurrentDiskStore", ""), true);
-    }
-
-    public ConcurrentDiskStore(GTInfo info, File diskFile) throws IOException {
-        this(info, diskFile, false);
-    }
-
-    private ConcurrentDiskStore(GTInfo info, File diskFile, boolean delOnClose) throws IOException {
-        this.info = info;
-        this.lock = this;
-        this.diskFile = diskFile;
-        this.delOnClose = delOnClose;
-
-        // in case user forget to call close()
-        if (delOnClose)
-            diskFile.deleteOnExit();
-
-        if (debug)
-            logger.debug(this + " disk file " + diskFile.getAbsolutePath());
-    }
-
-    @Override
-    public GTInfo getInfo() {
-        return info;
-    }
-
-    @Override
-    public IGTStoreWriter rebuild(int shard) throws IOException {
-        return newWriter(0);
-    }
-
-    @Override
-    public IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) throws IOException {
-        throw new IllegalStateException("does not support append yet");
-        //return newWriter(diskFile.length());
-    }
-
-    private IGTStoreWriter newWriter(long startOffset) throws IOException {
-        synchronized (lock) {
-            if (activeWriter != null || !activeReaders.isEmpty())
-                throw new IllegalStateException();
-
-            openWriteChannel(startOffset);
-            activeWriter = new Writer(startOffset);
-            return activeWriter;
-        }
-    }
-
-    private void closeWriter(Writer w) {
-        synchronized (lock) {
-            if (activeWriter != w)
-                throw new IllegalStateException();
-
-            activeWriter = null;
-            closeWriteChannel();
-        }
-    }
-
-    @Override
-    public IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) throws IOException {
-        return newReader();
-    }
-
-    private IGTStoreScanner newReader() throws IOException {
-        synchronized (lock) {
-            if (activeWriter != null)
-                throw new IllegalStateException();
-
-            openReadChannel();
-            Reader r = new Reader(0);
-            activeReaders.add(r);
-            return r;
-        }
-    }
-
-    private void closeReader(Reader r) throws IOException {
-        synchronized (lock) {
-            if (activeReaders.contains(r) == false)
-                throw new IllegalStateException();
-
-            activeReaders.remove(r);
-            if (activeReaders.isEmpty())
-                closeReadChannel();
-        }
-    }
-
-    private class Reader implements IGTStoreScanner {
-        final DataInputStream din;
-        long fileLen;
-        long readOffset;
-
-        GTRowBlock block = GTRowBlock.allocate(info);
-        GTRowBlock next = null;
-
-        Reader(long startOffset) throws IOException {
-            this.fileLen = diskFile.length();
-            this.readOffset = startOffset;
-
-            if (debug)
-                logger.debug(ConcurrentDiskStore.this + " read start @ " + readOffset);
-
-            InputStream in = new InputStream() {
-                byte[] tmp = new byte[1];
-
-                @Override
-                public int read() throws IOException {
-                    int n = read(tmp, 0, 1);
-                    if (n <= 0)
-                        return -1;
-                    else
-                        return (int) tmp[0];
-                }
-
-                @Override
-                public int read(byte[] b, int off, int len) throws IOException {
-                    if (available() <= 0)
-                        return -1;
-
-                    int lenToGo = Math.min(available(), len);
-                    int nRead = 0;
-                    while (lenToGo > 0) {
-                        int n = readChannel.read(ByteBuffer.wrap(b, off, lenToGo), readOffset);
-
-                        lenToGo -= n;
-                        nRead += n;
-                        off += n;
-                        readOffset += n;
-                    }
-                    return nRead;
-                }
-
-                @Override
-                public int available() throws IOException {
-                    return (int) (fileLen - readOffset);
-                }
-            };
-            din = new DataInputStream(new BufferedInputStream(in, STREAM_BUFFER_SIZE));
-        }
-
-        @Override
-        public boolean hasNext() {
-            if (next != null)
-                return true;
-
-            try {
-                if (din.available() > 0) {
-                    block.importFrom(din);
-                    next = block;
-                }
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
-
-            return next != null;
-        }
-
-        @Override
-        public GTRowBlock next() {
-            if (next == null) {
-                hasNext();
-                if (next == null)
-                    throw new NoSuchElementException();
-            }
-            GTRowBlock r = next;
-            next = null;
-            return r;
-        }
-
-        @Override
-        public void remove() {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void close() throws IOException {
-            din.close();
-            closeReader(this);
-
-            if (debug)
-                logger.debug(ConcurrentDiskStore.this + " read end @ " + readOffset);
-        }
-    }
-    
-    private class Writer implements IGTStoreWriter {
-        final DataOutputStream dout;
-        long writeOffset;
-
-        Writer(long startOffset) {
-            this.writeOffset = startOffset;
-            
-            if (debug)
-                logger.debug(ConcurrentDiskStore.this + " write start @ " + writeOffset);
-
-            OutputStream out = new OutputStream() {
-                byte[] tmp = new byte[1];
-
-                @Override
-                public void write(int b) throws IOException {
-                    tmp[0] = (byte) b;
-                    write(tmp, 0, 1);
-                }
-
-                @Override
-                public void write(byte[] bytes, int offset, int length) throws IOException {
-                    while (length > 0) {
-                        int n = writeChannel.write(ByteBuffer.wrap(bytes, offset, length), writeOffset);
-                        offset += n;
-                        length -= n;
-                        writeOffset += n;
-                    }
-                }
-            };
-            dout = new DataOutputStream(new BufferedOutputStream(out, STREAM_BUFFER_SIZE));
-        }
-        
-        @Override
-        public void write(GTRowBlock block) throws IOException {
-            block.export(dout);
-        }
-        
-        @Override
-        public void close() throws IOException {
-            dout.close();
-            closeWriter(this);
-
-            if (debug)
-                logger.debug(ConcurrentDiskStore.this + " write end @ " + writeOffset);
-        }
-    }
-
-    private void openWriteChannel(long startOffset) throws IOException {
-        if (startOffset > 0) { // TODO does not support append yet
-            writeChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.CREATE, StandardOpenOption.APPEND, StandardOpenOption.WRITE);
-        } else {
-            diskFile.delete();
-            writeChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
-        }
-    }
-
-    private void closeWriteChannel() {
-        IOUtils.closeQuietly(writeChannel);
-        writeChannel = null;
-    }
-
-    private void openReadChannel() throws IOException {
-        if (readChannel == null) {
-            readChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.READ);
-        }
-    }
-
-    private void closeReadChannel() throws IOException {
-        IOUtils.closeQuietly(readChannel);
-        readChannel = null;
-    }
-
-    @Override
-    public void close() throws IOException {
-        synchronized (lock) {
-            if (activeWriter != null || !activeReaders.isEmpty())
-                throw new IllegalStateException();
-
-            if (delOnClose) {
-                diskFile.delete();
-            }
-
-            if (debug)
-                logger.debug(this + " closed");
-        }
-    }
-
-    @Override
-    public String toString() {
-        return "ConcurrentDiskStore@" + (info.getTableName() == null ? this.hashCode() : info.getTableName());
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilder.java b/job/src/main/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilder.java
deleted file mode 100644
index ce6541f..0000000
--- a/job/src/main/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilder.java
+++ /dev/null
@@ -1,427 +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.inmemcubing;
-
-import com.google.common.collect.Lists;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.common.util.MemoryBudgetController;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.job.inmemcubing.InMemCubeBuilder.CuboidResult;
-import org.apache.kylin.metadata.measure.MeasureAggregators;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.cube.CuboidToGridTableMapping;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.IGTScanner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.TimeUnit;
-
-/**
- * When base cuboid does not fit in memory, cut the input into multiple splits and merge the split outputs at last.
- */
-public class DoggedCubeBuilder extends AbstractInMemCubeBuilder {
-
-    private static Logger logger = LoggerFactory.getLogger(DoggedCubeBuilder.class);
-
-    private int splitRowThreshold = Integer.MAX_VALUE;
-    private int unitRows = 1000;
-
-    public DoggedCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<?>> dictionaryMap) {
-        super(cubeDesc, dictionaryMap);
-    }
-
-    public void setSplitRowThreshold(int rowThreshold) {
-        this.splitRowThreshold = rowThreshold;
-        this.unitRows = Math.min(unitRows, rowThreshold);
-    }
-
-    @Override
-    public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
-        new BuildOnce().build(input, output);
-    }
-
-    private class BuildOnce {
-
-        public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
-            final List<SplitThread> splits = new ArrayList<SplitThread>();
-            final Merger merger = new Merger();
-
-            long start = System.currentTimeMillis();
-            logger.info("Dogged Cube Build start");
-
-            try {
-                SplitThread last = null;
-                boolean eof = false;
-
-                while (!eof) {
-
-                    if (last != null && shouldCutSplit(splits)) {
-                        cutSplit(last);
-                        last = null;
-                    }
-
-                    checkException(splits);
-
-                    if (last == null) {
-                        last = new SplitThread();
-                        splits.add(last);
-                        last.start();
-                        logger.info("Split #" + splits.size() + " kickoff");
-                    }
-
-                    eof = feedSomeInput(input, last, unitRows);
-                }
-
-                for (SplitThread split : splits) {
-                    split.join();
-                }
-                checkException(splits);
-                logger.info("Dogged Cube Build splits complete, took " + (System.currentTimeMillis() - start) + " ms");
-
-                merger.mergeAndOutput(splits, output);
-
-            } catch (Throwable e) {
-                logger.error("Dogged Cube Build error", e);
-                if (e instanceof Error)
-                    throw (Error) e;
-                else if (e instanceof RuntimeException)
-                    throw (RuntimeException) e;
-                else
-                    throw new IOException(e);
-            } finally {
-                closeGirdTables(splits);
-                logger.info("Dogged Cube Build end, totally took " + (System.currentTimeMillis() - start) + " ms");
-                ensureExit(splits);
-                logger.info("Dogged Cube Build return");
-            }
-        }
-
-        private void closeGirdTables(List<SplitThread> splits) {
-            for (SplitThread split : splits) {
-                if (split.buildResult != null) {
-                    for (CuboidResult r : split.buildResult.values()) {
-                        try {
-                            r.table.close();
-                        } catch (Throwable e) {
-                            logger.error("Error closing grid table " + r.table, e);
-                        }
-                    }
-                }
-            }
-        }
-
-        private void ensureExit(List<SplitThread> splits) throws IOException {
-            try {
-                for (int i = 0; i < splits.size(); i++) {
-                    SplitThread split = splits.get(i);
-                    if (split.isAlive()) {
-                        abort(splits);
-                    }
-                }
-            } catch (Throwable e) {
-                logger.error("Dogged Cube Build error", e);
-            }
-        }
-
-        private void checkException(List<SplitThread> splits) throws IOException {
-            for (int i = 0; i < splits.size(); i++) {
-                SplitThread split = splits.get(i);
-                if (split.exception != null)
-                    abort(splits);
-            }
-        }
-
-        private void abort(List<SplitThread> splits) throws IOException {
-            for (SplitThread split : splits) {
-                split.builder.abort();
-            }
-
-            ArrayList<Throwable> errors = new ArrayList<Throwable>();
-            for (SplitThread split : splits) {
-                try {
-                    split.join();
-                } catch (InterruptedException e) {
-                    errors.add(e);
-                }
-                if (split.exception != null)
-                    errors.add(split.exception);
-            }
-
-            if (errors.isEmpty()) {
-                return;
-            } else if (errors.size() == 1) {
-                Throwable t = errors.get(0);
-                if (t instanceof IOException)
-                    throw (IOException) t;
-                else
-                    throw new IOException(t);
-            } else {
-                for (Throwable t : errors)
-                    logger.error("Exception during in-mem cube build", t);
-                throw new IOException(errors.size() + " exceptions during in-mem cube build, cause set to the first, check log for more", errors.get(0));
-            }
-        }
-
-        private boolean feedSomeInput(BlockingQueue<List<String>> input, SplitThread split, int n) {
-            try {
-                int i = 0;
-                while (i < n) {
-                    List<String> record = input.take();
-                    i++;
-
-                    while (split.inputQueue.offer(record, 1, TimeUnit.SECONDS) == false) {
-                        if (split.exception != null)
-                            return true; // got some error
-                    }
-                    split.inputRowCount++;
-
-                    if (record == null || record.isEmpty()) {
-                        return true;
-                    }
-                }
-                return false;
-
-            } catch (InterruptedException e) {
-                throw new RuntimeException(e);
-            }
-        }
-
-        private void cutSplit(SplitThread last) {
-            try {
-                // signal the end of input
-                while (last.isAlive()) {
-                    if (last.inputQueue.offer(Collections.<String> emptyList())) {
-                        break;
-                    }
-                    Thread.sleep(1000);
-                }
-
-                // wait cuboid build done
-                while (last.isAlive()) {
-                    if (last.builder.isAllCuboidDone()) {
-                        break;
-                    }
-                    Thread.sleep(1000);
-                }
-            } catch (InterruptedException e) {
-                throw new RuntimeException(e);
-            }
-        }
-
-        private boolean shouldCutSplit(List<SplitThread> splits) {
-            int systemAvailMB = MemoryBudgetController.getSystemAvailMB();
-            int nSplit = splits.size();
-            long splitRowCount = nSplit == 0 ? 0 : splits.get(nSplit - 1).inputRowCount;
-
-            logger.debug(splitRowCount + " records went into split #" + nSplit + "; " + systemAvailMB + " MB left, " + reserveMemoryMB + " MB threshold");
-
-            return splitRowCount >= splitRowThreshold || systemAvailMB <= reserveMemoryMB;
-        }
-    }
-
-    private class SplitThread extends Thread {
-        final BlockingQueue<List<String>> inputQueue = new ArrayBlockingQueue<List<String>>(16);
-        final InMemCubeBuilder builder;
-
-        ConcurrentNavigableMap<Long, CuboidResult> buildResult;
-        long inputRowCount = 0;
-        RuntimeException exception;
-
-        public SplitThread() {
-            this.builder = new InMemCubeBuilder(cubeDesc, dictionaryMap);
-            this.builder.setConcurrentThreads(taskThreadCount);
-            this.builder.setReserveMemoryMB(reserveMemoryMB);
-        }
-
-        @Override
-        public void run() {
-            try {
-                buildResult = builder.build(inputQueue);
-            } catch (Exception e) {
-                if (e instanceof RuntimeException)
-                    this.exception = (RuntimeException) e;
-                else
-                    this.exception = new RuntimeException(e);
-            }
-        }
-    }
-
-    private class Merger {
-
-        MeasureAggregators reuseAggrs;
-        Object[] reuseMetricsArray;
-        ByteArray reuseMetricsSpace;
-
-        long lastCuboidColumnCount;
-        ImmutableBitSet lastMetricsColumns;
-
-        Merger() {
-            MeasureDesc[] measures = CuboidToGridTableMapping.getMeasureSequenceOnGridTable(cubeDesc);
-            reuseAggrs = new MeasureAggregators(measures);
-            reuseMetricsArray = new Object[measures.length];
-        }
-
-        public void mergeAndOutput(List<SplitThread> splits, ICuboidWriter output) throws IOException {
-            if (splits.size() == 1) {
-                for (CuboidResult cuboidResult : splits.get(0).buildResult.values()) {
-                    outputCuboid(cuboidResult.cuboidId, cuboidResult.table, output);
-                    cuboidResult.table.close();
-                }
-                return;
-            }
-
-            LinkedList<MergeSlot> open = Lists.newLinkedList();
-            for (SplitThread split : splits) {
-                open.add(new MergeSlot(split));
-            }
-
-            PriorityQueue<MergeSlot> heap = new PriorityQueue<MergeSlot>();
-
-            while (true) {
-                // ready records in open slots and add to heap
-                while (!open.isEmpty()) {
-                    MergeSlot slot = open.removeFirst();
-                    if (slot.fetchNext()) {
-                        heap.add(slot);
-                    }
-                }
-
-                // find the smallest on heap
-                MergeSlot smallest = heap.poll();
-                if (smallest == null)
-                    break;
-                open.add(smallest);
-
-                // merge with slots having the same key
-                if (smallest.isSameKey(heap.peek())) {
-                    Object[] metrics = getMetricsValues(smallest.currentRecord);
-                    reuseAggrs.reset();
-                    reuseAggrs.aggregate(metrics);
-                    do {
-                        MergeSlot slot = heap.poll();
-                        open.add(slot);
-                        metrics = getMetricsValues(slot.currentRecord);
-                        reuseAggrs.aggregate(metrics);
-                    } while (smallest.isSameKey(heap.peek()));
-
-                    reuseAggrs.collectStates(metrics);
-                    setMetricsValues(smallest.currentRecord, metrics);
-                }
-
-                output.write(smallest.currentCuboidId, smallest.currentRecord);
-            }
-        }
-
-        private void setMetricsValues(GTRecord record, Object[] metricsValues) {
-            ImmutableBitSet metrics = getMetricsColumns(record);
-
-            if (reuseMetricsSpace == null) {
-                reuseMetricsSpace = new ByteArray(record.getInfo().getMaxColumnLength(metrics));
-            }
-
-            record.setValues(metrics, reuseMetricsSpace, metricsValues);
-        }
-
-        private Object[] getMetricsValues(GTRecord record) {
-            ImmutableBitSet metrics = getMetricsColumns(record);
-            return record.getValues(metrics, reuseMetricsArray);
-        }
-
-        private ImmutableBitSet getMetricsColumns(GTRecord record) {
-            // metrics columns always come after dimension columns
-            if (lastCuboidColumnCount == record.getInfo().getColumnCount())
-                return lastMetricsColumns;
-
-            int to = record.getInfo().getColumnCount();
-            int from = to - reuseMetricsArray.length;
-            lastCuboidColumnCount = record.getInfo().getColumnCount();
-            lastMetricsColumns = new ImmutableBitSet(from, to);
-            return lastMetricsColumns;
-        }
-    }
-
-    private static class MergeSlot implements Comparable<MergeSlot> {
-
-        final Iterator<CuboidResult> cuboidIterator;
-        IGTScanner scanner;
-        Iterator<GTRecord> recordIterator;
-
-        long currentCuboidId;
-        GTRecord currentRecord;
-
-        public MergeSlot(SplitThread split) {
-            cuboidIterator = split.buildResult.values().iterator();
-        }
-
-        public boolean fetchNext() throws IOException {
-            if (recordIterator == null) {
-                if (cuboidIterator.hasNext()) {
-                    CuboidResult cuboid = cuboidIterator.next();
-                    currentCuboidId = cuboid.cuboidId;
-                    scanner = cuboid.table.scan(new GTScanRequest(cuboid.table.getInfo(), null, null, null));
-                    recordIterator = scanner.iterator();
-                } else {
-                    return false;
-                }
-            }
-
-            if (recordIterator.hasNext()) {
-                currentRecord = recordIterator.next();
-                return true;
-            } else {
-                scanner.close();
-                recordIterator = null;
-                return fetchNext();
-            }
-        }
-
-        @Override
-        public int compareTo(MergeSlot o) {
-            long cuboidComp = this.currentCuboidId - o.currentCuboidId;
-            if (cuboidComp != 0)
-                return cuboidComp < 0 ? -1 : 1;
-
-            // note GTRecord.equals() don't work because the two GTRecord comes from different GridTable
-            ImmutableBitSet pk = this.currentRecord.getInfo().getPrimaryKey();
-            for (int i = 0; i < pk.trueBitCount(); i++) {
-                int c = pk.trueBitAt(i);
-                int comp = this.currentRecord.get(c).compareTo(o.currentRecord.get(c));
-                if (comp != 0)
-                    return comp;
-            }
-            return 0;
-        }
-
-        public boolean isSameKey(MergeSlot o) {
-            if (o == null)
-                return false;
-            else
-                return this.compareTo(o) == 0;
-        }
-
-    };
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/inmemcubing/ICuboidWriter.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/inmemcubing/ICuboidWriter.java b/job/src/main/java/org/apache/kylin/job/inmemcubing/ICuboidWriter.java
deleted file mode 100644
index 91d4a2a..0000000
--- a/job/src/main/java/org/apache/kylin/job/inmemcubing/ICuboidWriter.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements. See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License. You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.kylin.job.inmemcubing;
-
-import org.apache.kylin.storage.gridtable.GTRecord;
-
-import java.io.IOException;
-
-/**
- */
-public interface ICuboidWriter {
-    void write(long cuboidId, GTRecord record) throws IOException;
-}


[03/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionMapperTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionMapperTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionMapperTest.java
new file mode 100644
index 0000000..97fe4ac
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionMapperTest.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase.steps;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.apache.kylin.engine.mr.steps.RangeKeyDistributionMapper;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionMapperTest {
+
+    @SuppressWarnings("rawtypes")
+    MapDriver mapDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    @Before
+    public void setUp() {
+        RangeKeyDistributionMapper mapper = new RangeKeyDistributionMapper();
+        mapDriver = MapDriver.newMapDriver(mapper);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testMapperWithoutHeader() throws IOException {
+
+        Text inputKey1 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey2 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey3 = new Text(new byte[] { 2, 2, 2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey4 = new Text(new byte[] { 3, 3, 3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey5 = new Text(new byte[] { 4, 4, 4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey6 = new Text(new byte[] { 5, 5, 5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey7 = new Text(new byte[] { 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+
+        mapDriver.addInput(inputKey1, new Text("abc"));
+        mapDriver.addInput(inputKey2, new Text("abc"));
+        mapDriver.addInput(inputKey3, new Text("abc"));
+        mapDriver.addInput(inputKey4, new Text("abc"));
+        mapDriver.addInput(inputKey5, new Text("abc"));
+        mapDriver.addInput(inputKey6, new Text("abc"));
+        mapDriver.addInput(inputKey7, new Text("abc"));
+
+        List<Pair<Text, LongWritable>> result = mapDriver.run();
+
+        assertEquals(1, result.size());
+
+        byte[] key1 = result.get(0).getFirst().getBytes();
+        LongWritable value1 = result.get(0).getSecond();
+        assertArrayEquals(new byte[] { 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 }, key1);
+        assertEquals(147, value1.get());
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testMapperWithHeader() throws IOException {
+
+        Text inputKey1 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey2 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0, 0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey3 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 2, 2, 2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey4 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 3, 3, 3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey5 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 4, 4, 4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey6 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 5, 5, 5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey7 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+
+        mapDriver.addInput(inputKey1, new Text("abc"));
+        mapDriver.addInput(inputKey2, new Text("abc"));
+        mapDriver.addInput(inputKey3, new Text("abc"));
+        mapDriver.addInput(inputKey4, new Text("abc"));
+        mapDriver.addInput(inputKey5, new Text("abc"));
+        mapDriver.addInput(inputKey6, new Text("abc"));
+        mapDriver.addInput(inputKey7, new Text("abc"));
+
+        List<Pair<Text, LongWritable>> result = mapDriver.run();
+
+        assertEquals(1, result.size());
+
+        byte[] key1 = result.get(0).getFirst().getBytes();
+        LongWritable value1 = result.get(0).getSecond();
+        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 }, key1);
+        assertEquals(273, value1.get());
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionReducerTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionReducerTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionReducerTest.java
new file mode 100644
index 0000000..cbf0657
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionReducerTest.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.storage.hbase.steps;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
+import org.apache.kylin.engine.mr.steps.RangeKeyDistributionReducer;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionReducerTest {
+
+    ReduceDriver<Text, LongWritable, Text, LongWritable> reduceDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    @Before
+    public void setUp() {
+        RangeKeyDistributionReducer reducer = new RangeKeyDistributionReducer();
+        reduceDriver = ReduceDriver.newReduceDriver(reducer);
+    }
+
+    @Test
+    public void testReducer() throws IOException {
+        // TODO
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/pom.xml
----------------------------------------------------------------------
diff --git a/storage/pom.xml b/storage/pom.xml
index e557363..efc1a4b 100644
--- a/storage/pom.xml
+++ b/storage/pom.xml
@@ -40,11 +40,6 @@
             <artifactId>kylin-storage-hbase</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-        <dependency>
-            <groupId>net.sf.ehcache</groupId>
-            <artifactId>ehcache</artifactId>
-            <version>2.8.1</version>
-        </dependency>
 
         <!-- Env & Test -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/ICachableStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/ICachableStorageQuery.java b/storage/src/main/java/org/apache/kylin/storage/ICachableStorageQuery.java
deleted file mode 100644
index 179202e..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/ICachableStorageQuery.java
+++ /dev/null
@@ -1,33 +0,0 @@
-package org.apache.kylin.storage;
-
-import com.google.common.collect.Range;
-
-/**
- */
-public interface ICachableStorageQuery extends IStorageQuery {
-    /**
-     *
-     * being dynamic => getVolatilePeriod() return not null
-     * being dynamic => partition column of its realization not null
-     *
-     * @return true for static storage like cubes
-     *          false for dynamic storage like II
-     */
-    boolean isDynamic();
-
-    /**
-     * volatile period is the period of time in which the returned data is not stable
-     * e.g. inverted index's last several minutes' data is dynamic as time goes by.
-     * data in this period cannot be cached
-     *
-     * This method should not be called before ITupleIterator.close() is called
-     *
-     * @return null if the underlying storage guarantees the data is static
-     */
-    Range<Long> getVolatilePeriod();
-
-    /**
-     * get the uuid for the realization that serves this query
-     */
-    String getStorageUUID();
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/IStorage.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/IStorage.java b/storage/src/main/java/org/apache/kylin/storage/IStorage.java
deleted file mode 100644
index 89b96e9..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/IStorage.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage;
-
-import org.apache.kylin.metadata.realization.IRealization;
-
-public interface IStorage {
-    
-    public IStorageQuery createStorageQuery(IRealization realization);
-
-    public <I> I adaptToBuildEngine(Class<I> engineInterface);
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java b/storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java
deleted file mode 100644
index f090ebb..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage;
-
-import org.apache.kylin.metadata.realization.SQLDigest;
-import org.apache.kylin.metadata.tuple.ITupleIterator;
-import org.apache.kylin.storage.tuple.TupleInfo;
-
-/**
- * 
- * @author xjiang
- * 
- */
-public interface IStorageQuery {
-
-    ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo);
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/StorageContext.java b/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
deleted file mode 100644
index caa2439..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-import com.google.common.collect.BiMap;
-import com.google.common.collect.HashBiMap;
-
-/**
- * @author xjiang
- */
-public class StorageContext {
-
-    public static final int HARD_THRESHOLD = 4000000;
-    public static final int DEFAULT_THRESHOLD = 1000000;
-
-    public enum OrderEnum {
-        ASCENDING, DESCENDING
-    }
-
-    private String connUrl;
-    private int threshold;
-    private int limit;
-    private boolean hasSort;
-    private List<MeasureDesc> sortMeasures;
-    private List<OrderEnum> sortOrders;
-    private boolean acceptPartialResult;
-
-    private boolean exactAggregation;
-    private boolean enableLimit;
-    private boolean enableCoprocessor;
-
-    private long totalScanCount;
-    private Cuboid cuboid;
-    private boolean partialResultReturned;
-
-    public StorageContext() {
-        this.threshold = DEFAULT_THRESHOLD;
-        this.limit = DEFAULT_THRESHOLD;
-        this.totalScanCount = 0;
-        this.cuboid = null;
-        this.hasSort = false;
-        this.sortOrders = new ArrayList<OrderEnum>();
-        this.sortMeasures = new ArrayList<MeasureDesc>();
-
-        this.exactAggregation = false;
-        this.enableLimit = false;
-        this.enableCoprocessor = false;
-
-        this.acceptPartialResult = false;
-        this.partialResultReturned = false;
-    }
-
-    public String getConnUrl() {
-        return connUrl;
-    }
-
-    public void setConnUrl(String connUrl) {
-        this.connUrl = connUrl;
-    }
-
-    public int getThreshold() {
-        return threshold;
-    }
-
-    public void setThreshold(int t) {
-        threshold = Math.min(t, HARD_THRESHOLD);
-    }
-
-    public int getLimit() {
-        return limit;
-    }
-
-    public void setLimit(int l) {
-        this.limit = l;
-    }
-
-    public void enableLimit() {
-        this.enableLimit = true;
-    }
-
-    public boolean isLimitEnabled() {
-        return this.enableLimit;
-    }
-
-    public void addSort(MeasureDesc measure, OrderEnum order) {
-        if (measure != null) {
-            sortMeasures.add(measure);
-            sortOrders.add(order);
-        }
-    }
-
-    public void markSort() {
-        this.hasSort = true;
-    }
-
-    public boolean hasSort() {
-        return this.hasSort;
-    }
-
-    public void setCuboid(Cuboid c) {
-        cuboid = c;
-    }
-
-    public Cuboid getCuboid() {
-        return cuboid;
-    }
-
-    public long getTotalScanCount() {
-        return totalScanCount;
-    }
-
-    public void setTotalScanCount(long totalScanCount) {
-        this.totalScanCount = totalScanCount;
-    }
-
-    public boolean isAcceptPartialResult() {
-        return acceptPartialResult;
-    }
-
-    public void setAcceptPartialResult(boolean acceptPartialResult) {
-        this.acceptPartialResult = acceptPartialResult;
-    }
-
-    public boolean isPartialResultReturned() {
-        return partialResultReturned;
-    }
-
-    public void setPartialResultReturned(boolean partialResultReturned) {
-        this.partialResultReturned = partialResultReturned;
-    }
-
-    public void setExactAggregation(boolean isExactAggregation) {
-        this.exactAggregation = isExactAggregation;
-    }
-
-    public boolean isExactAggregation() {
-        return this.exactAggregation;
-    }
-
-    public void enableCoprocessor() {
-        this.enableCoprocessor = true;
-    }
-
-    public boolean isCoprocessorEnabled() {
-        return this.enableCoprocessor;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/cube/CubeCodeSystem.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cube/CubeCodeSystem.java b/storage/src/main/java/org/apache/kylin/storage/cube/CubeCodeSystem.java
deleted file mode 100644
index 2e07c24..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/cube/CubeCodeSystem.java
+++ /dev/null
@@ -1,287 +0,0 @@
-package org.apache.kylin.storage.cube;
-
-import java.io.UnsupportedEncodingException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
-import org.apache.kylin.metadata.measure.serializer.DataTypeSerializer;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.IGTCodeSystem;
-import org.apache.kylin.storage.gridtable.IGTComparator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Created by shaoshi on 3/23/15.
- * This implementation uses Dictionary to encode and decode the table; If a column doesn't have dictionary, will check
- * its data type to serialize/deserialize it;
- */
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class CubeCodeSystem implements IGTCodeSystem {
-    private static final Logger logger = LoggerFactory.getLogger(CubeCodeSystem.class);
-
-    // ============================================================================
-
-    private GTInfo info;
-    private Map<Integer, Dictionary> dictionaryMap; // column index ==> dictionary of column
-    private Map<Integer, Integer> fixLenMap; // column index ==> fixed length of column
-    private Map<Integer, Integer> dependentMetricsMap;
-    private IGTComparator comparator;
-    private DataTypeSerializer[] serializers;
-
-    public CubeCodeSystem(Map<Integer, Dictionary> dictionaryMap) {
-        this(dictionaryMap, Collections.<Integer, Integer>emptyMap(), Collections.<Integer, Integer>emptyMap());
-    }
-            
-    public CubeCodeSystem(Map<Integer, Dictionary> dictionaryMap, Map<Integer, Integer> fixLenMap, Map<Integer, Integer> dependentMetricsMap) {
-        this.dictionaryMap = dictionaryMap;
-        this.fixLenMap = fixLenMap;
-        this.dependentMetricsMap = dependentMetricsMap;
-    }
-
-    @Override
-    public void init(GTInfo info) {
-        this.info = info;
-
-        serializers = new DataTypeSerializer[info.getColumnCount()];
-        for (int i = 0; i < info.getColumnCount(); i++) {
-            // dimension with dictionary
-            if (dictionaryMap.get(i) != null) {
-                serializers[i] = new DictionarySerializer(dictionaryMap.get(i));
-            }
-            // dimension of fixed length
-            else if (fixLenMap.get(i) != null) {
-                serializers[i] = new FixLenSerializer(fixLenMap.get(i));
-            }
-            // metrics
-            else {
-                serializers[i] = DataTypeSerializer.create(info.getColumnType(i));
-            }
-        }
-
-        this.comparator = new IGTComparator() {
-            @Override
-            public boolean isNull(ByteArray code) {
-                // all 0xff is null
-                byte[] array = code.array();
-                for (int i = 0, j = code.offset(), n = code.length(); i < n; i++, j++) {
-                    if (array[j] != Dictionary.NULL)
-                        return false;
-                }
-                return true;
-            }
-
-            @Override
-            public int compare(ByteArray code1, ByteArray code2) {
-                return code1.compareTo(code2);
-            }
-        };
-    }
-
-    @Override
-    public IGTComparator getComparator() {
-        return comparator;
-    }
-
-    @Override
-    public int codeLength(int col, ByteBuffer buf) {
-        return serializers[col].peekLength(buf);
-    }
-
-    @Override
-    public int maxCodeLength(int col) {
-        return serializers[col].maxLength();
-    }
-
-    @Override
-    public void encodeColumnValue(int col, Object value, ByteBuffer buf) {
-        encodeColumnValue(col, value, 0, buf);
-    }
-
-    @Override
-    public void encodeColumnValue(int col, Object value, int roundingFlag, ByteBuffer buf) {
-        // this is a bit too complicated, but encoding only happens once at build time, so it is OK
-        DataTypeSerializer serializer = serializers[col];
-        try {
-            if (serializer instanceof DictionarySerializer) {
-                ((DictionarySerializer) serializer).serializeWithRounding(value, roundingFlag, buf);
-            } else {
-                serializer.serialize(value, buf);
-            }
-        } catch (ClassCastException ex) {
-            // try convert string into a correct object type
-            try {
-                if (value instanceof String) {
-                    Object converted = serializer.valueOf((String) value);
-                    if ((converted instanceof String) == false) {
-                        encodeColumnValue(col, converted, roundingFlag, buf);
-                        return;
-                    }
-                }
-            } catch (Throwable e) {
-                logger.error("Fail to encode value '" + value + "'", e);
-            }
-            throw ex;
-        }
-    }
-
-    @Override
-    public Object decodeColumnValue(int col, ByteBuffer buf) {
-       return serializers[col].deserialize(buf);
-    }
-
-    @Override
-    public MeasureAggregator<?>[] newMetricsAggregators(ImmutableBitSet columns, String[] aggrFunctions) {
-        assert columns.trueBitCount() == aggrFunctions.length;
-        
-        MeasureAggregator<?>[] result = new MeasureAggregator[aggrFunctions.length];
-        for (int i = 0; i < result.length; i++) {
-            int col = columns.trueBitAt(i);
-            result[i] = MeasureAggregator.create(aggrFunctions[i], info.getColumnType(col).toString());
-        }
-        
-        // deal with holistic distinct count
-        if (dependentMetricsMap != null) {
-            for (Integer child : dependentMetricsMap.keySet()) {
-                if (columns.get(child)) {
-                    Integer parent = dependentMetricsMap.get(child);
-                    if (columns.get(parent) == false)
-                        throw new IllegalStateException();
-                    
-                    int childIdx = columns.trueBitIndexOf(child);
-                    int parentIdx = columns.trueBitIndexOf(parent);
-                    result[childIdx].setDependentAggregator(result[parentIdx]);
-                }
-            }
-        }
-        
-        return result;
-    }
-
-    static class DictionarySerializer extends DataTypeSerializer {
-        private Dictionary dictionary;
-
-        DictionarySerializer(Dictionary dictionary) {
-            this.dictionary = dictionary;
-        }
-
-        public void serializeWithRounding(Object value, int roundingFlag, ByteBuffer buf) {
-            int id = dictionary.getIdFromValue(value, roundingFlag);
-            BytesUtil.writeUnsigned(id, dictionary.getSizeOfId(), buf);
-        }
-
-        @Override
-        public void serialize(Object value, ByteBuffer buf) {
-            int id = dictionary.getIdFromValue(value);
-            BytesUtil.writeUnsigned(id, dictionary.getSizeOfId(), buf);
-        }
-
-        @Override
-        public Object deserialize(ByteBuffer in) {
-            int id = BytesUtil.readUnsigned(in, dictionary.getSizeOfId());
-            return dictionary.getValueFromId(id);
-        }
-
-        @Override
-        public int peekLength(ByteBuffer in) {
-            return dictionary.getSizeOfId();
-        }
-
-        @Override
-        public int maxLength() {
-            return dictionary.getSizeOfId();
-        }
-
-        @Override
-        public Object valueOf(byte[] value) {
-            throw new UnsupportedOperationException();
-        }
-    }
-
-    static class FixLenSerializer extends DataTypeSerializer {
-
-        // be thread-safe and avoid repeated obj creation
-        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
-
-        private int fixLen;
-
-        FixLenSerializer(int fixLen) {
-            this.fixLen = fixLen;
-        }
-        
-        private byte[] currentBuf() {
-            byte[] buf = current.get();
-            if (buf == null) {
-                buf = new byte[fixLen];
-                current.set(buf);
-            }
-            return buf;
-        }
-
-        @Override
-        public void serialize(Object value, ByteBuffer out) {
-            byte[] buf = currentBuf();
-            if (value == null) {
-                Arrays.fill(buf, Dictionary.NULL);
-                out.put(buf);
-            } else {
-                byte[] bytes = Bytes.toBytes(value.toString());
-                if (bytes.length > fixLen) {
-                    throw new IllegalStateException("Expect at most " + fixLen + " bytes, but got " + bytes.length + ", value string: " + value.toString());
-                }
-                out.put(bytes);
-                for (int i = bytes.length; i < fixLen; i++) {
-                    out.put(RowConstants.ROWKEY_PLACE_HOLDER_BYTE);
-                }
-            }
-        }
-
-        @Override
-        public Object deserialize(ByteBuffer in) {
-            byte[] buf = currentBuf();
-            in.get(buf);
-
-            int tail = fixLen;
-            while (tail > 0 && (buf[tail - 1] == RowConstants.ROWKEY_PLACE_HOLDER_BYTE || buf[tail - 1] == Dictionary.NULL)) {
-                tail--;
-            }
-            
-            if (tail == 0) {
-                return buf[0] == Dictionary.NULL ? null : "";
-            }
-            
-            return Bytes.toString(buf, 0, tail);
-        }
-
-        @Override
-        public int peekLength(ByteBuffer in) {
-            return fixLen;
-        }
-
-        @Override
-        public int maxLength() {
-            return fixLen;
-        }
-
-        @Override
-        public Object valueOf(byte[] value) {
-            try {
-                return new String(value, "UTF-8");
-            } catch (UnsupportedEncodingException e) {
-                // does not happen
-                throw new RuntimeException(e);
-            }
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/cube/CubeGridTable.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cube/CubeGridTable.java b/storage/src/main/java/org/apache/kylin/storage/cube/CubeGridTable.java
deleted file mode 100644
index f792402..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/cube/CubeGridTable.java
+++ /dev/null
@@ -1,70 +0,0 @@
-package org.apache.kylin.storage.cube;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.gridtable.GTInfo;
-
-import com.google.common.collect.Maps;
-
-@SuppressWarnings("rawtypes")
-public class CubeGridTable {
-
-    public static Map<TblColRef, Dictionary<?>> getDimensionToDictionaryMap(CubeSegment cubeSeg, long cuboidId) {
-        CubeDesc cubeDesc = cubeSeg.getCubeDesc();
-        CubeManager cubeMgr = CubeManager.getInstance(cubeSeg.getCubeInstance().getConfig());
-
-        // build a dictionary map
-        Map<TblColRef, Dictionary<?>> dictionaryMap = Maps.newHashMap();
-        List<TblColRef> dimCols = Cuboid.findById(cubeDesc, cuboidId).getColumns();
-        for (TblColRef col : dimCols) {
-            Dictionary<?> dictionary = cubeMgr.getDictionary(cubeSeg, col);
-            if (dictionary != null) {
-                dictionaryMap.put(col, dictionary);
-            }
-        }
-        return dictionaryMap;
-    }
-
-    public static GTInfo newGTInfo(CubeSegment cubeSeg, long cuboidId) {
-        Map<TblColRef, Dictionary<?>> dictionaryMap = getDimensionToDictionaryMap(cubeSeg, cuboidId);
-        return newGTInfo(cubeSeg.getCubeDesc(), cuboidId, dictionaryMap);
-    }
-    
-    public static GTInfo newGTInfo(CubeDesc cubeDesc, long cuboidId, Map<TblColRef, Dictionary<?>> dictionaryMap) {
-        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidId);
-        CuboidToGridTableMapping mapping = new CuboidToGridTableMapping(cuboid);
-        
-        Map<Integer, Dictionary> dictionaryByColIdx = Maps.newHashMap();
-        Map<Integer, Integer> fixLenByColIdx = Maps.newHashMap();
-
-        for (TblColRef dim : cuboid.getColumns()) {
-            int colIndex = mapping.getIndexOf(dim);
-            if (cubeDesc.getRowkey().isUseDictionary(dim)) {
-                Dictionary dict = dictionaryMap.get(dim);
-                dictionaryByColIdx.put(colIndex, dict);
-            } else {
-                int len = cubeDesc.getRowkey().getColumnLength(dim);
-                if (len == 0)
-                    throw new IllegalStateException();
-                
-                fixLenByColIdx.put(colIndex,  len);
-            }
-        }
-
-        GTInfo.Builder builder = GTInfo.builder();
-        builder.setTableName("Cuboid " + cuboidId);
-        builder.setCodeSystem(new CubeCodeSystem(dictionaryByColIdx, fixLenByColIdx, mapping.getDependentMetricsMap()));
-        builder.setColumns(mapping.getDataTypes());
-        builder.setPrimaryKey(mapping.getPrimaryKey());
-        builder.enableColumnBlock(mapping.getColumnBlocks());
-        return builder.build();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java b/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java
index e4b8657..8001fbd 100644
--- a/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java
+++ b/storage/src/main/java/org/apache/kylin/storage/cube/CubeHBaseReadonlyStore.java
@@ -21,12 +21,12 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
 import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
 import org.apache.kylin.cube.model.HBaseMappingDesc;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTRowBlock;
-import org.apache.kylin.storage.gridtable.GTRowBlock.Writer;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.IGTStore;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTRowBlock;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.IGTStore;
+import org.apache.kylin.gridtable.GTRowBlock.Writer;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/cube/CubeScanner.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cube/CubeScanner.java b/storage/src/main/java/org/apache/kylin/storage/cube/CubeScanner.java
index 038df9d..6af83d1 100644
--- a/storage/src/main/java/org/apache/kylin/storage/cube/CubeScanner.java
+++ b/storage/src/main/java/org/apache/kylin/storage/cube/CubeScanner.java
@@ -11,17 +11,19 @@ import java.util.Set;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.gridtable.CubeGridTable;
+import org.apache.kylin.cube.gridtable.CuboidToGridTableMapping;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRawScanner;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTScanRange;
+import org.apache.kylin.gridtable.GTScanRangePlanner;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.GTUtil;
+import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.GTRawScanner;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTScanRange;
-import org.apache.kylin.storage.gridtable.GTScanRangePlanner;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.GTUtil;
-import org.apache.kylin.storage.gridtable.IGTScanner;
 
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/cube/CubeTupleConverter.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cube/CubeTupleConverter.java b/storage/src/main/java/org/apache/kylin/storage/cube/CubeTupleConverter.java
index 7e64975..0657a69 100644
--- a/storage/src/main/java/org/apache/kylin/storage/cube/CubeTupleConverter.java
+++ b/storage/src/main/java/org/apache/kylin/storage/cube/CubeTupleConverter.java
@@ -29,10 +29,10 @@ import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
 import org.apache.kylin.dict.lookup.LookupStringTable;
+import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.gridtable.GTRecord;
 import org.apache.kylin.storage.tuple.Tuple;
 import org.apache.kylin.storage.tuple.TupleInfo;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/cube/CuboidToGridTableMapping.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cube/CuboidToGridTableMapping.java b/storage/src/main/java/org/apache/kylin/storage/cube/CuboidToGridTableMapping.java
deleted file mode 100644
index 0bf4573..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/cube/CuboidToGridTableMapping.java
+++ /dev/null
@@ -1,172 +0,0 @@
-package org.apache.kylin.storage.cube;
-
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import com.google.common.collect.LinkedListMultimap;
-import com.google.common.collect.ListMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-public class CuboidToGridTableMapping {
-
-    final private Cuboid cuboid;
-
-    private List<DataType> gtDataTypes;
-    private List<ImmutableBitSet> gtColBlocks;
-
-    private int nDimensions;
-    private Map<TblColRef, Integer> dim2gt;
-    private ImmutableBitSet gtPrimaryKey;
-
-    private int nMetrics;
-    private ListMultimap<FunctionDesc, Integer> metrics2gt; // because count distinct may have a holistic version
-
-    public CuboidToGridTableMapping(Cuboid cuboid) {
-        this.cuboid = cuboid;
-        init();
-    }
-    
-    private void init() {
-        int gtColIdx = 0;
-        gtDataTypes = Lists.newArrayList();
-        gtColBlocks = Lists.newArrayList();
-
-        // dimensions
-        dim2gt = Maps.newHashMap();
-        BitSet pk = new BitSet();
-        for (TblColRef dimension : cuboid.getColumns()) {
-            gtDataTypes.add(dimension.getType());
-            dim2gt.put(dimension, gtColIdx);
-            pk.set(gtColIdx);
-            gtColIdx++;
-        }
-        gtPrimaryKey = new ImmutableBitSet(pk);
-        gtColBlocks.add(gtPrimaryKey);
-
-        nDimensions = gtColIdx;
-        assert nDimensions == cuboid.getColumns().size();
-
-        // metrics
-        metrics2gt = LinkedListMultimap.create();
-        for (HBaseColumnFamilyDesc familyDesc : cuboid.getCube().getHbaseMapping().getColumnFamily()) {
-            for (HBaseColumnDesc hbaseColDesc : familyDesc.getColumns()) {
-                BitSet colBlock = new BitSet();
-                for (MeasureDesc measure : hbaseColDesc.getMeasures()) {
-                    // Count distinct & holistic count distinct are equals() but different.
-                    // Ensure the holistic version if exists is always the first.
-                    FunctionDesc func = measure.getFunction();
-                    if (func.isHolisticCountDistinct()) {
-                        List<Integer> existing = metrics2gt.removeAll(func);
-                        metrics2gt.put(func, gtColIdx);
-                        metrics2gt.putAll(func, existing);
-                    } else {
-                        metrics2gt.put(func, gtColIdx);
-                    }
-                    gtDataTypes.add(func.getReturnDataType());
-                    colBlock.set(gtColIdx);
-                    gtColIdx++;
-                }
-                gtColBlocks.add(new ImmutableBitSet(colBlock));
-            }
-        }
-        nMetrics = gtColIdx - nDimensions;
-        assert nMetrics == cuboid.getCube().getMeasures().size();
-    }
-
-    public int getColumnCount() {
-        return nDimensions + nMetrics;
-    }
-
-    public int getDimensionCount() {
-        return nDimensions;
-    }
-
-    public int getMetricsCount() {
-        return nMetrics;
-    }
-
-    public DataType[] getDataTypes() {
-        return (DataType[]) gtDataTypes.toArray(new DataType[gtDataTypes.size()]);
-    }
-
-    public ImmutableBitSet getPrimaryKey() {
-        return gtPrimaryKey;
-    }
-
-    public ImmutableBitSet[] getColumnBlocks() {
-        return (ImmutableBitSet[]) gtColBlocks.toArray(new ImmutableBitSet[gtColBlocks.size()]);
-    }
-
-    public int getIndexOf(TblColRef dimension) {
-        Integer i = dim2gt.get(dimension);
-        return i == null ? -1 : i.intValue();
-    }
-
-    public int getIndexOf(FunctionDesc metric) {
-        List<Integer> list = metrics2gt.get(metric);
-        // normal case
-        if (list.size() == 1) {
-            return list.get(0);
-        }
-        // count distinct & its holistic version
-        else if (list.size() == 2) {
-            assert metric.isCountDistinct();
-            return metric.isHolisticCountDistinct() ? list.get(0) : list.get(1);
-        }
-        // unexpected
-        else
-            return -1;
-    }
-    
-    public List<TblColRef> getCuboidDimensionsInGTOrder() {
-        return cuboid.getColumns();
-    }
-    
-    public Map<Integer, Integer> getDependentMetricsMap() {
-        Map<Integer, Integer> result = Maps.newHashMap();
-        List<MeasureDesc> measures = cuboid.getCube().getMeasures();
-        for (MeasureDesc child : measures) {
-            if (child.getDependentMeasureRef() != null) {
-                boolean ok = false;
-                for (MeasureDesc parent : measures) {
-                    if (parent.getName().equals(child.getDependentMeasureRef())) {
-                        int childIndex = getIndexOf(child.getFunction());
-                        int parentIndex = getIndexOf(parent.getFunction());
-                        result.put(childIndex, parentIndex);
-                        ok = true;
-                        break;
-                    }
-                }
-                if (!ok)
-                    throw new IllegalStateException("Cannot find dependent measure: " + child.getDependentMeasureRef());
-            }
-        }
-        return result.isEmpty() ? Collections.<Integer, Integer>emptyMap() : result;
-    }
-
-    public static MeasureDesc[] getMeasureSequenceOnGridTable(CubeDesc cube) {
-        MeasureDesc[] result = new MeasureDesc[cube.getMeasures().size()];
-        int i = 0;
-        for (HBaseColumnFamilyDesc familyDesc : cube.getHbaseMapping().getColumnFamily()) {
-            for (HBaseColumnDesc hbaseColDesc : familyDesc.getColumns()) {
-                for (MeasureDesc m : hbaseColDesc.getMeasures()) {
-                    result[i++] = m;
-                }
-            }
-        }
-        return result;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/cube/SequentialCubeTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cube/SequentialCubeTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/cube/SequentialCubeTupleIterator.java
index 10ff129..366e3c3 100644
--- a/storage/src/main/java/org/apache/kylin/storage/cube/SequentialCubeTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/cube/SequentialCubeTupleIterator.java
@@ -7,11 +7,11 @@ import java.util.NoSuchElementException;
 import java.util.Set;
 
 import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
 import org.apache.kylin.metadata.tuple.ITupleIterator;
-import org.apache.kylin.storage.gridtable.GTRecord;
 import org.apache.kylin.storage.tuple.Tuple;
 import org.apache.kylin.storage.tuple.TupleInfo;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTAggregateScanner.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTAggregateScanner.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTAggregateScanner.java
deleted file mode 100644
index 25b217c..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTAggregateScanner.java
+++ /dev/null
@@ -1,268 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.Map.Entry;
-import java.util.SortedMap;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.measure.HLLCAggregator;
-import org.apache.kylin.metadata.measure.LDCAggregator;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Maps;
-
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class GTAggregateScanner implements IGTScanner {
-
-    @SuppressWarnings("unused")
-    private static final Logger logger = LoggerFactory.getLogger(GTAggregateScanner.class);
-
-    final GTInfo info;
-    final ImmutableBitSet dimensions; // dimensions to return, can be more than group by
-    final ImmutableBitSet groupBy;
-    final ImmutableBitSet metrics;
-    final String[] metricsAggrFuncs;
-    final IGTScanner inputScanner;
-    final AggregationCache aggrCache;
-
-    public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req) {
-        if (req.hasAggregation() == false)
-            throw new IllegalStateException();
-
-        this.info = inputScanner.getInfo();
-        this.dimensions = req.getColumns().andNot(req.getAggrMetrics());
-        this.groupBy = req.getAggrGroupBy();
-        this.metrics = req.getAggrMetrics();
-        this.metricsAggrFuncs = req.getAggrMetricsFuncs();
-        this.inputScanner = inputScanner;
-        this.aggrCache = new AggregationCache();
-    }
-
-    @Override
-    public GTInfo getInfo() {
-        return info;
-    }
-
-    @Override
-    public int getScannedRowCount() {
-        return inputScanner.getScannedRowCount();
-    }
-
-    @Override
-    public int getScannedRowBlockCount() {
-        return inputScanner.getScannedRowBlockCount();
-    }
-
-    @Override
-    public void close() throws IOException {
-        inputScanner.close();
-    }
-
-    @Override
-    public Iterator<GTRecord> iterator() {
-        for (GTRecord r : inputScanner) {
-            aggrCache.aggregate(r);
-        }
-        return aggrCache.iterator();
-    }
-
-    /** return the estimate memory size of aggregation cache */
-    public long getEstimateSizeOfAggrCache() {
-        return aggrCache.esitmateMemSize();
-    }
-
-    public Object[] getTotalSumForSanityCheck() {
-        return aggrCache.calculateTotalSumSanityCheck();
-    }
-
-    class AggregationCache {
-        final SortedMap<byte[], MeasureAggregator[]> aggBufMap;
-        final int keyLength;
-        final boolean[] compareMask;
-
-        public AggregationCache() {
-            compareMask = createCompareMask();
-            keyLength = compareMask.length;
-            aggBufMap = Maps.newTreeMap(new Comparator<byte[]>() {
-                @Override
-                public int compare(byte[] o1, byte[] o2) {
-                    int result = 0;
-                    // profiler shows this check is slow
-                    // Preconditions.checkArgument(keyLength == o1.length && keyLength == o2.length);
-                    for (int i = 0; i < keyLength; ++i) {
-                        if (compareMask[i]) {
-                            int a = (o1[i] & 0xff);
-                            int b = (o2[i] & 0xff);
-                            result = a - b;
-                            if (result == 0) {
-                                continue;
-                            } else {
-                                return result;
-                            }
-                        }
-                    }
-                    return result;
-                }
-            });
-        }
-
-        private boolean[] createCompareMask() {
-            int keyLength = 0;
-            for (int i = 0; i < dimensions.trueBitCount(); i++) {
-                int c = dimensions.trueBitAt(i);
-                int l = info.codeSystem.maxCodeLength(c);
-                keyLength += l;
-            }
-
-            boolean[] mask = new boolean[keyLength];
-            int p = 0;
-            for (int i = 0; i < dimensions.trueBitCount(); i++) {
-                int c = dimensions.trueBitAt(i);
-                int l = info.codeSystem.maxCodeLength(c);
-                boolean m = groupBy.get(c) ? true : false;
-                for (int j = 0; j < l; j++) {
-                    mask[p++] = m;
-                }
-            }
-            return mask;
-        }
-
-        private byte[] createKey(GTRecord record) {
-            byte[] result = new byte[keyLength];
-            int offset = 0;
-            for (int i = 0; i < dimensions.trueBitCount(); i++) {
-                int c = dimensions.trueBitAt(i);
-                final ByteArray byteArray = record.cols[c];
-                final int columnLength = info.codeSystem.maxCodeLength(c);
-                System.arraycopy(byteArray.array(), byteArray.offset(), result, offset, byteArray.length());
-                offset += columnLength;
-            }
-            assert offset == result.length;
-            return result;
-        }
-
-        void aggregate(GTRecord r) {
-            final byte[] key = createKey(r);
-            MeasureAggregator[] aggrs = aggBufMap.get(key);
-            if (aggrs == null) {
-                aggrs = newAggregators();
-                aggBufMap.put(key, aggrs);
-            }
-            for (int i = 0; i < aggrs.length; i++) {
-                int col = metrics.trueBitAt(i);
-                Object metrics = info.codeSystem.decodeColumnValue(col, r.cols[col].asBuffer());
-                aggrs[i].aggregate(metrics);
-            }
-        }
-
-        private MeasureAggregator[] newAggregators() {
-            return info.codeSystem.newMetricsAggregators(metrics, metricsAggrFuncs);
-        }
-
-        public Object[] calculateTotalSumSanityCheck() {
-            MeasureAggregator[] totalSum = newAggregators();
-
-            // skip expensive aggregation
-            for (int i = 0; i < totalSum.length; i++) {
-                if (totalSum[i] instanceof HLLCAggregator || totalSum[i] instanceof LDCAggregator)
-                    totalSum[i] = null;
-            }
-
-            for (MeasureAggregator[] entry : aggBufMap.values()) {
-                for (int i = 0; i < totalSum.length; i++) {
-                    if (totalSum[i] != null)
-                        totalSum[i].aggregate(entry[i].getState());
-                }
-            }
-            Object[] result = new Object[totalSum.length];
-            for (int i = 0; i < totalSum.length; i++) {
-                if (totalSum[i] != null)
-                    result[i] = totalSum[i].getState();
-            }
-            return result;
-        }
-
-        public long esitmateMemSize() {
-            if (aggBufMap.isEmpty())
-                return 0;
-
-            byte[] sampleKey = aggBufMap.firstKey();
-            MeasureAggregator<?>[] sampleValue = aggBufMap.get(sampleKey);
-            return estimateSizeOfAggrCache(sampleKey, sampleValue, aggBufMap.size());
-        }
-
-        public Iterator<GTRecord> iterator() {
-            return new Iterator<GTRecord>() {
-
-                final Iterator<Entry<byte[], MeasureAggregator[]>> it = aggBufMap.entrySet().iterator();
-
-                final ByteBuffer metricsBuf = ByteBuffer.allocate(info.getMaxColumnLength(metrics));
-                final GTRecord secondRecord = new GTRecord(info);
-
-                @Override
-                public boolean hasNext() {
-                    return it.hasNext();
-                }
-
-                @Override
-                public GTRecord next() {
-                    Entry<byte[], MeasureAggregator[]> entry = it.next();
-                    create(entry.getKey(), entry.getValue());
-                    return secondRecord;
-                }
-
-                private void create(byte[] key, MeasureAggregator[] value) {
-                    int offset = 0;
-                    for (int i = 0; i < dimensions.trueBitCount(); i++) {
-                        int c = dimensions.trueBitAt(i);
-                        final int columnLength = info.codeSystem.maxCodeLength(c);
-                        secondRecord.set(c, new ByteArray(key, offset, columnLength));
-                        offset += columnLength;
-                    }
-                    metricsBuf.clear();
-                    for (int i = 0; i < value.length; i++) {
-                        int col = metrics.trueBitAt(i);
-                        int pos = metricsBuf.position();
-                        info.codeSystem.encodeColumnValue(col, value[i].getState(), metricsBuf);
-                        secondRecord.cols[col].set(metricsBuf.array(), pos, metricsBuf.position() - pos);
-                    }
-                }
-
-                @Override
-                public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-            };
-        }
-    }
-
-    public static long estimateSizeOfAggrCache(byte[] keySample, MeasureAggregator<?>[] aggrSample, int size) {
-        // Aggregation cache is basically a tree map. The tree map entry overhead is
-        // - 40 according to http://java-performance.info/memory-consumption-of-java-data-types-2/
-        // - 41~52 according to AggregationCacheMemSizeTest
-        return (estimateSizeOf(keySample) + estimateSizeOf(aggrSample) + 64) * size;
-    }
-
-    public static long estimateSizeOf(MeasureAggregator[] aggrs) {
-        // size of array, AggregationCacheMemSizeTest reports 4 for [0], 12 for [1], 12 for [2], 20 for [3] etc..
-        // Memory alignment to 8 bytes
-        long est = (aggrs.length + 1) / 2 * 8 + 4 + (4 /* extra */);
-        for (MeasureAggregator aggr : aggrs) {
-            if (aggr != null)
-                est += aggr.getMemBytesEstimate();
-        }
-        return est;
-    }
-
-    public static long estimateSizeOf(byte[] bytes) {
-        // AggregationCacheMemSizeTest reports 20 for byte[10] and 20 again for byte[16]
-        // Memory alignment to 8 bytes
-        return (bytes.length + 7) / 8 * 8 + 4 + (4 /* extra */);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTBuilder.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTBuilder.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTBuilder.java
deleted file mode 100644
index 7552ab3..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTBuilder.java
+++ /dev/null
@@ -1,74 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.io.Closeable;
-import java.io.Flushable;
-import java.io.IOException;
-
-import org.apache.kylin.storage.gridtable.IGTStore.IGTStoreWriter;
-
-public class GTBuilder implements Closeable, Flushable {
-
-    @SuppressWarnings("unused")
-    final private GTInfo info;
-    final private IGTStoreWriter storeWriter;
-
-    final private GTRowBlock block;
-    final private GTRowBlock.Writer blockWriter;
-
-    private int writtenRowCount;
-    private int writtenRowBlockCount;
-
-    GTBuilder(GTInfo info, int shard, IGTStore store) throws IOException {
-        this(info, shard, store, false);
-    }
-
-    GTBuilder(GTInfo info, int shard, IGTStore store, boolean append) throws IOException {
-        this.info = info;
-
-        block = GTRowBlock.allocate(info);
-        blockWriter = block.getWriter();
-        if (append) {
-            storeWriter = store.append(shard, blockWriter);
-            if (block.isFull()) {
-                blockWriter.clearForNext();
-            }
-        } else {
-            storeWriter = store.rebuild(shard);
-        }
-    }
-
-    public void write(GTRecord r) throws IOException {
-        blockWriter.append(r);
-        writtenRowCount++;
-
-        if (block.isFull()) {
-            flush();
-        }
-    }
-
-    @Override
-    public void flush() throws IOException {
-        blockWriter.readyForFlush();
-        storeWriter.write(block);
-        writtenRowBlockCount++;
-        if (block.isFull()) {
-            blockWriter.clearForNext();
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (block.isEmpty() == false) {
-            flush();
-        }
-        storeWriter.close();
-    }
-
-    public int getWrittenRowCount() {
-        return writtenRowCount;
-    }
-
-    public int getWrittenRowBlockCount() {
-        return writtenRowBlockCount;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTComboStore.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTComboStore.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTComboStore.java
deleted file mode 100644
index 42b577a..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTComboStore.java
+++ /dev/null
@@ -1,115 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.io.IOException;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.storage.gridtable.diskstore.GTDiskStore;
-import org.apache.kylin.storage.gridtable.memstore.GTSimpleMemStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-public class GTComboStore implements IGTStore {
-
-    private static final Logger logger = LoggerFactory.getLogger(GTComboStore.class);
-
-    private final GTInfo gtInfo;
-
-    private void convert(IGTStore input, IGTStore output) throws IOException {
-        final IGTStoreScanner scanner = input.scan(null, null, null, null);
-        final IGTStoreWriter writer = output.rebuild(-1);
-        while (scanner.hasNext()) {
-            writer.write(scanner.next());
-        }
-    }
-
-    private GTDiskStore gtDiskStore;
-    private GTSimpleMemStore gtSimpleMemStore;
-
-    public GTComboStore(GTInfo gtInfo) {
-        this(gtInfo, true);
-    }
-
-    public GTComboStore(GTInfo gtInfo, boolean useMemStore) {
-        this.gtInfo = gtInfo;
-        if (useMemStore) {
-            this.gtSimpleMemStore = new GTSimpleMemStore(gtInfo);
-        } else {
-            this.gtDiskStore = new GTDiskStore(gtInfo);
-        }
-    }
-    
-    @Override
-    public GTInfo getInfo() {
-        return gtInfo;
-    }
-
-    private IGTStore getCurrent() {
-        if (gtSimpleMemStore != null) {
-            return gtSimpleMemStore;
-        } else {
-            return gtDiskStore;
-        }
-    }
-    
-    public long memoryUsage() {
-        if (gtSimpleMemStore != null) {
-            return gtSimpleMemStore.memoryUsage();
-        } else {
-            return gtDiskStore.memoryUsage();
-        }
-    }
-
-    public void switchToMemStore() {
-        try {
-            if (gtSimpleMemStore == null) {
-                gtSimpleMemStore = new GTSimpleMemStore(gtInfo);
-                convert(gtDiskStore, gtSimpleMemStore);
-                gtDiskStore.drop();
-                gtDiskStore = null;
-            }
-        } catch (IOException e) {
-            logger.error("fail to switch to mem store", e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void switchToDiskStore() {
-        try {
-            if (gtDiskStore == null) {
-                gtDiskStore = new GTDiskStore(gtInfo);
-                convert(gtSimpleMemStore, gtDiskStore);
-                gtSimpleMemStore.drop();
-                gtSimpleMemStore = null;
-            }
-        } catch (IOException e) {
-            logger.error("fail to switch to disk store", e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public IGTStoreWriter rebuild(int shard) throws IOException {
-        return getCurrent().rebuild(shard);
-    }
-
-    @Override
-    public IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) throws IOException {
-        return getCurrent().append(shard, fillLast);
-    }
-
-    @Override
-    public IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) throws IOException {
-        return getCurrent().scan(pkStart, pkEnd, selectedColBlocks, additionalPushDown);
-    }
-
-    public void drop() throws IOException {
-        if (gtSimpleMemStore != null) {
-            gtSimpleMemStore.drop();
-        }
-        if (gtDiskStore != null) {
-            gtDiskStore.drop();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTFilterScanner.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTFilterScanner.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTFilterScanner.java
deleted file mode 100644
index d60e4ba..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTFilterScanner.java
+++ /dev/null
@@ -1,100 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.metadata.filter.IFilterCodeSystem;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.tuple.IEvaluatableTuple;
-
-public class GTFilterScanner implements IGTScanner {
-
-    final private IGTScanner inputScanner;
-    final private TupleFilter filter;
-    final private IEvaluatableTuple oneTuple; // avoid instance creation
-    
-    private GTRecord next = null;
-
-    public GTFilterScanner(IGTScanner inputScanner, GTScanRequest req) throws IOException {
-        this.inputScanner = inputScanner;
-        this.filter = req.getFilterPushDown();
-        this.oneTuple = new IEvaluatableTuple() {
-            @Override
-            public Object getValue(TblColRef col) {
-                return next.get(col.getColumnDesc().getZeroBasedIndex());
-            }
-        };
-
-        if (TupleFilter.isEvaluableRecursively(filter) == false)
-            throw new IllegalArgumentException();
-    }
-
-    @Override
-    public GTInfo getInfo() {
-        return inputScanner.getInfo();
-    }
-
-    @Override
-    public int getScannedRowCount() {
-        return inputScanner.getScannedRowCount();
-    }
-
-    @Override
-    public int getScannedRowBlockCount() {
-        return inputScanner.getScannedRowBlockCount();
-    }
-
-    @Override
-    public void close() throws IOException {
-        inputScanner.close();
-    }
-
-    @Override
-    public Iterator<GTRecord> iterator() {
-        return new Iterator<GTRecord>() {
-            
-            private Iterator<GTRecord> inputIterator = inputScanner.iterator();
-
-            @Override
-            public boolean hasNext() {
-                if (next != null)
-                    return true;
-
-                IFilterCodeSystem<ByteArray> filterCodeSystem = GTUtil.wrap(getInfo().codeSystem.getComparator());
-
-                while (inputIterator.hasNext()) {
-                    next = inputIterator.next();
-                    if (filter != null && filter.evaluate(oneTuple, filterCodeSystem) == false) {
-                        continue;
-                    }
-                    return true;
-                }
-                next = null;
-                return false;
-            }
-
-            @Override
-            public GTRecord next() {
-                // fetch next record
-                if (next == null) {
-                    hasNext();
-                    if (next == null)
-                        throw new NoSuchElementException();
-                }
-
-                GTRecord result = next;
-                next = null;
-                return result;
-            }
-
-            @Override
-            public void remove() {
-                throw new UnsupportedOperationException();
-            }
-
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInfo.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInfo.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInfo.java
deleted file mode 100644
index 1c69f8e..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInfo.java
+++ /dev/null
@@ -1,246 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.metadata.model.TblColRef;
-
-public class GTInfo {
-
-    public static Builder builder() {
-        return new Builder();
-    }
-
-    String tableName;
-    IGTCodeSystem codeSystem;
-
-    // column schema
-    int nColumns;
-    DataType[] colTypes;
-    ImmutableBitSet colAll;
-    ImmutableBitSet colPreferIndex;
-    transient TblColRef[] colRefs;
-
-    // grid info
-    ImmutableBitSet primaryKey; // order by, uniqueness is not required
-    ImmutableBitSet[] colBlocks; // primary key must be the first column block
-    ImmutableBitSet colBlocksAll;
-    int rowBlockSize; // 0: disable row block
-
-    // sharding
-    int nShards; // 0: no sharding
-
-    // must create from builder
-    private GTInfo() {
-    }
-
-    public String getTableName() {
-        return tableName;
-    }
-
-    public IGTCodeSystem getCodeSystem() {
-        return codeSystem;
-    }
-
-    public int getColumnCount() {
-        return nColumns;
-    }
-
-    public DataType getColumnType(int i) {
-        return colTypes[i];
-    }
-
-    public ImmutableBitSet getPrimaryKey() {
-        return primaryKey;
-    }
-
-    public boolean isShardingEnabled() {
-        return nShards > 0;
-    }
-
-    public boolean isRowBlockEnabled() {
-        return rowBlockSize > 0;
-    }
-
-    public int getRowBlockSize() {
-        return rowBlockSize;
-    }
-
-    public int getMaxRecordLength() {
-        return getMaxColumnLength(colAll);
-    }
-
-    public int getMaxColumnLength(ImmutableBitSet selectedCols) {
-        int result = 0;
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            result += codeSystem.maxCodeLength(c);
-        }
-        return result;
-    }
-
-    public int getMaxColumnLength() {
-        int max = 0;
-        for (int i = 0; i < nColumns; i++)
-            max = Math.max(max, codeSystem.maxCodeLength(i));
-        return max;
-    }
-
-    public ImmutableBitSet selectColumnBlocks(ImmutableBitSet columns) {
-        if (columns == null)
-            columns = colAll;
-
-        BitSet result = new BitSet();
-        for (int i = 0; i < colBlocks.length; i++) {
-            ImmutableBitSet cb = colBlocks[i];
-            if (cb.intersects(columns)) {
-                result.set(i);
-            }
-        }
-        return new ImmutableBitSet(result);
-    }
-
-    public TblColRef colRef(int i) {
-        if (colRefs == null) {
-            colRefs = new TblColRef[nColumns];
-        }
-        if (colRefs[i] == null) {
-            colRefs[i] = GTUtil.tblColRef(i, colTypes[i].toString());
-        }
-        return colRefs[i];
-    }
-
-    public void validateColRef(TblColRef ref) {
-        TblColRef expected = colRef(ref.getColumnDesc().getZeroBasedIndex());
-        if (expected.equals(ref) == false)
-            throw new IllegalArgumentException();
-    }
-
-    void validate() {
-
-        if (codeSystem == null)
-            throw new IllegalStateException();
-
-        if (primaryKey.cardinality() == 0)
-            throw new IllegalStateException();
-
-        codeSystem.init(this);
-
-        validateColumnBlocks();
-    }
-
-    private void validateColumnBlocks() {
-        colAll = new ImmutableBitSet(0, nColumns);
-
-        if (colBlocks == null) {
-            colBlocks = new ImmutableBitSet[2];
-            colBlocks[0] = primaryKey;
-            colBlocks[1] = colAll.andNot(primaryKey);
-        }
-
-        colBlocksAll = new ImmutableBitSet(0, colBlocks.length);
-
-        if (colPreferIndex == null)
-            colPreferIndex = ImmutableBitSet.EMPTY;
-
-        // column blocks must not overlap
-        for (int i = 0; i < colBlocks.length; i++) {
-            for (int j = i + 1; j < colBlocks.length; j++) {
-                if (colBlocks[i].intersects(colBlocks[j]))
-                    throw new IllegalStateException();
-            }
-        }
-
-        // column block must cover all columns
-        ImmutableBitSet merge = ImmutableBitSet.EMPTY;
-        for (int i = 0; i < colBlocks.length; i++) {
-            merge = merge.or(colBlocks[i]);
-        }
-        if (merge.equals(colAll) == false)
-            throw new IllegalStateException();
-
-        // primary key must be the first column block
-        if (primaryKey.equals(colBlocks[0]) == false)
-            throw new IllegalStateException();
-
-        // drop empty column block
-        LinkedList<ImmutableBitSet> list = new LinkedList<ImmutableBitSet>(Arrays.asList(colBlocks));
-        Iterator<ImmutableBitSet> it = list.iterator();
-        while (it.hasNext()) {
-            ImmutableBitSet cb = it.next();
-            if (cb.isEmpty())
-                it.remove();
-        }
-        colBlocks = (ImmutableBitSet[]) list.toArray(new ImmutableBitSet[list.size()]);
-    }
-
-    public static class Builder {
-        final GTInfo info;
-
-        private Builder() {
-            this.info = new GTInfo();
-        }
-
-        /** optional */
-        public Builder setTableName(String name) {
-            info.tableName = name;
-            return this;
-        }
-
-        /** required */
-        public Builder setCodeSystem(IGTCodeSystem cs) {
-            info.codeSystem = cs;
-            return this;
-        }
-
-        /** required */
-        public Builder setColumns(DataType... colTypes) {
-            info.nColumns = colTypes.length;
-            info.colTypes = colTypes;
-            return this;
-        }
-
-        /** required */
-        public Builder setPrimaryKey(ImmutableBitSet primaryKey) {
-            info.primaryKey = primaryKey;
-            return this;
-        }
-
-        /** optional */
-        public Builder enableColumnBlock(ImmutableBitSet[] columnBlocks) {
-            info.colBlocks = new ImmutableBitSet[columnBlocks.length];
-            for (int i = 0; i < columnBlocks.length; i++) {
-                info.colBlocks[i] = columnBlocks[i];
-            }
-            return this;
-        }
-
-        /** optional */
-        public Builder enableRowBlock(int rowBlockSize) {
-            info.rowBlockSize = rowBlockSize;
-            return this;
-        }
-
-        /** optional */
-        public Builder enableSharding(int nShards) {
-            info.nShards = nShards;
-            return this;
-        }
-
-        /** optional */
-        public Builder setColumnPreferIndex(ImmutableBitSet colPreferIndex) {
-            info.colPreferIndex = colPreferIndex;
-            return this;
-        }
-
-        public GTInfo build() {
-            info.validate();
-            return info;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInvertedIndex.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInvertedIndex.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInvertedIndex.java
deleted file mode 100644
index ceb1463..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInvertedIndex.java
+++ /dev/null
@@ -1,205 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-
-/**
- * A thread-safe inverted index of row blocks in memory.
- * 
- * Note function not() must return all blocks, because index only know what block contains a value,
- * but not sure what block does not contain a value.
- * 
- * @author yangli9
- */
-public class GTInvertedIndex {
-
-    private final GTInfo info;
-    private final ImmutableBitSet colPreferIndex;
-    private final ImmutableBitSet colBlocks;
-    private final GTInvertedIndexOfColumn[] index; // for each column
-
-    private volatile int nIndexedBlocks;
-
-    public GTInvertedIndex(GTInfo info) {
-        this.info = info;
-        this.colPreferIndex = info.colPreferIndex;
-        this.colBlocks = info.selectColumnBlocks(colPreferIndex);
-
-        index = new GTInvertedIndexOfColumn[info.getColumnCount()];
-        for (int i = 0; i < colPreferIndex.trueBitCount(); i++) {
-            int c = colPreferIndex.trueBitAt(i);
-            index[c] = new GTInvertedIndexOfColumn(info.codeSystem.getComparator());
-        }
-    }
-
-    public void add(GTRowBlock block) {
-
-        @SuppressWarnings("unchecked")
-        Set<ByteArray>[] distinctValues = new Set[info.getColumnCount()];
-        for (int i = 0; i < colPreferIndex.trueBitCount(); i++) {
-            int c = colPreferIndex.trueBitAt(i);
-            distinctValues[c] = new HashSet<ByteArray>();
-        }
-
-        GTRowBlock.Reader reader = block.getReader(colBlocks);
-        GTRecord record = new GTRecord(info);
-        while (reader.hasNext()) {
-            reader.fetchNext(record);
-            for (int i = 0; i < colPreferIndex.trueBitCount(); i++) {
-                int c = colPreferIndex.trueBitAt(i);
-                distinctValues[c].add(record.get(c));
-            }
-        }
-
-        for (int i = 0; i < colPreferIndex.trueBitCount(); i++) {
-            int c = colPreferIndex.trueBitAt(i);
-            index[c].add(distinctValues[c], block.getSequenceId());
-        }
-
-        nIndexedBlocks = Math.max(nIndexedBlocks, block.seqId + 1);
-    }
-
-    public ConciseSet filter(TupleFilter filter) {
-        return filter(filter, nIndexedBlocks);
-    }
-
-    public ConciseSet filter(TupleFilter filter, int totalBlocks) {
-        // number of indexed blocks may increase as we do evaluation
-        int indexedBlocks = nIndexedBlocks;
-
-        Evaluator evaluator = new Evaluator(indexedBlocks);
-        ConciseSet r = evaluator.evaluate(filter);
-
-        // add blocks that have not been indexed
-        for (int i = indexedBlocks; i < totalBlocks; i++) {
-            r.add(i);
-        }
-
-        return r;
-    }
-
-    private class Evaluator {
-        private int indexedBlocks;
-
-        Evaluator(int indexedBlocks) {
-            this.indexedBlocks = indexedBlocks;
-        }
-
-        public ConciseSet evaluate(TupleFilter filter) {
-            if (filter == null) {
-                return all();
-            }
-
-            if (filter instanceof LogicalTupleFilter)
-                return evalLogical((LogicalTupleFilter) filter);
-
-            if (filter instanceof CompareTupleFilter)
-                return evalCompare((CompareTupleFilter) filter);
-
-            // unable to evaluate
-            return all();
-        }
-
-        @SuppressWarnings("unchecked")
-        private ConciseSet evalCompare(CompareTupleFilter filter) {
-            int col = col(filter);
-            if (index[col] == null)
-                return all();
-
-            switch (filter.getOperator()) {
-            case ISNULL:
-                return index[col].getNull();
-            case ISNOTNULL:
-                return all();
-            case EQ:
-                return index[col].getEquals((ByteArray) filter.getFirstValue());
-            case NEQ:
-                return all();
-            case IN:
-                return index[col].getIn((Iterable<ByteArray>) filter.getValues());
-            case NOTIN:
-                return all();
-            case LT:
-                return index[col].getRange(null, false, (ByteArray) filter.getFirstValue(), false);
-            case LTE:
-                return index[col].getRange(null, false, (ByteArray) filter.getFirstValue(), true);
-            case GT:
-                return index[col].getRange((ByteArray) filter.getFirstValue(), false, null, false);
-            case GTE:
-                return index[col].getRange((ByteArray) filter.getFirstValue(), true, null, false);
-            default:
-                throw new IllegalStateException("Unsupported operator " + filter.getOperator());
-            }
-        }
-
-        private ConciseSet evalLogical(LogicalTupleFilter filter) {
-            List<? extends TupleFilter> children = filter.getChildren();
-
-            switch (filter.getOperator()) {
-            case AND:
-                return evalLogicalAnd(children);
-            case OR:
-                return evalLogicalOr(children);
-            case NOT:
-                return evalLogicalNot(children);
-            default:
-                throw new IllegalStateException("Unsupported operator " + filter.getOperator());
-            }
-        }
-
-        private ConciseSet evalLogicalAnd(List<? extends TupleFilter> children) {
-            ConciseSet set = all();
-
-            for (TupleFilter c : children) {
-                ConciseSet t = evaluate(c);
-                if (t == null)
-                    continue; // because it's AND
-
-                set.retainAll(t);
-            }
-            return set;
-        }
-
-        private ConciseSet evalLogicalOr(List<? extends TupleFilter> children) {
-            ConciseSet set = new ConciseSet();
-
-            for (TupleFilter c : children) {
-                ConciseSet t = evaluate(c);
-                if (t == null)
-                    return null; // because it's OR
-
-                set.addAll(t);
-            }
-            return set;
-        }
-
-        private ConciseSet evalLogicalNot(List<? extends TupleFilter> children) {
-            return all();
-        }
-
-        private ConciseSet all() {
-            return not(new ConciseSet());
-        }
-
-        private ConciseSet not(ConciseSet set) {
-            set.add(indexedBlocks);
-            set.complement();
-            return set;
-        }
-
-        private int col(CompareTupleFilter filter) {
-            return filter.getColumn().getColumnDesc().getZeroBasedIndex();
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInvertedIndexOfColumn.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInvertedIndexOfColumn.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInvertedIndexOfColumn.java
deleted file mode 100644
index 10a944c..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTInvertedIndexOfColumn.java
+++ /dev/null
@@ -1,115 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-import java.util.NavigableMap;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.kylin.common.util.ByteArray;
-
-import com.google.common.collect.Maps;
-
-public class GTInvertedIndexOfColumn {
-
-    final private IGTComparator comparator;
-    final private ReentrantReadWriteLock rwLock;
-
-    private int nBlocks;
-    private NavigableMap<ByteArray, ConciseSet> rangeIndex;
-    private ConciseSet nullIndex;
-
-    public GTInvertedIndexOfColumn(IGTComparator comparator) {
-        this.comparator = comparator;
-        this.rwLock = new ReentrantReadWriteLock();
-        this.rangeIndex = Maps.newTreeMap(comparator);
-        this.nullIndex = new ConciseSet();
-    }
-
-    public void add(Iterable<ByteArray> codes, int blockId) {
-        rwLock.writeLock().lock();
-        try {
-            for (ByteArray code : codes) {
-                if (comparator.isNull(code)) {
-                    nullIndex.add(blockId);
-                    continue;
-                }
-                ConciseSet set = rangeIndex.get(code);
-                if (set == null) {
-                    set = new ConciseSet();
-                    rangeIndex.put(code.copy(), set);
-                }
-                set.add(blockId);
-            }
-
-            if (blockId >= nBlocks) {
-                nBlocks = blockId + 1;
-            }
-
-        } finally {
-            rwLock.writeLock().unlock();
-        }
-    }
-
-    public ConciseSet getNull() {
-        rwLock.readLock().lock();
-        try {
-            return nullIndex.clone();
-        } finally {
-            rwLock.readLock().unlock();
-        }
-    }
-
-    public ConciseSet getEquals(ByteArray code) {
-        rwLock.readLock().lock();
-        try {
-            ConciseSet set = rangeIndex.get(code);
-            if (set == null)
-                return new ConciseSet();
-            else
-                return set.clone();
-        } finally {
-            rwLock.readLock().unlock();
-        }
-    }
-
-    public ConciseSet getIn(Iterable<ByteArray> codes) {
-        rwLock.readLock().lock();
-        try {
-            ConciseSet r = new ConciseSet();
-            for (ByteArray code : codes) {
-                ConciseSet set = rangeIndex.get(code);
-                if (set != null)
-                    r.addAll(set);
-            }
-            return r;
-        } finally {
-            rwLock.readLock().unlock();
-        }
-    }
-
-    public ConciseSet getRange(ByteArray from, boolean fromInclusive, ByteArray to, boolean toInclusive) {
-        rwLock.readLock().lock();
-        try {
-            ConciseSet r = new ConciseSet();
-            if (from == null && to == null) {
-                r.add(nBlocks);
-                r.complement();
-                return r;
-            }
-            NavigableMap<ByteArray, ConciseSet> subMap;
-            if (from == null) {
-                subMap = rangeIndex.headMap(to, toInclusive);
-            } else if (to == null) {
-                subMap = rangeIndex.tailMap(from, fromInclusive);
-            } else {
-                subMap = rangeIndex.subMap(from, fromInclusive, to, toInclusive);
-            }
-            for (ConciseSet set : subMap.values()) {
-                r.addAll(set);
-            }
-            return r;
-        } finally {
-            rwLock.readLock().unlock();
-        }
-    }
-}


[24/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTRawScanner.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTRawScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRawScanner.java
new file mode 100644
index 0000000..3e34bf4
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRawScanner.java
@@ -0,0 +1,111 @@
+package org.apache.kylin.gridtable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.gridtable.IGTStore.IGTStoreScanner;
+
+public class GTRawScanner implements IGTScanner {
+
+    final GTInfo info;
+    final IGTStoreScanner storeScanner;
+    final ImmutableBitSet selectedColBlocks;
+
+    private GTRowBlock.Reader curBlockReader;
+    private GTRecord next;
+    final private GTRecord oneRecord; // avoid instance creation
+
+    private int scannedRowCount = 0;
+    private int scannedRowBlockCount = 0;
+
+    public GTRawScanner(GTInfo info, IGTStore store, GTScanRequest req) throws IOException {
+        this.info = info;
+        this.selectedColBlocks = info.selectColumnBlocks(req.getColumns());
+        this.storeScanner = store.scan(req.getPkStart(), req.getPkEnd(), selectedColBlocks, req);
+        this.oneRecord = new GTRecord(info);
+    }
+
+    @Override
+    public GTInfo getInfo() {
+        return info;
+    }
+
+    @Override
+    public int getScannedRowCount() {
+        return scannedRowCount;
+    }
+
+    @Override
+    public int getScannedRowBlockCount() {
+        return scannedRowBlockCount;
+    }
+
+    @Override
+    public void close() throws IOException {
+        storeScanner.close();
+    }
+
+    @Override
+    public Iterator<GTRecord> iterator() {
+        return new Iterator<GTRecord>() {
+
+            @Override
+            public boolean hasNext() {
+                if (next != null)
+                    return true;
+
+                if (fetchOneRecord()) {
+                    next = oneRecord;
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            private boolean fetchOneRecord() {
+                while (true) {
+                    // get a block
+                    if (curBlockReader == null) {
+                        if (storeScanner.hasNext()) {
+                            curBlockReader = storeScanner.next().getReader(selectedColBlocks);
+                            scannedRowBlockCount++;
+                        } else {
+                            return false;
+                        }
+                    }
+                    // if block exhausted, try next block
+                    if (curBlockReader.hasNext() == false) {
+                        curBlockReader = null;
+                        continue;
+                    }
+                    // fetch a row
+                    curBlockReader.fetchNext(oneRecord);
+                    scannedRowCount++;
+                    return true;
+                }
+            }
+
+            @Override
+            public GTRecord next() {
+                // fetch next record
+                if (next == null) {
+                    hasNext();
+                    if (next == null)
+                        throw new NoSuchElementException();
+                }
+
+                GTRecord result = next;
+                next = null;
+                return result;
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
new file mode 100644
index 0000000..ef2efe0
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
@@ -0,0 +1,285 @@
+package org.apache.kylin.gridtable;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.BitSet;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.ImmutableBitSet;
+
+public class GTRecord implements Comparable<GTRecord> {
+
+    final GTInfo info;
+    final ByteArray[] cols;
+
+    private ImmutableBitSet maskForEqualHashComp;
+
+    public GTRecord(GTInfo info, ImmutableBitSet maskForEqualHashComp) {
+        this.info = info;
+        this.cols = new ByteArray[info.getColumnCount()];
+        for (int i = 0; i < cols.length; i++) {
+            if (maskForEqualHashComp.get(i)) {
+                this.cols[i] = new ByteArray();
+            }
+        }
+        this.maskForEqualHashComp = maskForEqualHashComp;
+    }
+
+    public GTRecord(GTInfo info) {
+        this(info, info.colAll);
+    }
+
+    public GTInfo getInfo() {
+        return info;
+    }
+
+    public ByteArray get(int i) {
+        return cols[i];
+    }
+
+    public void set(int i, ByteArray data) {
+        cols[i].set(data.array(), data.offset(), data.length());
+    }
+
+    /** set record to the codes of specified values, new space allocated to hold the codes */
+    public GTRecord setValues(Object... values) {
+        setValues(info.colAll, new ByteArray(info.getMaxRecordLength()), values);
+        return this;
+    }
+
+    /** set record to the codes of specified values, reuse given space to hold the codes */
+    public GTRecord setValues(ImmutableBitSet selectedCols, ByteArray space, Object... values) {
+        assert selectedCols.cardinality() == values.length;
+        
+        ByteBuffer buf = space.asBuffer();
+        int pos = buf.position();
+        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
+            int c = selectedCols.trueBitAt(i);
+            info.codeSystem.encodeColumnValue(c, values[i], buf);
+            int newPos = buf.position();
+            cols[c].set(buf.array(), buf.arrayOffset() + pos, newPos - pos);
+            pos = newPos;
+        }
+        return this;
+    }
+
+    /** decode and return the values of this record */
+    public Object[] getValues() {
+        return getValues(info.colAll, new Object[info.getColumnCount()]);
+    }
+
+    /** decode and return the values of this record */
+    public Object[] getValues(ImmutableBitSet selectedCols, Object[] result) {
+        assert selectedCols.cardinality() == result.length;
+        
+        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
+            int c = selectedCols.trueBitAt(i);
+            if (cols[c] == null || cols[c].array() == null) {
+                result[i] = null;
+            } else {
+                result[i] = info.codeSystem.decodeColumnValue(c, cols[c].asBuffer());
+            }
+        }
+        return result;
+    }
+
+    public Object[] getValues(int[] selectedColumns, Object[] result) {
+        assert selectedColumns.length <= result.length;
+        for (int i = 0; i < selectedColumns.length; i++) {
+            int c = selectedColumns[i];
+            if (cols[c].array() == null) {
+                result[i] = null;
+            } else {
+                result[i] = info.codeSystem.decodeColumnValue(c, cols[c].asBuffer());
+            }
+        }
+        return result;
+    }
+
+    public GTRecord copy() {
+        return copy(info.colAll);
+    }
+
+    public GTRecord copy(ImmutableBitSet selectedCols) {
+        int len = 0;
+        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
+            int c = selectedCols.trueBitAt(i);
+            len += cols[c].length();
+        }
+
+        byte[] space = new byte[len];
+
+        GTRecord copy = new GTRecord(info, this.maskForEqualHashComp);
+        int pos = 0;
+        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
+            int c = selectedCols.trueBitAt(i);
+            System.arraycopy(cols[c].array(), cols[c].offset(), space, pos, cols[c].length());
+            copy.cols[c].set(space, pos, cols[c].length());
+            pos += cols[c].length();
+        }
+
+        return copy;
+    }
+
+    public ImmutableBitSet maskForEqualHashComp() {
+        return maskForEqualHashComp;
+    }
+
+    public void maskForEqualHashComp(ImmutableBitSet set) {
+        this.maskForEqualHashComp = set;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+
+        GTRecord o = (GTRecord) obj;
+        if (this.info != o.info)
+            return false;
+        if (this.maskForEqualHashComp != o.maskForEqualHashComp)
+            return false;
+        for (int i = 0; i < maskForEqualHashComp.trueBitCount(); i++) {
+            int c = maskForEqualHashComp.trueBitAt(i);
+            if (this.cols[c].equals(o.cols[c]) == false) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        int hash = 1;
+        for (int i = 0; i < maskForEqualHashComp.trueBitCount(); i++) {
+            int c = maskForEqualHashComp.trueBitAt(i);
+            hash = (31 * hash) + cols[c].hashCode();
+        }
+        return hash;
+    }
+
+    @Override
+    public int compareTo(GTRecord o) {
+        assert this.info == o.info;
+        assert this.maskForEqualHashComp == o.maskForEqualHashComp; // reference equal for performance
+        IGTComparator comparator = info.codeSystem.getComparator();
+
+        int comp = 0;
+        for (int i = 0; i < maskForEqualHashComp.trueBitCount(); i++) {
+            int c = maskForEqualHashComp.trueBitAt(i);
+            comp = comparator.compare(cols[c], o.cols[c]);
+            if (comp != 0)
+                return comp;
+        }
+        return comp;
+    }
+
+    @Override
+    public String toString() {
+        return toString(maskForEqualHashComp);
+    }
+    
+    public String toString(ImmutableBitSet selectedColumns) {
+        Object[] values = new Object[selectedColumns.cardinality()];
+        getValues(selectedColumns, values);
+        return Arrays.toString(values);
+    }
+
+    // ============================================================================
+
+    public ByteArray exportColumns(ImmutableBitSet selectedCols) {
+        int len = 0;
+        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
+            int c = selectedCols.trueBitAt(i);
+            len += cols[c].length();
+        }
+
+        ByteArray buf = ByteArray.allocate(len);
+        exportColumns(info.primaryKey, buf);
+        return buf;
+    }
+
+    /** write data to given buffer, like serialize */
+    public void exportColumns(ImmutableBitSet selectedCols, ByteArray buf) {
+        int pos = 0;
+        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
+            int c = selectedCols.trueBitAt(i);
+            System.arraycopy(cols[c].array(), cols[c].offset(), buf.array(), buf.offset() + pos, cols[c].length());
+            pos += cols[c].length();
+        }
+        buf.setLength(pos);
+    }
+
+    /** write data to given buffer, like serialize */
+    public void exportColumns(ImmutableBitSet selectedCols, ByteBuffer buf) {
+        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
+            int c = selectedCols.trueBitAt(i);
+            buf.put(cols[c].array(), cols[c].offset(), cols[c].length());
+        }
+    }
+
+    public void exportColumns(int[] fieldIndex, ByteBuffer buf) {
+        for (int i : fieldIndex) {
+            buf.put(cols[i].array(), cols[i].offset(), cols[i].length());
+        }
+    }
+
+
+    /** write data to given buffer, like serialize */
+    public void exportColumnBlock(int c, ByteBuffer buf) {
+        exportColumns(info.colBlocks[c], buf);
+    }
+
+    /** change pointers to point to data in given buffer, UNLIKE deserialize */
+    public void loadPrimaryKey(ByteBuffer buf) {
+        loadColumns(info.primaryKey, buf);
+    }
+
+    /** change pointers to point to data in given buffer, UNLIKE deserialize */
+    public void loadCellBlock(int c, ByteBuffer buf) {
+        loadColumns(info.colBlocks[c], buf);
+    }
+
+    /** change pointers to point to data in given buffer, UNLIKE deserialize */
+    public void loadColumns(ImmutableBitSet selectedCols, ByteBuffer buf) {
+        int pos = buf.position();
+        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
+            int c = selectedCols.trueBitAt(i);
+            int len = info.codeSystem.codeLength(c, buf);
+            cols[c].set(buf.array(), buf.arrayOffset() + pos, len);
+            pos += len;
+            buf.position(pos);
+        }
+    }
+
+    /** similar to export(primaryKey) but will stop at the first null value */
+    public static ByteArray exportScanKey(GTRecord rec) {
+        if (rec == null)
+            return null;
+        
+        GTInfo info = rec.getInfo();
+        
+        BitSet selectedColumns = new BitSet();
+        int len = 0;
+        for (int i = 0; i < info.primaryKey.trueBitCount(); i++) {
+            int c = info.primaryKey.trueBitAt(i);
+            if (rec.cols[c].array() == null) {
+                break;
+            }
+            selectedColumns.set(c);
+            len += rec.cols[c].length();
+        }
+        
+        if (selectedColumns.cardinality() == 0)
+            return null;
+
+        ByteArray buf = ByteArray.allocate(len);
+        rec.exportColumns(new ImmutableBitSet(selectedColumns), buf);
+        return buf;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTRowBlock.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTRowBlock.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRowBlock.java
new file mode 100644
index 0000000..2c5111b
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRowBlock.java
@@ -0,0 +1,259 @@
+package org.apache.kylin.gridtable;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.ImmutableBitSet;
+
+public class GTRowBlock {
+
+    /** create a row block, allocate memory, get ready for writing */
+    public static GTRowBlock allocate(GTInfo info) {
+        GTRowBlock b = new GTRowBlock(info);
+
+        byte[] array = new byte[info.getMaxColumnLength(info.primaryKey)];
+        b.primaryKey.set(array);
+
+        int maxRows = info.isRowBlockEnabled() ? info.rowBlockSize : 1;
+        for (int i = 0; i < b.cellBlocks.length; i++) {
+            array = new byte[info.getMaxColumnLength(info.colBlocks[i]) * maxRows];
+            b.cellBlocks[i].set(array);
+        }
+        return b;
+    }
+
+    final GTInfo info;
+
+    int seqId; // 0, 1, 2...
+    int nRows;
+    ByteArray primaryKey; // the primary key of the first (smallest) row
+    ByteArray[] cellBlocks; // cells for each column block
+
+    /** create a row block that has no underlying space */
+    public GTRowBlock(GTInfo info) {
+        this.info = info;
+        this.primaryKey = new ByteArray();
+        this.cellBlocks = new ByteArray[info.colBlocks.length];
+        for (int i = 0; i < this.cellBlocks.length; i++) {
+            this.cellBlocks[i] = new ByteArray();
+        }
+    }
+
+    public int getSequenceId() {
+        return seqId;
+    }
+
+    public ByteArray getPrimaryKey() {
+        return primaryKey;
+    }
+
+    public ByteArray getCellBlock(int i) {
+        return cellBlocks[i];
+    }
+
+    public Writer getWriter() {
+        return new Writer();
+    }
+
+    public class Writer {
+        ByteBuffer[] cellBlockBuffers;
+
+        Writer() {
+            cellBlockBuffers = new ByteBuffer[info.colBlocks.length];
+            for (int i = 0; i < cellBlockBuffers.length; i++) {
+                cellBlockBuffers[i] = cellBlocks[i].asBuffer();
+            }
+        }
+
+        public void copyFrom(GTRowBlock other) {
+            assert info == other.info;
+
+            seqId = other.seqId;
+            nRows = other.nRows;
+            primaryKey.copyFrom(other.primaryKey);
+            for (int i = 0; i < info.colBlocks.length; i++) {
+                cellBlockBuffers[i].clear();
+                cellBlockBuffers[i].put(other.cellBlocks[i].array(), other.cellBlocks[i].offset(), other.cellBlocks[i].length());
+            }
+        }
+
+        public void append(GTRecord r) {
+            // add record to block
+            if (isEmpty()) {
+                r.exportColumns(info.primaryKey, primaryKey);
+            }
+            for (int i = 0; i < info.colBlocks.length; i++) {
+                r.exportColumnBlock(i, cellBlockBuffers[i]);
+            }
+            nRows++;
+        }
+
+        public void readyForFlush() {
+            for (int i = 0; i < cellBlocks.length; i++) {
+                cellBlocks[i].setLength(cellBlockBuffers[i].position());
+            }
+        }
+
+        public void clearForNext() {
+            seqId++;
+            nRows = 0;
+            for (int i = 0; i < cellBlockBuffers.length; i++) {
+                cellBlockBuffers[i].clear();
+            }
+        }
+    }
+
+    public Reader getReader() {
+        return new Reader(info.colBlocksAll);
+    }
+
+    public Reader getReader(ImmutableBitSet selectedColBlocks) {
+        return new Reader(selectedColBlocks);
+    }
+
+    public class Reader {
+        int cur;
+        ByteBuffer primaryKeyBuffer;
+        ByteBuffer[] cellBlockBuffers;
+        ImmutableBitSet selectedColBlocks;
+
+        Reader(ImmutableBitSet selectedColBlocks) {
+            primaryKeyBuffer = primaryKey.asBuffer();
+            cellBlockBuffers = new ByteBuffer[info.colBlocks.length];
+            for (int i = 0; i < cellBlockBuffers.length; i++) {
+                cellBlockBuffers[i] = cellBlocks[i].asBuffer();
+            }
+            this.selectedColBlocks = selectedColBlocks;
+        }
+
+        public boolean hasNext() {
+            return cur < nRows;
+        }
+
+        public void fetchNext(GTRecord result) {
+            if (hasNext() == false)
+                throw new IllegalArgumentException();
+
+            for (int i = 0; i < selectedColBlocks.trueBitCount(); i++) {
+                int c = selectedColBlocks.trueBitAt(i);
+                result.loadCellBlock(c, cellBlockBuffers[c]);
+            }
+            cur++;
+        }
+    }
+
+    public GTRowBlock copy() {
+        GTRowBlock copy = new GTRowBlock(info);
+
+        ByteBuffer buf = ByteBuffer.allocate(this.exportLength());
+        this.export(buf);
+        buf.clear();
+        copy.load(buf);
+
+        return copy;
+    }
+
+    public boolean isEmpty() {
+        return nRows == 0;
+    }
+
+    public boolean isFull() {
+        if (info.isRowBlockEnabled())
+            return nRows >= info.rowBlockSize;
+        else
+            return nRows > 0;
+    }
+
+    public int getNumberOfRows() {
+        return nRows;
+    }
+
+    public void setNumberOfRows(int nRows) {
+        this.nRows = nRows;
+    }
+    
+    // ============================================================================
+
+    public int exportLength() {
+        int len = 4; // seq Id
+        if (info.isRowBlockEnabled())
+            len += 4; // nRows
+        len += 4 + primaryKey.length(); // PK byte array
+        for (ByteArray array : cellBlocks) {
+            len += 4 + array.length(); // cell block byte array
+        }
+        return len;
+    }
+
+    /** write data to given output stream, like serialize */
+    public void export(DataOutputStream out) throws IOException {
+        out.writeInt(seqId);
+        if (info.isRowBlockEnabled())
+            out.writeInt(nRows);
+        export(out, primaryKey);
+        for (ByteArray cb : cellBlocks) {
+            export(out, cb);
+        }
+    }
+
+    private void export(DataOutputStream out, ByteArray array) throws IOException {
+        out.writeInt(array.length());
+        out.write(array.array(), array.offset(), array.length());
+    }
+
+    /** write data to given buffer, like serialize */
+    public void export(ByteBuffer buf) {
+        buf.putInt(seqId);
+        if (info.isRowBlockEnabled())
+            buf.putInt(nRows);
+        export(primaryKey, buf);
+        for (ByteArray cb : cellBlocks) {
+            export(cb, buf);
+        }
+    }
+
+    private void export(ByteArray array, ByteBuffer buf) {
+        buf.putInt(array.length());
+        buf.put(array.array(), array.offset(), array.length());
+    }
+    
+    /** read data from given input stream, like deserialize */
+    public void importFrom(DataInputStream in) throws IOException {
+        seqId = in.readInt();
+        nRows = info.isRowBlockEnabled() ? in.readInt() : 1;
+        importFrom(in, primaryKey);
+        for (int i = 0; i < info.colBlocks.length; i++) {
+            ByteArray cb = cellBlocks[i];
+            importFrom(in, cb);
+        }
+    }
+
+    private void importFrom(DataInputStream in, ByteArray result) throws IOException {
+        byte[] data = result.array();
+        int len = in.readInt();
+        in.read(data, 0, len);
+        result.set(data, 0, len);
+    }
+
+    /** change pointers to point to data in given buffer, UNLIKE deserialize */
+    public void load(ByteBuffer buf) {
+        seqId = buf.getInt();
+        nRows = info.isRowBlockEnabled() ? buf.getInt() : 1;
+        load(primaryKey, buf);
+        for (int i = 0; i < info.colBlocks.length; i++) {
+            ByteArray cb = cellBlocks[i];
+            load(cb, buf);
+        }
+    }
+
+    private void load(ByteArray array, ByteBuffer buf) {
+        int len = buf.getInt();
+        int pos = buf.position();
+        array.set(buf.array(), buf.arrayOffset() + pos, len);
+        buf.position(pos + len);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java
new file mode 100644
index 0000000..e6e00b7
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java
@@ -0,0 +1,101 @@
+package org.apache.kylin.gridtable;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.metadata.measure.MeasureAggregator;
+import org.apache.kylin.metadata.measure.serializer.DataTypeSerializer;
+
+@SuppressWarnings({ "rawtypes", "unchecked" })
+/**
+ * This is just for example and is INCORRECT when numbers are encoded to bytes and compared in filter.
+ * 
+ * A correct implementation must ensure dimension values preserve order after encoded, e.g. by using an
+ * order preserving dictionary.
+ * 
+ * @author yangli9
+ */
+public class GTSampleCodeSystem implements IGTCodeSystem {
+
+    private GTInfo info;
+    private DataTypeSerializer[] serializers;
+    private IGTComparator comparator;
+
+    public GTSampleCodeSystem() {
+    }
+    
+    @Override
+    public void init(GTInfo info) {
+        this.info = info;
+
+        this.serializers = new DataTypeSerializer[info.getColumnCount()];
+        for (int i = 0; i < info.getColumnCount(); i++) {
+            this.serializers[i] = DataTypeSerializer.create(info.colTypes[i]);
+        }
+
+        this.comparator = new IGTComparator() {
+            @Override
+            public boolean isNull(ByteArray code) {
+                // all 0xff is null
+                byte[] array = code.array();
+                for (int i = 0, j = code.offset(), n = code.length(); i < n; i++, j++) {
+                    if (array[j] != (byte) 0xff)
+                        return false;
+                }
+                return true;
+            }
+
+            @Override
+            public int compare(ByteArray code1, ByteArray code2) {
+                return code1.compareTo(code2);
+            }
+        };
+    }
+
+    @Override
+    public int codeLength(int col, ByteBuffer buf) {
+        return serializers[col].peekLength(buf);
+    }
+
+    @Override
+    public int maxCodeLength(int col) {
+        return serializers[col].maxLength();
+    }
+
+    @Override
+    public IGTComparator getComparator() {
+        return comparator;
+    }
+
+    // ============================================================================
+
+    @Override
+    public MeasureAggregator<?>[] newMetricsAggregators(ImmutableBitSet columns, String[] aggrFunctions) {
+        assert columns.trueBitCount() == aggrFunctions.length;
+        
+        MeasureAggregator<?>[] result = new MeasureAggregator[aggrFunctions.length];
+        for (int i = 0; i < result.length; i++) {
+            int col = columns.trueBitAt(i);
+            result[i] = MeasureAggregator.create(aggrFunctions[i], info.getColumnType(col).toString());
+        }
+        return result;
+    }
+
+    @Override
+    public void encodeColumnValue(int col, Object value, ByteBuffer buf) {
+        serializers[col].serialize(value, buf);
+    }
+
+    @Override
+    public void encodeColumnValue(int col, Object value, int roundingFlag, ByteBuffer buf) {
+        // ignore rounding
+        encodeColumnValue(col, value, buf);
+    }
+
+    @Override
+    public Object decodeColumnValue(int col, ByteBuffer buf) {
+        return serializers[col].deserialize(buf);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRange.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRange.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRange.java
new file mode 100644
index 0000000..197fde4
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRange.java
@@ -0,0 +1,83 @@
+package org.apache.kylin.gridtable;
+
+import java.util.Collections;
+import java.util.List;
+
+public class GTScanRange {
+
+    final public GTRecord pkStart; // inclusive, record must not be null, col[pk].array() can be null to mean unbounded
+    final public GTRecord pkEnd; // inclusive, record must not be null, col[pk].array() can be null to mean unbounded
+    final public List<GTRecord> hbaseFuzzyKeys; // partial matching primary keys
+
+    public GTScanRange(GTRecord pkStart, GTRecord pkEnd) {
+        this(pkStart, pkEnd, null);
+    }
+
+    public GTScanRange(GTRecord pkStart, GTRecord pkEnd, List<GTRecord> hbaseFuzzyKeys) {
+        GTInfo info = pkStart.info;
+        assert info == pkEnd.info;
+
+        validateRangeKey(pkStart);
+        validateRangeKey(pkEnd);
+
+        this.pkStart = pkStart;
+        this.pkEnd = pkEnd;
+        this.hbaseFuzzyKeys = hbaseFuzzyKeys == null ? Collections.<GTRecord> emptyList() : hbaseFuzzyKeys;
+    }
+
+    private void validateRangeKey(GTRecord pk) {
+        pk.maskForEqualHashComp(pk.info.primaryKey);
+        boolean afterNull = false;
+        for (int i = 0; i < pk.info.primaryKey.trueBitCount(); i++) {
+            int c = pk.info.primaryKey.trueBitAt(i);
+            if (afterNull) {
+                pk.cols[c].set(null, 0, 0);
+            } else {
+                afterNull = pk.cols[c].array() == null;
+            }
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((hbaseFuzzyKeys == null) ? 0 : hbaseFuzzyKeys.hashCode());
+        result = prime * result + ((pkEnd == null) ? 0 : pkEnd.hashCode());
+        result = prime * result + ((pkStart == null) ? 0 : pkStart.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        GTScanRange other = (GTScanRange) obj;
+        if (hbaseFuzzyKeys == null) {
+            if (other.hbaseFuzzyKeys != null)
+                return false;
+        } else if (!hbaseFuzzyKeys.equals(other.hbaseFuzzyKeys))
+            return false;
+        if (pkEnd == null) {
+            if (other.pkEnd != null)
+                return false;
+        } else if (!pkEnd.equals(other.pkEnd))
+            return false;
+        if (pkStart == null) {
+            if (other.pkStart != null)
+                return false;
+        } else if (!pkStart.equals(other.pkStart))
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return (pkStart == null ? "null" : pkStart.toString(pkStart.info.primaryKey)) //
+                + "-" + (pkEnd == null ? "null" : pkEnd.toString(pkEnd.info.primaryKey));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRangePlanner.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRangePlanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRangePlanner.java
new file mode 100644
index 0000000..c09ecf0
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRangePlanner.java
@@ -0,0 +1,486 @@
+package org.apache.kylin.gridtable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.LogicalTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+public class GTScanRangePlanner {
+
+    private static final int MAX_HBASE_FUZZY_KEYS = 100;
+
+    final private GTInfo info;
+    final private ComparatorEx<ByteArray> byteUnknownIsSmaller;
+    final private ComparatorEx<ByteArray> byteUnknownIsBigger;
+    final private ComparatorEx<GTRecord> recordUnknownIsSmaller;
+    final private ComparatorEx<GTRecord> recordUnknownIsBigger;
+
+    public GTScanRangePlanner(GTInfo info) {
+        this.info = info;
+
+        IGTComparator comp = info.codeSystem.getComparator();
+        this.byteUnknownIsSmaller = byteComparatorTreatsUnknownSmaller(comp);
+        this.byteUnknownIsBigger = byteComparatorTreatsUnknownBigger(comp);
+        this.recordUnknownIsSmaller = recordComparatorTreatsUnknownSmaller(comp);
+        this.recordUnknownIsBigger = recordComparatorTreatsUnknownBigger(comp);
+    }
+
+    // return empty list meaning filter is always false
+    public List<GTScanRange> planScanRanges(TupleFilter filter) {
+        return planScanRanges(filter, Integer.MAX_VALUE);
+    }
+
+    // return empty list meaning filter is always false
+    public List<GTScanRange> planScanRanges(TupleFilter filter, int maxRanges) {
+
+        TupleFilter flatFilter = flattenToOrAndFilter(filter);
+
+        List<Collection<ColumnRange>> orAndDimRanges = translateToOrAndDimRanges(flatFilter);
+
+        List<GTScanRange> scanRanges = Lists.newArrayListWithCapacity(orAndDimRanges.size());
+        for (Collection<ColumnRange> andDimRanges : orAndDimRanges) {
+            GTScanRange scanRange = newScanRange(andDimRanges);
+            scanRanges.add(scanRange);
+        }
+
+        List<GTScanRange> mergedRanges = mergeOverlapRanges(scanRanges);
+        mergedRanges = mergeTooManyRanges(mergedRanges, maxRanges);
+
+        return mergedRanges;
+    }
+
+    private GTScanRange newScanRange(Collection<ColumnRange> andDimRanges) {
+        GTRecord pkStart = new GTRecord(info);
+        GTRecord pkEnd = new GTRecord(info);
+        List<GTRecord> hbaseFuzzyKeys = Lists.newArrayList();
+
+        for (ColumnRange range : andDimRanges) {
+            int col = range.column.getColumnDesc().getZeroBasedIndex();
+            if (info.primaryKey.get(col) == false)
+                continue;
+
+            pkStart.set(col, range.begin);
+            pkEnd.set(col, range.end);
+
+            if (range.equals != null) {
+                ImmutableBitSet fuzzyMask = new ImmutableBitSet(col);
+                for (ByteArray v : range.equals) {
+                    GTRecord fuzzy = new GTRecord(info);
+                    fuzzy.set(col, v);
+                    fuzzy.maskForEqualHashComp(fuzzyMask);
+                    hbaseFuzzyKeys.add(fuzzy);
+                }
+            }
+        }
+
+        return new GTScanRange(pkStart, pkEnd, hbaseFuzzyKeys);
+    }
+
+    private TupleFilter flattenToOrAndFilter(TupleFilter filter) {
+        if (filter == null)
+            return null;
+
+        TupleFilter flatFilter = filter.flatFilter();
+
+        // normalize to OR-AND filter
+        if (flatFilter.getOperator() == FilterOperatorEnum.AND) {
+            LogicalTupleFilter f = new LogicalTupleFilter(FilterOperatorEnum.OR);
+            f.addChild(flatFilter);
+            flatFilter = f;
+        }
+
+        if (flatFilter.getOperator() != FilterOperatorEnum.OR)
+            throw new IllegalStateException();
+
+        return flatFilter;
+    }
+
+    private List<Collection<ColumnRange>> translateToOrAndDimRanges(TupleFilter flatFilter) {
+        List<Collection<ColumnRange>> result = Lists.newArrayList();
+
+        if (flatFilter == null) {
+            result.add(Collections.<ColumnRange> emptyList());
+            return result;
+        }
+
+        for (TupleFilter andFilter : flatFilter.getChildren()) {
+            if (andFilter.getOperator() != FilterOperatorEnum.AND)
+                throw new IllegalStateException("Filter should be AND instead of " + andFilter);
+
+            Collection<ColumnRange> andRanges = translateToAndDimRanges(andFilter.getChildren());
+            result.add(andRanges);
+        }
+
+        return preEvaluateConstantConditions(result);
+    }
+
+    private Collection<ColumnRange> translateToAndDimRanges(List<? extends TupleFilter> andFilters) {
+        Map<TblColRef, ColumnRange> rangeMap = new HashMap<TblColRef, ColumnRange>();
+        for (TupleFilter filter : andFilters) {
+            if ((filter instanceof CompareTupleFilter) == false) {
+                continue;
+            }
+
+            CompareTupleFilter comp = (CompareTupleFilter) filter;
+            if (comp.getColumn() == null) {
+                continue;
+            }
+
+            @SuppressWarnings("unchecked")
+            ColumnRange newRange = new ColumnRange(comp.getColumn(), (Set<ByteArray>) comp.getValues(), comp.getOperator());
+            ColumnRange existing = rangeMap.get(newRange.column);
+            if (existing == null) {
+                rangeMap.put(newRange.column, newRange);
+            } else {
+                existing.andMerge(newRange);
+            }
+        }
+        return rangeMap.values();
+    }
+
+    private List<Collection<ColumnRange>> preEvaluateConstantConditions(List<Collection<ColumnRange>> orAndRanges) {
+        boolean globalAlwaysTrue = false;
+        Iterator<Collection<ColumnRange>> iterator = orAndRanges.iterator();
+        while (iterator.hasNext()) {
+            Collection<ColumnRange> andRanges = iterator.next();
+            Iterator<ColumnRange> iterator2 = andRanges.iterator();
+            boolean hasAlwaysFalse = false;
+            while (iterator2.hasNext()) {
+                ColumnRange range = iterator2.next();
+                if (range.satisfyAll())
+                    iterator2.remove();
+                else if (range.satisfyNone())
+                    hasAlwaysFalse = true;
+            }
+            if (hasAlwaysFalse) {
+                iterator.remove();
+            } else if (andRanges.isEmpty()) {
+                globalAlwaysTrue = true;
+                break;
+            }
+        }
+        // return empty OR list means global false
+        // return an empty AND collection inside OR list means global true
+        if (globalAlwaysTrue) {
+            orAndRanges.clear();
+            orAndRanges.add(Collections.<ColumnRange> emptyList());
+        }
+        return orAndRanges;
+    }
+
+    private List<GTScanRange> mergeOverlapRanges(List<GTScanRange> ranges) {
+        if (ranges.size() <= 1) {
+            return ranges;
+        }
+
+        // sort ranges by start key
+        Collections.sort(ranges, new Comparator<GTScanRange>() {
+            @Override
+            public int compare(GTScanRange a, GTScanRange b) {
+                return recordUnknownIsSmaller.compare(a.pkStart, b.pkStart);
+            }
+        });
+
+        // merge the overlap range
+        List<GTScanRange> mergedRanges = new ArrayList<GTScanRange>();
+        int mergeBeginIndex = 0;
+        GTRecord mergeEnd = ranges.get(0).pkEnd;
+        for (int index = 0; index < ranges.size(); index++) {
+            GTScanRange range = ranges.get(index);
+
+            // if overlap, swallow it
+            if (recordUnknownIsSmaller.min(range.pkStart, mergeEnd) == range.pkStart //
+                    || recordUnknownIsBigger.max(mergeEnd, range.pkStart) == mergeEnd) {
+                mergeEnd = recordUnknownIsBigger.max(mergeEnd, range.pkEnd);
+                continue;
+            }
+
+            // not overlap, split here
+            GTScanRange mergedRange = mergeKeyRange(ranges.subList(mergeBeginIndex, index));
+            mergedRanges.add(mergedRange);
+
+            // start new split
+            mergeBeginIndex = index;
+            mergeEnd = recordUnknownIsBigger.max(mergeEnd, range.pkEnd);
+        }
+
+        // don't miss the last range
+        GTScanRange mergedRange = mergeKeyRange(ranges.subList(mergeBeginIndex, ranges.size()));
+        mergedRanges.add(mergedRange);
+
+        return mergedRanges;
+    }
+
+    private GTScanRange mergeKeyRange(List<GTScanRange> ranges) {
+        GTScanRange first = ranges.get(0);
+        if (ranges.size() == 1)
+            return first;
+
+        GTRecord start = first.pkStart;
+        GTRecord end = first.pkEnd;
+        List<GTRecord> newFuzzyKeys = new ArrayList<GTRecord>();
+
+        boolean hasNonFuzzyRange = false;
+        for (GTScanRange range : ranges) {
+            hasNonFuzzyRange = hasNonFuzzyRange || range.hbaseFuzzyKeys.isEmpty();
+            newFuzzyKeys.addAll(range.hbaseFuzzyKeys);
+            end = recordUnknownIsBigger.max(end, range.pkEnd);
+        }
+
+        // if any range is non-fuzzy, then all fuzzy keys must be cleared
+        // also too many fuzzy keys will slow down HBase scan
+        if (hasNonFuzzyRange || newFuzzyKeys.size() > MAX_HBASE_FUZZY_KEYS) {
+            newFuzzyKeys.clear();
+        }
+
+        return new GTScanRange(start, end, newFuzzyKeys);
+    }
+
+    private List<GTScanRange> mergeTooManyRanges(List<GTScanRange> ranges, int maxRanges) {
+        if (ranges.size() <= maxRanges) {
+            return ranges;
+        }
+
+        // TODO: check the distance between range and merge the large distance range
+        List<GTScanRange> result = new ArrayList<GTScanRange>(1);
+        GTScanRange mergedRange = mergeKeyRange(ranges);
+        result.add(mergedRange);
+        return result;
+    }
+
+    private class ColumnRange {
+        private TblColRef column;
+        private ByteArray begin = ByteArray.EMPTY;
+        private ByteArray end = ByteArray.EMPTY;
+        private Set<ByteArray> equals;
+
+        public ColumnRange(TblColRef column, Set<ByteArray> values, FilterOperatorEnum op) {
+            this.column = column;
+
+            switch (op) {
+            case EQ:
+            case IN:
+                equals = new HashSet<ByteArray>(values);
+                refreshBeginEndFromEquals();
+                break;
+            case LT:
+            case LTE:
+                end = byteUnknownIsBigger.max(values);
+                break;
+            case GT:
+            case GTE:
+                begin = byteUnknownIsSmaller.min(values);
+                break;
+            case NEQ:
+            case NOTIN:
+            case ISNULL:
+            case ISNOTNULL:
+                // let Optiq filter it!
+                break;
+            default:
+                throw new UnsupportedOperationException(op.name());
+            }
+        }
+
+        void copy(TblColRef column, ByteArray beginValue, ByteArray endValue, Set<ByteArray> equalValues) {
+            this.column = column;
+            this.begin = beginValue;
+            this.end = endValue;
+            this.equals = equalValues;
+        }
+
+        private void refreshBeginEndFromEquals() {
+            if (equals.isEmpty()) {
+                begin = ByteArray.EMPTY;
+                end = ByteArray.EMPTY;
+            } else {
+                begin = byteUnknownIsSmaller.min(equals);
+                end = byteUnknownIsBigger.max(equals);
+            }
+        }
+
+        public boolean satisfyAll() {
+            return begin.array() == null && end.array() == null; // the NEQ case
+        }
+
+        public boolean satisfyNone() {
+            if (equals != null) {
+                return equals.isEmpty();
+            } else if (begin.array() != null && end.array() != null) {
+                return info.codeSystem.getComparator().compare(begin, end) > 0;
+            } else {
+                return false;
+            }
+        }
+
+        public void andMerge(ColumnRange another) {
+            assert this.column.equals(another.column);
+
+            if (another.satisfyAll()) {
+                return;
+            }
+
+            if (this.satisfyAll()) {
+                copy(another.column, another.begin, another.end, another.equals);
+                return;
+            }
+
+            if (this.equals != null && another.equals != null) {
+                this.equals.retainAll(another.equals);
+                refreshBeginEndFromEquals();
+                return;
+            }
+
+            if (this.equals != null) {
+                this.equals = filter(this.equals, another.begin, another.end);
+                refreshBeginEndFromEquals();
+                return;
+            }
+
+            if (another.equals != null) {
+                this.equals = filter(another.equals, this.begin, this.end);
+                refreshBeginEndFromEquals();
+                return;
+            }
+
+            this.begin = byteUnknownIsSmaller.max(this.begin, another.begin);
+            this.end = byteUnknownIsBigger.min(this.end, another.end);
+        }
+
+        private Set<ByteArray> filter(Set<ByteArray> equalValues, ByteArray beginValue, ByteArray endValue) {
+            Set<ByteArray> result = Sets.newHashSetWithExpectedSize(equalValues.size());
+            for (ByteArray v : equalValues) {
+                if (byteUnknownIsSmaller.compare(beginValue, v) <= 0 && byteUnknownIsBigger.compare(v, endValue) <= 0) {
+                    result.add(v);
+                }
+            }
+            return equalValues;
+        }
+
+        public String toString() {
+            if (equals == null) {
+                return column.getName() + " between " + begin + " and " + end;
+            } else {
+                return column.getName() + " in " + equals;
+            }
+        }
+    }
+
+    public static abstract class ComparatorEx<T> implements Comparator<T> {
+
+        public T min(Collection<T> v) {
+            if (v.size() <= 0) {
+                return null;
+            }
+
+            Iterator<T> iterator = v.iterator();
+            T min = iterator.next();
+            while (iterator.hasNext()) {
+                min = min(min, iterator.next());
+            }
+            return min;
+        }
+
+        public T max(Collection<T> v) {
+            if (v.size() <= 0) {
+                return null;
+            }
+
+            Iterator<T> iterator = v.iterator();
+            T max = iterator.next();
+            while (iterator.hasNext()) {
+                max = max(max, iterator.next());
+            }
+            return max;
+        }
+
+        public T min(T a, T b) {
+            return compare(a, b) <= 0 ? a : b;
+        }
+
+        public T max(T a, T b) {
+            return compare(a, b) >= 0 ? a : b;
+        }
+
+        public boolean between(T v, T start, T end) {
+            return compare(start, v) <= 0 && compare(v, end) <= 0;
+        }
+    }
+
+    public static ComparatorEx<ByteArray> byteComparatorTreatsUnknownSmaller(final IGTComparator comp) {
+        return new ComparatorEx<ByteArray>() {
+            @Override
+            public int compare(ByteArray a, ByteArray b) {
+                if (a.array() == null)
+                    return -1;
+                else if (b.array() == null)
+                    return 1;
+                else
+                    return comp.compare(a, b);
+            }
+        };
+    }
+
+    public static ComparatorEx<ByteArray> byteComparatorTreatsUnknownBigger(final IGTComparator comp) {
+        return new ComparatorEx<ByteArray>() {
+            @Override
+            public int compare(ByteArray a, ByteArray b) {
+                if (a.array() == null)
+                    return 1;
+                else if (b.array() == null)
+                    return -1;
+                else
+                    return comp.compare(a, b);
+            }
+        };
+    }
+
+    public static ComparatorEx<GTRecord> recordComparatorTreatsUnknownSmaller(IGTComparator comp) {
+        return new RecordComparator(byteComparatorTreatsUnknownSmaller(comp));
+    }
+
+    public static ComparatorEx<GTRecord> recordComparatorTreatsUnknownBigger(IGTComparator comp) {
+        return new RecordComparator(byteComparatorTreatsUnknownBigger(comp));
+    }
+
+    private static class RecordComparator extends ComparatorEx<GTRecord> {
+        final ComparatorEx<ByteArray> comparator;
+
+        RecordComparator(ComparatorEx<ByteArray> byteComparator) {
+            this.comparator = byteComparator;
+        }
+
+        @Override
+        public int compare(GTRecord a, GTRecord b) {
+            assert a.info == b.info;
+            assert a.maskForEqualHashComp() == b.maskForEqualHashComp();
+            ImmutableBitSet mask = a.maskForEqualHashComp();
+
+            int comp = 0;
+            for (int i = 0; i < mask.trueBitCount(); i++) {
+                int c = mask.trueBitAt(i);
+                comp = comparator.compare(a.cols[c], b.cols[c]);
+                if (comp != 0)
+                    return comp;
+            }
+            return 0; // equals
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
new file mode 100644
index 0000000..c5443b6
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
@@ -0,0 +1,155 @@
+package org.apache.kylin.gridtable;
+
+import java.util.Arrays;
+import java.util.Set;
+
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.Sets;
+
+public class GTScanRequest {
+
+    // basic
+    private GTInfo info;
+    private GTScanRange range;
+    private ImmutableBitSet columns;
+
+    // optional filtering
+    private TupleFilter filterPushDown;
+
+    // optional aggregation
+    private ImmutableBitSet aggrGroupBy;
+    private ImmutableBitSet aggrMetrics;
+    private String[] aggrMetricsFuncs;
+
+    public GTScanRequest(GTInfo info) {
+        this(info, null, null, null);
+    }
+
+    public GTScanRequest(GTInfo info, GTScanRange range, ImmutableBitSet columns, TupleFilter filterPushDown) {
+        this.info = info;
+        this.range = range == null ? new GTScanRange(new GTRecord(info), new GTRecord(info)) : range;
+        this.columns = columns;
+        this.filterPushDown = filterPushDown;
+        validate();
+    }
+
+    public GTScanRequest(GTInfo info, GTScanRange range, ImmutableBitSet aggrGroupBy, ImmutableBitSet aggrMetrics, //
+            String[] aggrMetricsFuncs, TupleFilter filterPushDown) {
+        this(info, range, null, aggrGroupBy, aggrMetrics, aggrMetricsFuncs, filterPushDown);
+    }
+
+    public GTScanRequest(GTInfo info, GTScanRange range, ImmutableBitSet dimensions, ImmutableBitSet aggrGroupBy, //
+            ImmutableBitSet aggrMetrics, String[] aggrMetricsFuncs, TupleFilter filterPushDown) {
+        this.info = info;
+        this.range = range == null ? new GTScanRange(new GTRecord(info), new GTRecord(info)) : range;
+        this.columns = dimensions;
+        this.filterPushDown = filterPushDown;
+
+        this.aggrGroupBy = aggrGroupBy;
+        this.aggrMetrics = aggrMetrics;
+        this.aggrMetricsFuncs = aggrMetricsFuncs;
+
+        validate();
+    }
+
+    private void validate() {
+        if (range == null)
+            range = new GTScanRange(null, null);
+
+        if (hasAggregation()) {
+            if (aggrGroupBy.intersects(aggrMetrics))
+                throw new IllegalStateException();
+            if (aggrMetrics.cardinality() != aggrMetricsFuncs.length)
+                throw new IllegalStateException();
+
+            if (columns == null)
+                columns = ImmutableBitSet.EMPTY;
+            columns = columns.or(aggrGroupBy);
+            columns = columns.or(aggrMetrics);
+        }
+
+        if (columns == null)
+            columns = info.colAll;
+
+        if (hasFilterPushDown()) {
+            validateFilterPushDown();
+        }
+    }
+
+    private void validateFilterPushDown() {
+        if (hasFilterPushDown() == false)
+            return;
+
+        Set<TblColRef> filterColumns = Sets.newHashSet();
+        TupleFilter.collectColumns(filterPushDown, filterColumns);
+
+        for (TblColRef col : filterColumns) {
+            // filter columns must belong to the table
+            info.validateColRef(col);
+            // filter columns must be returned to satisfy upper layer evaluation (calcite)
+            columns = columns.set(col.getColumnDesc().getZeroBasedIndex());
+        }
+
+        // un-evaluatable filter must be removed
+        if (TupleFilter.isEvaluableRecursively(filterPushDown) == false) {
+            Set<TblColRef> unevaluableColumns = Sets.newHashSet();
+            filterPushDown = GTUtil.convertFilterUnevaluatable(filterPushDown, info, unevaluableColumns);
+
+            // columns in un-evaluatable filter must be returned without loss so upper layer can do final evaluation
+            if (hasAggregation()) {
+                for (TblColRef col : unevaluableColumns) {
+                    aggrGroupBy = aggrGroupBy.set(col.getColumnDesc().getZeroBasedIndex());
+                }
+            }
+        }
+    }
+
+    public boolean hasFilterPushDown() {
+        return filterPushDown != null;
+    }
+
+    public boolean hasAggregation() {
+        return aggrGroupBy != null && aggrMetrics != null && aggrMetricsFuncs != null;
+    }
+
+    public GTInfo getInfo() {
+        return info;
+    }
+
+    public GTRecord getPkStart() {
+        return range.pkStart;
+    }
+
+    public GTRecord getPkEnd() {
+        return range.pkEnd;
+    }
+
+    public ImmutableBitSet getColumns() {
+        return columns;
+    }
+
+    public TupleFilter getFilterPushDown() {
+        return filterPushDown;
+    }
+
+    public ImmutableBitSet getAggrGroupBy() {
+        return aggrGroupBy;
+    }
+
+    public ImmutableBitSet getAggrMetrics() {
+        return aggrMetrics;
+    }
+
+    public String[] getAggrMetricsFuncs() {
+        return aggrMetricsFuncs;
+    }
+
+    @Override
+    public String toString() {
+        return "GTScanRequest [range=" + range + ", columns=" + columns + ", filterPushDown=" + filterPushDown + ", aggrGroupBy=" + aggrGroupBy + ", aggrMetrics=" + aggrMetrics + ", aggrMetricsFuncs=" + Arrays.toString(aggrMetricsFuncs) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTUtil.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTUtil.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTUtil.java
new file mode 100644
index 0000000..94ac755
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTUtil.java
@@ -0,0 +1,221 @@
+package org.apache.kylin.gridtable;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.filter.ColumnTupleFilter;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.ConstantTupleFilter;
+import org.apache.kylin.metadata.filter.IFilterCodeSystem;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilterSerializer;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.Sets;
+
+public class GTUtil {
+
+    static final TableDesc MOCKUP_TABLE = TableDesc.mockup("GT_MOCKUP_TABLE");
+
+    static TblColRef tblColRef(int col, String datatype) {
+        ColumnDesc desc = ColumnDesc.mockup(MOCKUP_TABLE, col + 1, "" + col, datatype);
+        return new TblColRef(desc);
+    }
+
+    public static TupleFilter convertFilterUnevaluatable(TupleFilter rootFilter, GTInfo info, //
+            Set<TblColRef> unevaluatableColumnCollector) {
+        return convertFilter(rootFilter, info, null, false, unevaluatableColumnCollector);
+    }
+
+    public static TupleFilter convertFilterConstants(TupleFilter rootFilter, GTInfo info) {
+        return convertFilter(rootFilter, info, null, true, null);
+    }
+
+    public static TupleFilter convertFilterColumnsAndConstants(TupleFilter rootFilter, GTInfo info, //
+            List<TblColRef> colMapping, Set<TblColRef> unevaluatableColumnCollector) {
+        return convertFilter(rootFilter, info, colMapping, true, unevaluatableColumnCollector);
+    }
+
+    // converts TblColRef to GridTable column, encode constants, drop unEvaluatable parts
+    private static TupleFilter convertFilter(TupleFilter rootFilter, final GTInfo info, //
+            final List<TblColRef> colMapping, final boolean encodeConstants, //
+            final Set<TblColRef> unevaluatableColumnCollector) {
+
+        IFilterCodeSystem<ByteArray> filterCodeSystem = wrap(info.codeSystem.getComparator());
+        
+        byte[] bytes = TupleFilterSerializer.serialize(rootFilter, new TupleFilterSerializer.Decorator() {
+            @Override
+            public TupleFilter onSerialize(TupleFilter filter) {
+                if (filter == null)
+                    return null;
+
+                // In case of NOT(unEvaluatableFilter), we should immediately replace it as TRUE,
+                // Otherwise, unEvaluatableFilter will later be replace with TRUE and NOT(unEvaluatableFilter)
+                // will always return FALSE.
+                if (filter.getOperator() == TupleFilter.FilterOperatorEnum.NOT && !TupleFilter.isEvaluableRecursively(filter)) {
+                    TupleFilter.collectColumns(filter, unevaluatableColumnCollector);
+                    return ConstantTupleFilter.TRUE;
+                }
+
+                // shortcut for unEvaluatable filter
+                if (filter.isEvaluable() == false) {
+                    TupleFilter.collectColumns(filter, unevaluatableColumnCollector);
+                    return ConstantTupleFilter.TRUE;
+                }
+
+                // map to column onto grid table
+                if (colMapping != null && filter instanceof ColumnTupleFilter) {
+                    ColumnTupleFilter colFilter = (ColumnTupleFilter) filter;
+                    int gtColIdx = colMapping.indexOf(colFilter.getColumn());
+                    return new ColumnTupleFilter(info.colRef(gtColIdx));
+                }
+
+                // encode constants
+                if (encodeConstants && filter instanceof CompareTupleFilter) {
+                    return encodeConstants((CompareTupleFilter) filter);
+                }
+
+                return filter;
+            }
+
+            @SuppressWarnings({ "rawtypes", "unchecked" })
+            private TupleFilter encodeConstants(CompareTupleFilter oldCompareFilter) {
+                // extract ColumnFilter & ConstantFilter
+                TblColRef externalCol = oldCompareFilter.getColumn();
+
+                if (externalCol == null) {
+                    return oldCompareFilter;
+                }
+
+                Collection constValues = oldCompareFilter.getValues();
+                if (constValues == null || constValues.isEmpty()) {
+                    return oldCompareFilter;
+                }
+
+                CompareTupleFilter newCompareFilter = new CompareTupleFilter(oldCompareFilter.getOperator());
+                newCompareFilter.addChild(new ColumnTupleFilter(externalCol));
+
+                Object firstValue = constValues.iterator().next();
+                int col = colMapping == null ? externalCol.getColumnDesc().getZeroBasedIndex() : colMapping.indexOf(externalCol);
+
+                TupleFilter result;
+                ByteArray code;
+
+                // translate constant into code
+                switch (newCompareFilter.getOperator()) {
+                case EQ:
+                case IN:
+                    Set newValues = Sets.newHashSet();
+                    for (Object value : constValues) {
+                        code = translate(col, value, 0);
+                        if (code != null)
+                            newValues.add(code);
+                    }
+                    if (newValues.isEmpty()) {
+                        result = ConstantTupleFilter.FALSE;
+                    } else {
+                        newCompareFilter.addChild(new ConstantTupleFilter(newValues));
+                        result = newCompareFilter;
+                    }
+                    break;
+                case NEQ:
+                    code = translate(col, firstValue, 0);
+                    if (code == null) {
+                        result = ConstantTupleFilter.TRUE;
+                    } else {
+                        newCompareFilter.addChild(new ConstantTupleFilter(code));
+                        result = newCompareFilter;
+                    }
+                    break;
+                case LT:
+                    code = translate(col, firstValue, 1);
+                    if (code == null) {
+                        result = ConstantTupleFilter.TRUE;
+                    } else {
+                        newCompareFilter.addChild(new ConstantTupleFilter(code));
+                        result = newCompareFilter;
+                    }
+                    break;
+                case LTE:
+                    code = translate(col, firstValue, -1);
+                    if (code == null) {
+                        result = ConstantTupleFilter.FALSE;
+                    } else {
+                        newCompareFilter.addChild(new ConstantTupleFilter(code));
+                        result = newCompareFilter;
+                    }
+                    break;
+                case GT:
+                    code = translate(col, firstValue, -1);
+                    if (code == null) {
+                        result = ConstantTupleFilter.TRUE;
+                    } else {
+                        newCompareFilter.addChild(new ConstantTupleFilter(code));
+                        result = newCompareFilter;
+                    }
+                    break;
+                case GTE:
+                    code = translate(col, firstValue, 1);
+                    if (code == null) {
+                        result = ConstantTupleFilter.FALSE;
+                    } else {
+                        newCompareFilter.addChild(new ConstantTupleFilter(code));
+                        result = newCompareFilter;
+                    }
+                    break;
+                default:
+                    throw new IllegalStateException("Cannot handle operator " + newCompareFilter.getOperator());
+                }
+                return result;
+            }
+
+            transient ByteBuffer buf = ByteBuffer.allocate(info.getMaxColumnLength());
+
+            private ByteArray translate(int col, Object value, int roundingFlag) {
+                try {
+                    buf.clear();
+                    info.codeSystem.encodeColumnValue(col, value, roundingFlag, buf);
+                    return ByteArray.copyOf(buf.array(), 0, buf.position());
+                } catch (IllegalArgumentException ex) {
+                    return null;
+                }
+            }
+        }, filterCodeSystem);
+
+        return TupleFilterSerializer.deserialize(bytes, filterCodeSystem);
+    }
+
+    public static IFilterCodeSystem<ByteArray> wrap(final IGTComparator comp) {
+        return new IFilterCodeSystem<ByteArray>() {
+
+            @Override
+            public int compare(ByteArray o1, ByteArray o2) {
+                return comp.compare(o1, o2);
+            }
+
+            @Override
+            public boolean isNull(ByteArray code) {
+                return comp.isNull(code);
+            }
+
+            @Override
+            public void serialize(ByteArray code, ByteBuffer buffer) {
+                if (code == null)
+                    BytesUtil.writeByteArray(null, 0, 0, buffer);
+                else
+                    BytesUtil.writeByteArray(code.array(), code.offset(), code.length(), buffer);
+            }
+
+            @Override
+            public ByteArray deserialize(ByteBuffer buffer) {
+                return new ByteArray(BytesUtil.readByteArray(buffer));
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GridTable.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GridTable.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GridTable.java
new file mode 100644
index 0000000..b8e59f6
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GridTable.java
@@ -0,0 +1,61 @@
+package org.apache.kylin.gridtable;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public class GridTable implements Closeable {
+
+    final GTInfo info;
+    final IGTStore store;
+
+    public GridTable(GTInfo info, IGTStore store) {
+        this.info = info;
+        this.store = store;
+    }
+
+    public GTBuilder rebuild() throws IOException {
+        assert info.isShardingEnabled() == false;
+        return rebuild(-1);
+    }
+
+    public GTBuilder rebuild(int shard) throws IOException {
+        assert shard < info.nShards;
+        return new GTBuilder(info, shard, store);
+    }
+
+    public GTBuilder append() throws IOException {
+        assert info.isShardingEnabled() == false;
+        return append(-1);
+    }
+
+    public GTBuilder append(int shard) throws IOException {
+        return new GTBuilder(info, shard, store, true);
+    }
+
+    public IGTScanner scan(GTScanRequest req) throws IOException {
+        IGTScanner result = new GTRawScanner(info, store, req);
+        
+        if (req.hasFilterPushDown()) {
+            result = new GTFilterScanner(result, req);
+        }
+        if (req.hasAggregation()) {
+            result = new GTAggregateScanner(result, req);
+        }
+        return result;
+    }
+
+    public GTInfo getInfo() {
+        return info;
+    }
+
+    public IGTStore getStore() {
+        return store;
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (store instanceof Closeable) {
+            ((Closeable) store).close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/IGTCodeSystem.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/IGTCodeSystem.java b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTCodeSystem.java
new file mode 100644
index 0000000..bb715b2
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTCodeSystem.java
@@ -0,0 +1,43 @@
+package org.apache.kylin.gridtable;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.metadata.measure.MeasureAggregator;
+
+public interface IGTCodeSystem {
+    
+    void init(GTInfo info);
+
+    IGTComparator getComparator();
+    
+    /** Return the length of code starting at the specified buffer, buffer position must not change after return */
+    int codeLength(int col, ByteBuffer buf);
+    
+    /** Return the max possible length of a column */
+    int maxCodeLength(int col);
+    
+    /**
+     * Encode a value into code.
+     * 
+     * @throws IllegalArgumentException if the value is not in dictionary
+     */
+    void encodeColumnValue(int col, Object value, ByteBuffer buf) throws IllegalArgumentException;
+    
+    /**
+     * Encode a value into code, with option to floor rounding -1, no rounding 0,  or ceiling rounding 1
+     * 
+     * @throws IllegalArgumentException
+     * - if rounding=0 and the value is not in dictionary
+     * - if rounding=-1 and there's no equal or smaller value in dictionary
+     * - if rounding=1 and there's no equal or bigger value in dictionary
+     */
+    void encodeColumnValue(int col, Object value, int roundingFlag, ByteBuffer buf) throws IllegalArgumentException;
+    
+    /** Decode a code into value */
+    Object decodeColumnValue(int col, ByteBuffer buf);
+    
+    /** Return aggregators for metrics */
+    MeasureAggregator<?>[] newMetricsAggregators(ImmutableBitSet columns, String[] aggrFunctions);
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/IGTComparator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/IGTComparator.java b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTComparator.java
new file mode 100644
index 0000000..4edc4db
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTComparator.java
@@ -0,0 +1,15 @@
+package org.apache.kylin.gridtable;
+
+import java.util.Comparator;
+
+import org.apache.kylin.common.util.ByteArray;
+
+public interface IGTComparator extends Comparator<ByteArray> {
+
+    /** if given code represents the NULL value */
+    boolean isNull(ByteArray code);
+
+    /** compare two values by their codes */
+    // int compare(T code1, T code2);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/IGTScanner.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/IGTScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTScanner.java
new file mode 100644
index 0000000..42e2dec
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTScanner.java
@@ -0,0 +1,13 @@
+package org.apache.kylin.gridtable;
+
+import java.io.Closeable;
+
+public interface IGTScanner extends Iterable<GTRecord>, Closeable {
+    
+    GTInfo getInfo();
+    
+    int getScannedRowCount();
+    
+    int getScannedRowBlockCount();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/IGTStore.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/IGTStore.java b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTStore.java
new file mode 100644
index 0000000..70620b3
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTStore.java
@@ -0,0 +1,26 @@
+package org.apache.kylin.gridtable;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.kylin.common.util.ImmutableBitSet;
+
+public interface IGTStore {
+
+    GTInfo getInfo();
+    
+    IGTStoreWriter rebuild(int shard) throws IOException;
+    
+    IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) throws IOException;
+    
+    IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) throws IOException;
+
+    interface IGTStoreWriter extends Closeable {
+        void write(GTRowBlock block) throws IOException;
+    }
+    
+    interface IGTStoreScanner extends Iterator<GTRowBlock>, Closeable {
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java b/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
new file mode 100644
index 0000000..c6af8e5
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
@@ -0,0 +1,101 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.gridtable;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.gridtable.GTInfo.Builder;
+import org.apache.kylin.metadata.measure.LongMutable;
+import org.apache.kylin.metadata.model.DataType;
+
+public class UnitTestSupport {
+
+    public static GTInfo basicInfo() {
+        Builder builder = infoBuilder();
+        GTInfo info = builder.build();
+        return info;
+    }
+
+    public static GTInfo advancedInfo() {
+        Builder builder = infoBuilder();
+        builder.enableColumnBlock(new ImmutableBitSet[] { setOf(0), setOf(1, 2), setOf(3, 4) });
+        builder.enableRowBlock(4);
+        GTInfo info = builder.build();
+        return info;
+    }
+
+    private static Builder infoBuilder() {
+        Builder builder = GTInfo.builder();
+        builder.setCodeSystem(new GTSampleCodeSystem());
+        builder.setColumns( //
+                DataType.getInstance("varchar(10)"), //
+                DataType.getInstance("varchar(10)"), //
+                DataType.getInstance("varchar(10)"), //
+                DataType.getInstance("bigint"), //
+                DataType.getInstance("decimal") //
+        );
+        builder.setPrimaryKey(setOf(0));
+        builder.setColumnPreferIndex(setOf(0));
+        return builder;
+    }
+
+    public static List<GTRecord> mockupData(GTInfo info, int nRows) {
+        List<GTRecord> result = new ArrayList<GTRecord>(nRows);
+        int round = nRows / 10;
+        for (int i = 0; i < round; i++) {
+            String d_01_14 = datePlus("2015-01-14", i * 4);
+            String d_01_15 = datePlus("2015-01-15", i * 4);
+            String d_01_16 = datePlus("2015-01-16", i * 4);
+            String d_01_17 = datePlus("2015-01-17", i * 4);
+            result.add(newRec(info, d_01_14, "Yang", "Food", new LongMutable(10), new BigDecimal("10.5")));
+            result.add(newRec(info, d_01_14, "Luke", "Food", new LongMutable(10), new BigDecimal("10.5")));
+            result.add(newRec(info, d_01_15, "Xu", "Food", new LongMutable(10), new BigDecimal("10.5")));
+            result.add(newRec(info, d_01_15, "Dong", "Food", new LongMutable(10), new BigDecimal("10.5")));
+            result.add(newRec(info, d_01_15, "Jason", "Food", new LongMutable(10), new BigDecimal("10.5")));
+            result.add(newRec(info, d_01_16, "Mahone", "Food", new LongMutable(10), new BigDecimal("10.5")));
+            result.add(newRec(info, d_01_16, "Shaofeng", "Food", new LongMutable(10), new BigDecimal("10.5")));
+            result.add(newRec(info, d_01_16, "Qianhao", "Food", new LongMutable(10), new BigDecimal("10.5")));
+            result.add(newRec(info, d_01_16, "George", "Food", new LongMutable(10), new BigDecimal("10.5")));
+            result.add(newRec(info, d_01_17, "Kejia", "Food", new LongMutable(10), new BigDecimal("10.5")));
+        }
+        return result;
+    }
+    
+    private static String datePlus(String date, int plusDays) {
+        long millis = DateFormat.stringToMillis(date);
+        millis += (1000L * 3600L * 24L) * plusDays;
+        return DateFormat.formatToDateStr(millis);
+    }
+
+    private static GTRecord newRec(GTInfo info, String date, String name, String category, LongMutable amount, BigDecimal price) {
+        GTRecord rec = new GTRecord(info);
+        return rec.setValues(date, name, category, amount, price);
+    }
+
+    private static ImmutableBitSet setOf(int... values) {
+        BitSet set = new BitSet();
+        for (int i : values)
+            set.set(i);
+        return new ImmutableBitSet(set);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/memstore/GTSimpleMemStore.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/memstore/GTSimpleMemStore.java b/core-cube/src/main/java/org/apache/kylin/gridtable/memstore/GTSimpleMemStore.java
new file mode 100644
index 0000000..0011d83
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/memstore/GTSimpleMemStore.java
@@ -0,0 +1,112 @@
+package org.apache.kylin.gridtable.memstore;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTRowBlock;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.IGTStore;
+
+public class GTSimpleMemStore implements IGTStore {
+
+    final GTInfo info;
+    final List<GTRowBlock> rowBlockList;
+
+    public GTSimpleMemStore(GTInfo info) {
+        this.info = info;
+        this.rowBlockList = new ArrayList<GTRowBlock>();
+
+        if (info.isShardingEnabled())
+            throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public GTInfo getInfo() {
+        return info;
+    }
+
+    public long memoryUsage() {
+        if (rowBlockList.size() == 0) {
+            return 0;
+        } else {
+            return rowBlockList.get(0).exportLength() * Long.valueOf(rowBlockList.size());
+        }
+    }
+
+    @Override
+    public IGTStoreWriter rebuild(int shard) {
+        rowBlockList.clear();
+        return new Writer(rowBlockList);
+    }
+
+    @Override
+    public IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) {
+        if (rowBlockList.size() > 0) {
+            GTRowBlock last = rowBlockList.get(rowBlockList.size() - 1);
+            fillLast.copyFrom(last);
+        }
+        return new Writer(rowBlockList);
+    }
+
+    private static class Writer implements IGTStoreWriter {
+
+        private final List<GTRowBlock> rowBlockList;
+
+        Writer(List<GTRowBlock> rowBlockList) {
+            this.rowBlockList = rowBlockList;
+        }
+        @Override
+        public void close() throws IOException {
+        }
+
+        @Override
+        public void write(GTRowBlock block) throws IOException {
+            GTRowBlock copy = block.copy();
+            int id = block.getSequenceId();
+            if (id < rowBlockList.size()) {
+                rowBlockList.set(id, copy);
+            } else {
+                assert id == rowBlockList.size();
+                rowBlockList.add(copy);
+            }
+        }
+    }
+
+    @Override
+    public IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) {
+
+        return new IGTStoreScanner() {
+            Iterator<GTRowBlock> it = rowBlockList.iterator();
+
+            @Override
+            public boolean hasNext() {
+                return it.hasNext();
+            }
+
+            @Override
+            public GTRowBlock next() {
+                return it.next();
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public void close() throws IOException {
+            }
+        };
+    }
+
+    public void drop() throws IOException {
+        //will there be any concurrent issue? If yes, ArrayList should be replaced
+        rowBlockList.clear();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStoreTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStoreTest.java b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStoreTest.java
new file mode 100644
index 0000000..0b4a3c6
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStoreTest.java
@@ -0,0 +1,94 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.kylin.cube.inmemcubing;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.kylin.cube.inmemcubing.ConcurrentDiskStore;
+import org.apache.kylin.gridtable.GTBuilder;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.GridTable;
+import org.apache.kylin.gridtable.IGTScanner;
+import org.apache.kylin.gridtable.UnitTestSupport;
+import org.junit.Test;
+
+public class ConcurrentDiskStoreTest {
+
+    final GTInfo info = UnitTestSupport.advancedInfo();
+    final List<GTRecord> data = UnitTestSupport.mockupData(info, 1000000); // converts to about 34 MB data
+
+    @Test
+    public void testSingleThreadRead() throws IOException, InterruptedException {
+        long start = System.currentTimeMillis();
+        verifyOneTableWriteAndRead(1);
+        long end = System.currentTimeMillis();
+        System.out.println("Cost " + (end - start) + " millis");
+    }
+
+    @Test
+    public void testMultiThreadRead() throws IOException, InterruptedException {
+        long start = System.currentTimeMillis();
+        verifyOneTableWriteAndRead(20);
+        long end = System.currentTimeMillis();
+        System.out.println("Cost " + (end - start) + " millis");
+    }
+    
+    private void verifyOneTableWriteAndRead(int readThreads) throws IOException, InterruptedException {
+        ConcurrentDiskStore store = new ConcurrentDiskStore(info);
+        GridTable table = new GridTable(info, store);
+        verifyWriteAndRead(table, readThreads);
+    }
+
+    private void verifyWriteAndRead(final GridTable table, int readThreads) throws IOException, InterruptedException {
+        GTBuilder builder = table.rebuild();
+        for (GTRecord r : data) {
+            builder.write(r);
+        }
+        builder.close();
+
+        int nThreads = readThreads;
+        Thread[] t = new Thread[nThreads];
+        for (int i = 0; i < nThreads; i++) {
+            t[i] = new Thread() {
+                public void run() {
+                    try {
+                        IGTScanner scanner = table.scan(new GTScanRequest(table.getInfo()));
+                        int i = 0;
+                        for (GTRecord r : scanner) {
+                            assertEquals(data.get(i++), r);
+                        }
+                        scanner.close();
+                    } catch (Exception ex) {
+                        ex.printStackTrace();
+                    }
+                }
+            };
+            t[i].start();
+        }
+        for (int i = 0; i < nThreads; i++) {
+            t[i].join();
+        }
+        
+        ((ConcurrentDiskStore) table.getStore()).close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilderStressTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilderStressTest.java b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilderStressTest.java
new file mode 100644
index 0000000..de949ba
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilderStressTest.java
@@ -0,0 +1,97 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.inmemcubing.DoggedCubeBuilder;
+import org.apache.kylin.cube.inmemcubing.ICuboidWriter;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class DoggedCubeBuilderStressTest extends LocalFileMetadataTestCase {
+
+    @SuppressWarnings("unused")
+    private static final Logger logger = LoggerFactory.getLogger(DoggedCubeBuilderStressTest.class);
+
+    // CI sandbox memory is no more than 512MB, this many input should hit memory threshold
+    private static final int INPUT_ROWS = 200000;
+    private static final int THREADS = 4;
+
+    private static CubeInstance cube;
+    private static String flatTable;
+    private static Map<TblColRef, Dictionary<?>> dictionaryMap;
+
+    @BeforeClass
+    public static void before() throws IOException {
+        staticCreateTestMetadata();
+
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        CubeManager cubeManager = CubeManager.getInstance(kylinConfig);
+
+        cube = cubeManager.getCube("test_kylin_cube_without_slr_left_join_empty");
+        flatTable = "../examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv";
+        dictionaryMap = InMemCubeBuilderTest.getDictionaryMap(cube, flatTable);
+    }
+
+    @AfterClass
+    public static void after() throws Exception {
+        staticCleanupTestMetadata();
+    }
+
+    @Test
+    public void test() throws Exception {
+
+        ArrayBlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(1000);
+        ExecutorService executorService = Executors.newSingleThreadExecutor();
+        long randSeed = System.currentTimeMillis();
+
+        DoggedCubeBuilder doggedBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap);
+        doggedBuilder.setConcurrentThreads(THREADS);
+
+        {
+            Future<?> future = executorService.submit(doggedBuilder.buildAsRunnable(queue, new NoopWriter()));
+            InMemCubeBuilderTest.feedData(cube, flatTable, queue, INPUT_ROWS, randSeed);
+            future.get();
+        }
+    }
+
+    class NoopWriter implements ICuboidWriter {
+        @Override
+        public void write(long cuboidId, GTRecord record) throws IOException {
+        }
+    }
+}
\ No newline at end of file



[19/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..f62df42
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -0,0 +1,379 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.common;
+
+/**
+ * @author George Song (ysong1)
+ *
+ */
+
+import static org.apache.hadoop.util.StringUtils.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.util.CliCommandExecutor;
+import org.apache.kylin.common.util.StringSplitter;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.job.JobInstance;
+import org.apache.kylin.job.cmd.ShellCmdOutput;
+import org.apache.kylin.job.common.OptionsHelper;
+import org.apache.kylin.job.exception.JobException;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("static-access")
+public abstract class AbstractHadoopJob extends Configured implements Tool {
+    protected static final Logger logger = LoggerFactory.getLogger(AbstractHadoopJob.class);
+
+    protected static final Option OPTION_JOB_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Job name. For exmaple, Kylin_Cuboid_Builder-clsfd_v2_Step_22-D)").create("jobname");
+    protected static final Option OPTION_JOB_FLOW_ID = OptionBuilder.withArgName("job flow ID").hasArg().isRequired(true).withDescription("job flow ID").create("jobflowid");
+    protected static final Option OPTION_CUBE_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Cube name. For exmaple, flat_item_cube").create("cubename");
+    protected static final Option OPTION_II_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("II name. For exmaple, some_ii").create("iiname");
+    protected static final Option OPTION_SEGMENT_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Cube segment name)").create("segmentname");
+    protected static final Option OPTION_TABLE_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Hive table name.").create("tablename");
+    protected static final Option OPTION_INPUT_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Input path").create("input");
+    protected static final Option OPTION_INPUT_FORMAT = OptionBuilder.withArgName("inputformat").hasArg().isRequired(false).withDescription("Input format").create("inputformat");
+    protected static final Option OPTION_INPUT_DELIM = OptionBuilder.withArgName("inputdelim").hasArg().isRequired(false).withDescription("Input delimeter").create("inputdelim");
+    protected static final Option OPTION_OUTPUT_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Output path").create("output");
+    protected static final Option OPTION_NCUBOID_LEVEL = OptionBuilder.withArgName("level").hasArg().isRequired(true).withDescription("N-Cuboid build level, e.g. 1, 2, 3...").create("level");
+    protected static final Option OPTION_PARTITION_FILE_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Partition file path.").create("input");
+    protected static final Option OPTION_HTABLE_NAME = OptionBuilder.withArgName("htable name").hasArg().isRequired(true).withDescription("HTable name").create("htablename");
+    protected static final Option OPTION_KEY_COLUMN_PERCENTAGE = OptionBuilder.withArgName("rowkey column percentage").hasArg().isRequired(true).withDescription("Percentage of row key columns").create("columnpercentage");
+    protected static final Option OPTION_KEY_SPLIT_NUMBER = OptionBuilder.withArgName("key split number").hasArg().isRequired(true).withDescription("Number of key split range").create("splitnumber");
+
+    protected static final Option OPTION_STATISTICS_ENABLED = OptionBuilder.withArgName("statisticsenabled").hasArg().isRequired(false).withDescription("Statistics enabled").create("statisticsenabled");
+    protected static final Option OPTION_STATISTICS_OUTPUT = OptionBuilder.withArgName("statisticsoutput").hasArg().isRequired(false).withDescription("Statistics output").create("statisticsoutput");
+    protected static final Option OPTION_STATISTICS_SAMPLING_PERCENT = OptionBuilder.withArgName("statisticssamplingpercent").hasArg().isRequired(false).withDescription("Statistics sampling percentage").create("statisticssamplingpercent");
+
+    protected String name;
+    protected String description;
+    protected boolean isAsync = false;
+    protected OptionsHelper optionsHelper = new OptionsHelper();
+
+    protected Job job;
+
+    public AbstractHadoopJob() {
+        super(HadoopUtil.newHadoopJobConfiguration());
+    }
+
+    protected void parseOptions(Options options, String[] args) throws ParseException {
+        optionsHelper.parseOptions(options, args);
+    }
+
+    public void printUsage(Options options) {
+        optionsHelper.printUsage(getClass().getSimpleName(), options);
+    }
+
+    public Option[] getOptions() {
+        return optionsHelper.getOptions();
+    }
+
+    public String getOptionsAsString() {
+        return optionsHelper.getOptionsAsString();
+    }
+
+    protected String getOptionValue(Option option) {
+        return optionsHelper.getOptionValue(option);
+    }
+
+    protected boolean hasOption(Option option) {
+        return optionsHelper.hasOption(option);
+    }
+
+    protected int waitForCompletion(Job job) throws IOException, InterruptedException, ClassNotFoundException {
+        int retVal = 0;
+        long start = System.nanoTime();
+        if (isAsync) {
+            job.submit();
+        } else {
+            job.waitForCompletion(true);
+            retVal = job.isSuccessful() ? 0 : 1;
+            logger.debug("Job '" + job.getJobName() + "' finished " + (job.isSuccessful() ? "successfully in " : "with failures.  Time taken ") + formatTime((System.nanoTime() - start) / 1000000L));
+        }
+        return retVal;
+    }
+
+    protected static void runJob(Tool job, String[] args) {
+        try {
+            int exitCode = ToolRunner.run(job, args);
+            System.exit(exitCode);
+        } catch (Exception e) {
+            e.printStackTrace(System.err);
+            System.exit(5);
+        }
+    }
+
+    private static final String MAP_REDUCE_CLASSPATH = "mapreduce.application.classpath";
+
+    protected void setJobClasspath(Job job) {
+        String jarPath = KylinConfig.getInstanceFromEnv().getKylinJobJarPath();
+        File jarFile = new File(jarPath);
+        if (jarFile.exists()) {
+            job.setJar(jarPath);
+            logger.info("append job jar: " + jarPath);
+        } else {
+            job.setJarByClass(this.getClass());
+        }
+
+        String kylinHiveDependency = System.getProperty("kylin.hive.dependency");
+        String kylinHBaseDependency = System.getProperty("kylin.hbase.dependency");
+        logger.info("append kylin.hive.dependency: " + kylinHiveDependency + " and kylin.hive.dependency: " + kylinHBaseDependency + " to " + MAP_REDUCE_CLASSPATH);
+
+        Configuration jobConf = job.getConfiguration();
+        String classpath = jobConf.get(MAP_REDUCE_CLASSPATH);
+        if (classpath == null || classpath.length() == 0) {
+            logger.info("Didn't find " + MAP_REDUCE_CLASSPATH + " in job configuration, will run 'mapred classpath' to get the default value.");
+            classpath = getDefaultMapRedClasspath();
+            logger.info("The default mapred classpath is: " + classpath);
+        }
+
+
+        if (kylinHBaseDependency != null) {
+            // yarn classpath is comma separated
+            kylinHBaseDependency = kylinHBaseDependency.replace(":", ",");
+            classpath = classpath + "," + kylinHBaseDependency;
+        }
+
+        if (kylinHiveDependency != null) {
+            // yarn classpath is comma separated
+            kylinHiveDependency = kylinHiveDependency.replace(":", ",");
+            classpath = classpath + "," + kylinHiveDependency;
+        }
+
+        jobConf.set(MAP_REDUCE_CLASSPATH, classpath + "," + kylinHiveDependency);
+        logger.info("Hadoop job classpath is: " + job.getConfiguration().get(MAP_REDUCE_CLASSPATH));
+    }
+
+
+    private String getDefaultMapRedClasspath() {
+
+        String classpath = "";
+        try {
+            CliCommandExecutor executor = KylinConfig.getInstanceFromEnv().getCliCommandExecutor();
+            ShellCmdOutput output = new ShellCmdOutput();
+            executor.execute("mapred classpath", output);
+
+            classpath = output.getOutput().trim().replace(':', ',');
+        } catch (IOException e) {
+            logger.error("Failed to run: 'mapred classpath'.", e);
+        }
+
+        return classpath;
+    }
+
+
+    public void addInputDirs(String input, Job job) throws IOException {
+        for (String inp : StringSplitter.split(input, ",")) {
+            inp = inp.trim();
+            if (inp.endsWith("/*")) {
+                inp = inp.substring(0, inp.length() - 2);
+                FileSystem fs = FileSystem.get(job.getConfiguration());
+                Path path = new Path(inp);
+                FileStatus[] fileStatuses = fs.listStatus(path);
+                boolean hasDir = false;
+                for (FileStatus stat : fileStatuses) {
+                    if (stat.isDirectory() && !stat.getPath().getName().startsWith("_")) {
+                        hasDir = true;
+                        addInputDirs(stat.getPath().toString(), job);
+                    }
+                }
+                if (fileStatuses.length > 0 && !hasDir) {
+                    addInputDirs(path.toString(), job);
+                }
+            } else {
+                logger.debug("Add input " + inp);
+                FileInputFormat.addInputPath(job, new Path(inp));
+            }
+        }
+    }
+
+    public static KylinConfig loadKylinPropsAndMetadata() throws IOException {
+        File metaDir = new File("meta");
+        System.setProperty(KylinConfig.KYLIN_CONF, metaDir.getAbsolutePath());
+        logger.info("The absolute path for meta dir is " + metaDir.getAbsolutePath());
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        kylinConfig.setMetadataUrl(metaDir.getCanonicalPath());
+        return kylinConfig;
+    }
+
+    protected void attachKylinPropsAndMetadata(TableDesc table, Configuration conf) throws IOException {
+        ArrayList<String> dumpList = new ArrayList<String>();
+        dumpList.add(table.getResourcePath());
+        attachKylinPropsAndMetadata(dumpList, conf);
+    }
+    
+    protected void attachKylinPropsAndMetadata(CubeInstance cube, Configuration conf) throws IOException {
+        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+        
+        // write cube / model_desc / cube_desc / dict / table
+        ArrayList<String> dumpList = new ArrayList<String>();
+        dumpList.add(cube.getResourcePath());
+        dumpList.add(cube.getDescriptor().getModel().getResourcePath());
+        dumpList.add(cube.getDescriptor().getResourcePath());
+        
+        for (String tableName : cube.getDescriptor().getModel().getAllTables()) {
+            TableDesc table = metaMgr.getTableDesc(tableName);
+            dumpList.add(table.getResourcePath());
+        }
+        for (CubeSegment segment : cube.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", "");
+        tmp.delete(); // we need a directory, so delete the file first
+
+        File metaDir = new File(tmp, "meta");
+        metaDir.mkdirs();
+        metaDir.getParentFile().deleteOnExit();
+
+        // write kylin.properties
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        File kylinPropsFile = new File(metaDir, "kylin.properties");
+        kylinConfig.writeProperties(kylinPropsFile);
+
+        // write resources
+        dumpResources(kylinConfig, metaDir, dumpList);
+
+        // hadoop distributed cache
+        conf.set("tmpfiles", "file:///" + OptionsHelper.convertToFileURL(metaDir.getAbsolutePath()));
+    }
+
+    private void dumpResources(KylinConfig kylinConfig, File metaDir, ArrayList<String> dumpList) throws IOException {
+        ResourceStore from = ResourceStore.getStore(kylinConfig);
+        KylinConfig localConfig = KylinConfig.createInstanceFromUri(metaDir.getAbsolutePath());
+        ResourceStore to = ResourceStore.getStore(localConfig);
+        for (String path : dumpList) {
+            InputStream in = from.getResource(path);
+            if (in == null)
+                throw new IllegalStateException("No resource found at -- " + path);
+            long ts = from.getResourceTimestamp(path);
+            to.putResource(path, in, ts);
+            //The following log is duplicate with in ResourceStore
+            //log.info("Dumped resource " + path + " to " + metaDir.getAbsolutePath());
+        }
+    }
+
+    protected void deletePath(Configuration conf, Path path) throws IOException {
+        HadoopUtil.deletePath(conf, path);
+    }
+
+    protected double getTotalMapInputMB() throws ClassNotFoundException, IOException, InterruptedException, JobException {
+        if (job == null) {
+            throw new JobException("Job is null");
+        }
+
+        long mapInputBytes = 0;
+        InputFormat<?, ?> input = ReflectionUtils.newInstance(job.getInputFormatClass(), job.getConfiguration());
+        for (InputSplit split : input.getSplits(job)) {
+            mapInputBytes += split.getLength();
+        }
+        if (mapInputBytes == 0) {
+            throw new IllegalArgumentException("Map input splits are 0 bytes, something is wrong!");
+        }
+        double totalMapInputMB = (double) mapInputBytes / 1024 / 1024;
+        return totalMapInputMB;
+    }
+
+    protected int getMapInputSplitCount() throws ClassNotFoundException, JobException, IOException, InterruptedException {
+        if (job == null) {
+            throw new JobException("Job is null");
+        }
+        InputFormat<?, ?> input = ReflectionUtils.newInstance(job.getInputFormatClass(), job.getConfiguration());
+        return input.getSplits(job).size();
+    }
+
+    public void kill() throws JobException {
+        if (job != null) {
+            try {
+                job.killJob();
+            } catch (IOException e) {
+                throw new JobException(e);
+            }
+        }
+    }
+
+    public Map<String, String> getInfo() throws JobException {
+        if (job != null) {
+            Map<String, String> status = new HashMap<String, String>();
+            if (null != job.getJobID()) {
+                status.put(JobInstance.MR_JOB_ID, job.getJobID().toString());
+            }
+            if (null != job.getTrackingURL()) {
+                status.put(JobInstance.YARN_APP_URL, job.getTrackingURL().toString());
+            }
+
+            return status;
+        } else {
+            throw new JobException("Job is null");
+        }
+    }
+
+    public Counters getCounters() throws JobException {
+        if (job != null) {
+            try {
+                return job.getCounters();
+            } catch (IOException e) {
+                throw new JobException(e);
+            }
+        } else {
+            throw new JobException("Job is null");
+        }
+    }
+
+    public void setAsync(boolean isAsync) {
+        this.isAsync = isAsync;
+    }
+
+    public Job getJob() {
+        return this.job;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
new file mode 100644
index 0000000..3a40ce2
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.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.common;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public interface BatchConstants {
+
+    public static final char INTERMEDIATE_TABLE_ROW_DELIMITER = 127;
+
+    public static final String CFG_CUBE_NAME = "cube.name";
+    public static final String CFG_CUBE_SEGMENT_NAME = "cube.segment.name";
+
+    public static final String CFG_II_NAME = "ii.name";
+    public static final String CFG_II_SEGMENT_NAME = "ii.segment.name";
+
+    public static final String INPUT_DELIM = "input.delim";
+    public static final String OUTPUT_PATH = "output.path";
+
+    public static final String TABLE_NAME = "table.name";
+    public static final String TABLE_COLUMNS = "table.columns";
+
+    public static final String CFG_IS_MERGE = "is.merge";
+    public static final String CFG_CUBE_INTERMEDIATE_TABLE_ROW_DELIMITER = "cube.intermediate.table.row.delimiter";
+
+    public static final String MAPREDUCE_COUTNER_GROUP_NAME = "Cube Builder";
+
+    public static final String MAPPER_SAMPLE_NUMBER = "mapper.sample.number";
+    public static final String REGION_NUMBER = "region.number";
+    public static final String CUBE_CAPACITY = "cube.capacity";
+
+    public static final String CFG_STATISTICS_ENABLED = "statistics.enabled";
+    public static final String CFG_STATISTICS_OUTPUT = "statistics.ouput";
+    public static final String CFG_STATISTICS_SAMPLING_PERCENT = "statistics.sampling.percent";
+    public static final String CFG_STATISTICS_CUBE_ESTIMATION = "cube_statistics.txt";
+    public static final String CFG_STATISTICS_CUBOID_ESTIMATION = "cuboid_statistics.seq";
+
+    public static final int COUNTER_MAX = 100000;
+    public static final int ERROR_RECORD_THRESHOLD = 100;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultSslProtocolSocketFactory.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultSslProtocolSocketFactory.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultSslProtocolSocketFactory.java
new file mode 100644
index 0000000..c8e74f6
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultSslProtocolSocketFactory.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.common;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.UnknownHostException;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManager;
+
+import org.apache.commons.httpclient.ConnectTimeoutException;
+import org.apache.commons.httpclient.HttpClientError;
+import org.apache.commons.httpclient.params.HttpConnectionParams;
+import org.apache.commons.httpclient.protocol.ControllerThreadSocketFactory;
+import org.apache.commons.httpclient.protocol.SecureProtocolSocketFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author xduo
+ * 
+ */
+public class DefaultSslProtocolSocketFactory implements SecureProtocolSocketFactory {
+    /** Log object for this class. */
+    private static Logger LOG = LoggerFactory.getLogger(DefaultSslProtocolSocketFactory.class);
+    private SSLContext sslcontext = null;
+
+    /**
+     * Constructor for DefaultSslProtocolSocketFactory.
+     */
+    public DefaultSslProtocolSocketFactory() {
+        super();
+    }
+
+    /**
+     * @see SecureProtocolSocketFactory#createSocket(java.lang.String,int,java.net.InetAddress,int)
+     */
+    public Socket createSocket(String host, int port, InetAddress clientHost, int clientPort) throws IOException, UnknownHostException {
+        return getSSLContext().getSocketFactory().createSocket(host, port, clientHost, clientPort);
+    }
+
+    /**
+     * Attempts to get a new socket connection to the given host within the
+     * given time limit.
+     * 
+     * <p>
+     * To circumvent the limitations of older JREs that do not support connect
+     * timeout a controller thread is executed. The controller thread attempts
+     * to create a new socket within the given limit of time. If socket
+     * constructor does not return until the timeout expires, the controller
+     * terminates and throws an {@link ConnectTimeoutException}
+     * </p>
+     * 
+     * @param host
+     *            the host name/IP
+     * @param port
+     *            the port on the host
+     * @param localAddress
+     *            the local host name/IP to bind the socket to
+     * @param localPort
+     *            the port on the local machine
+     * @param params
+     *            {@link HttpConnectionParams Http connection parameters}
+     * 
+     * @return Socket a new socket
+     * 
+     * @throws IOException
+     *             if an I/O error occurs while creating the socket
+     * @throws UnknownHostException
+     *             if the IP address of the host cannot be determined
+     * @throws ConnectTimeoutException
+     *             DOCUMENT ME!
+     * @throws IllegalArgumentException
+     *             DOCUMENT ME!
+     */
+    public Socket createSocket(final String host, final int port, final InetAddress localAddress, final int localPort, final HttpConnectionParams params) throws IOException, UnknownHostException, ConnectTimeoutException {
+        if (params == null) {
+            throw new IllegalArgumentException("Parameters may not be null");
+        }
+
+        int timeout = params.getConnectionTimeout();
+
+        if (timeout == 0) {
+            return createSocket(host, port, localAddress, localPort);
+        } else {
+            // To be eventually deprecated when migrated to Java 1.4 or above
+            return ControllerThreadSocketFactory.createSocket(this, host, port, localAddress, localPort, timeout);
+        }
+    }
+
+    /**
+     * @see SecureProtocolSocketFactory#createSocket(java.lang.String,int)
+     */
+    public Socket createSocket(String host, int port) throws IOException, UnknownHostException {
+        return getSSLContext().getSocketFactory().createSocket(host, port);
+    }
+
+    /**
+     * @see SecureProtocolSocketFactory#createSocket(java.net.Socket,java.lang.String,int,boolean)
+     */
+    public Socket createSocket(Socket socket, String host, int port, boolean autoClose) throws IOException, UnknownHostException {
+        return getSSLContext().getSocketFactory().createSocket(socket, host, port, autoClose);
+    }
+
+    public boolean equals(Object obj) {
+        return ((obj != null) && obj.getClass().equals(DefaultX509TrustManager.class));
+    }
+
+    public int hashCode() {
+        return DefaultX509TrustManager.class.hashCode();
+    }
+
+    private static SSLContext createEasySSLContext() {
+        try {
+            SSLContext context = SSLContext.getInstance("TLS");
+            context.init(null, new TrustManager[] { new DefaultX509TrustManager(null) }, null);
+
+            return context;
+        } catch (Exception e) {
+            LOG.error(e.getMessage(), e);
+            throw new HttpClientError(e.toString());
+        }
+    }
+
+    private SSLContext getSSLContext() {
+        if (this.sslcontext == null) {
+            this.sslcontext = createEasySSLContext();
+        }
+
+        return this.sslcontext;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultX509TrustManager.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultX509TrustManager.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultX509TrustManager.java
new file mode 100644
index 0000000..d7901e5
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultX509TrustManager.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.common;
+
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author xduo
+ * 
+ */
+public class DefaultX509TrustManager implements X509TrustManager {
+
+    /** Log object for this class. */
+    private static Logger LOG = LoggerFactory.getLogger(DefaultX509TrustManager.class);
+    private X509TrustManager standardTrustManager = null;
+
+    /**
+     * Constructor for DefaultX509TrustManager.
+     * 
+     */
+    public DefaultX509TrustManager(KeyStore keystore) throws NoSuchAlgorithmException, KeyStoreException {
+        super();
+
+        TrustManagerFactory factory = TrustManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        factory.init(keystore);
+
+        TrustManager[] trustmanagers = factory.getTrustManagers();
+
+        if (trustmanagers.length == 0) {
+            throw new NoSuchAlgorithmException("SunX509 trust manager not supported");
+        }
+
+        this.standardTrustManager = (X509TrustManager) trustmanagers[0];
+    }
+
+    public X509Certificate[] getAcceptedIssuers() {
+        return this.standardTrustManager.getAcceptedIssuers();
+    }
+
+    public boolean isClientTrusted(X509Certificate[] certificates) {
+        return true;
+        // return this.standardTrustManager.isClientTrusted(certificates);
+    }
+
+    public boolean isServerTrusted(X509Certificate[] certificates) {
+        if ((certificates != null) && LOG.isDebugEnabled()) {
+            LOG.debug("Server certificate chain:");
+
+            for (int i = 0; i < certificates.length; i++) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("X509Certificate[" + i + "]=" + certificates[i]);
+                }
+            }
+        }
+
+        if ((certificates != null) && (certificates.length == 1)) {
+            X509Certificate certificate = certificates[0];
+
+            try {
+                certificate.checkValidity();
+            } catch (CertificateException e) {
+                LOG.error(e.toString());
+
+                return false;
+            }
+
+            return true;
+        } else {
+            return true;
+            // return this.standardTrustManager.isServerTrusted(certificates);
+        }
+    }
+
+    @Override
+    public void checkClientTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void checkServerTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+        // TODO Auto-generated method stub
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
new file mode 100644
index 0000000..294c957
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.common;
+
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * @author xduo
+ * 
+ */
+public class HadoopCmdOutput {
+
+    protected static final Logger log = LoggerFactory.getLogger(HadoopCmdOutput.class);
+
+    private final StringBuilder output;
+    private final Job job;
+
+    public HadoopCmdOutput(Job job, StringBuilder output) {
+        super();
+        this.job = job;
+        this.output = output;
+    }
+
+    public String getMrJobId() {
+        return getInfo().get(ExecutableConstants.MR_JOB_ID);
+    }
+
+    public Map<String, String> getInfo() {
+        if (job != null) {
+            Map<String, String> status = new HashMap<String, String>();
+            if (null != job.getJobID()) {
+                status.put(ExecutableConstants.MR_JOB_ID, job.getJobID().toString());
+            }
+            if (null != job.getTrackingURL()) {
+                status.put(ExecutableConstants.YARN_APP_URL, job.getTrackingURL().toString());
+            }
+            return status;
+        } else {
+            return Collections.emptyMap();
+        }
+    }
+
+    private String mapInputRecords;
+    private String hdfsBytesWritten;
+    private String hdfsBytesRead;
+
+    public String getMapInputRecords() {
+        return mapInputRecords;
+    }
+
+    public String getHdfsBytesWritten() {
+        return hdfsBytesWritten;
+    }
+
+    public String getHdfsBytesRead() {
+        return hdfsBytesRead;
+    }
+    
+    public void updateJobCounter() {
+        try {
+            Counters counters = job.getCounters();
+            if (counters == null) {
+                String errorMsg = "no counters for job " + getMrJobId();
+                log.warn(errorMsg);
+                output.append(errorMsg);
+                return;
+            }
+            this.output.append(counters.toString()).append("\n");
+            log.debug(counters.toString());
+
+            mapInputRecords = String.valueOf(counters.findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue());
+            hdfsBytesWritten = String.valueOf(counters.findCounter("FileSystemCounters", "HDFS_BYTES_WRITTEN").getValue());
+            hdfsBytesRead = String.valueOf(counters.findCounter("FileSystemCounters", "HDFS_BYTES_READ").getValue());
+        } catch (Exception e) {
+            log.error(e.getLocalizedMessage(), e);
+            output.append(e.getLocalizedMessage());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopShellExecutable.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopShellExecutable.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopShellExecutable.java
new file mode 100644
index 0000000..089df5f
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopShellExecutable.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.common;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.reflect.Constructor;
+
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.util.ClassUtil;
+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 com.google.common.base.Preconditions;
+
+/**
+ */
+public class HadoopShellExecutable extends AbstractExecutable {
+
+    private static final String KEY_MR_JOB = "HADOOP_SHELL_JOB_CLASS";
+    private static final String KEY_PARAMS = "HADOOP_SHELL_JOB_PARAMS";
+
+    public HadoopShellExecutable() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final String mapReduceJobClass = getJobClass();
+        String params = getJobParams();
+        Preconditions.checkNotNull(mapReduceJobClass);
+        Preconditions.checkNotNull(params);
+        try {
+            final Constructor<? extends AbstractHadoopJob> constructor = ClassUtil.forName(mapReduceJobClass, AbstractHadoopJob.class).getConstructor();
+            final AbstractHadoopJob job = constructor.newInstance();
+            String[] args = params.trim().split("\\s+");
+            logger.info("parameters of the HadoopShellExecutable:");
+            logger.info(params);
+            int result;
+            StringBuilder log = new StringBuilder();
+            try {
+                result = ToolRunner.run(job, args);
+            } catch (Exception ex) {
+                logger.error("error execute " + this.toString(), ex);
+                StringWriter stringWriter = new StringWriter();
+                ex.printStackTrace(new PrintWriter(stringWriter));
+                log.append(stringWriter.toString()).append("\n");
+                result = 2;
+            }
+            log.append("result code:").append(result);
+            return result == 0 ? new ExecuteResult(ExecuteResult.State.SUCCEED, log.toString()):new ExecuteResult(ExecuteResult.State.FAILED, log.toString());
+        } catch (ReflectiveOperationException e) {
+            logger.error("error getMapReduceJobClass, class name:" + getParam(KEY_MR_JOB), e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        } catch (Exception e) {
+            logger.error("error execute " + this.toString(), e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+
+    public void setJobClass(Class<? extends AbstractHadoopJob> clazzName) {
+        setParam(KEY_MR_JOB, clazzName.getName());
+    }
+
+    public String getJobClass() throws ExecuteException {
+        return getParam(KEY_MR_JOB);
+    }
+
+    public void setJobParams(String param) {
+        setParam(KEY_PARAMS, param);
+    }
+
+    public String getJobParams() {
+        return getParam(KEY_PARAMS);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusChecker.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusChecker.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusChecker.java
new file mode 100644
index 0000000..1b71b92
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusChecker.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.common;
+
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.kylin.job.constant.JobStepStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author xduo
+ * 
+ */
+public class HadoopStatusChecker {
+
+    protected static final Logger logger = LoggerFactory.getLogger(HadoopStatusChecker.class);
+
+    private final String yarnUrl;
+    private final String mrJobID;
+    private final StringBuilder output;
+
+    public HadoopStatusChecker(String yarnUrl, String mrJobID, StringBuilder output) {
+        this.yarnUrl = yarnUrl;
+        this.mrJobID = mrJobID;
+        this.output = output;
+    }
+
+    public JobStepStatusEnum checkStatus() {
+        if (null == mrJobID) {
+            this.output.append("Skip status check with empty job id..\n");
+            return JobStepStatusEnum.WAITING;
+        }
+        JobStepStatusEnum status = null;
+        try {
+            final Pair<RMAppState, FinalApplicationStatus> result = new HadoopStatusGetter(yarnUrl, mrJobID).get();
+            logger.debug("State of Hadoop job: " + mrJobID + ":" + result.getLeft() + "-" + result.getRight());
+            output.append(new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S").format(new Date()) + " - State of Hadoop job: " + mrJobID + ":" + result.getLeft() + " - " + result.getRight() + "\n");
+
+            switch (result.getRight()) {
+            case SUCCEEDED:
+                status = JobStepStatusEnum.FINISHED;
+                break;
+            case FAILED:
+                status = JobStepStatusEnum.ERROR;
+                break;
+            case KILLED:
+                status = JobStepStatusEnum.KILLED;
+                break;
+            case UNDEFINED:
+                switch (result.getLeft()) {
+                case NEW:
+                case NEW_SAVING:
+                case SUBMITTED:
+                case ACCEPTED:
+                    status = JobStepStatusEnum.WAITING;
+                    break;
+                case RUNNING:
+                    status = JobStepStatusEnum.RUNNING;
+                    break;
+                case FINAL_SAVING:
+                case FINISHING:
+                case FINISHED:
+                case FAILED:
+                case KILLING:
+                case KILLED:
+                }
+                break;
+            }
+        } catch (Exception e) {
+            logger.error("error check status", e);
+            output.append("Exception: " + e.getLocalizedMessage() + "\n");
+            status = JobStepStatusEnum.ERROR;
+        }
+
+        return status;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
new file mode 100644
index 0000000..53b8850
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.common;
+
+import java.io.IOException;
+
+import org.apache.commons.httpclient.Header;
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.HttpMethod;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.commons.httpclient.protocol.Protocol;
+import org.apache.commons.httpclient.protocol.ProtocolSocketFactory;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class HadoopStatusGetter {
+
+    private final String mrJobId;
+    private final String yarnUrl;
+
+    protected static final Logger log = LoggerFactory.getLogger(HadoopStatusChecker.class);
+
+    public HadoopStatusGetter(String yarnUrl, String mrJobId) {
+        this.yarnUrl = yarnUrl;
+        this.mrJobId = mrJobId;
+    }
+
+    public Pair<RMAppState, FinalApplicationStatus> get() throws IOException {
+        String applicationId = mrJobId.replace("job", "application");
+        String url = yarnUrl.replace("${job_id}", applicationId);
+        JsonNode root = new ObjectMapper().readTree(getHttpResponse(url));
+        RMAppState state = RMAppState.valueOf(root.findValue("state").getTextValue());
+        FinalApplicationStatus finalStatus = FinalApplicationStatus.valueOf(root.findValue("finalStatus").getTextValue());
+        return Pair.of(state, finalStatus);
+    }
+
+    private String getHttpResponse(String url) throws IOException {
+        HttpClient client = new HttpClient();
+
+        String response = null;
+        while (response == null) { // follow redirects via 'refresh'
+            if (url.startsWith("https://")) {
+                registerEasyHttps();
+            }
+            if (url.contains("anonymous=true") == false) {
+                url += url.contains("?") ? "&" : "?";
+                url += "anonymous=true";
+            }
+
+            HttpMethod get = new GetMethod(url);
+            get.addRequestHeader("accept", "application/json");
+
+            try {
+                client.executeMethod(get);
+
+                String redirect = null;
+                Header h = get.getResponseHeader("Refresh");
+                if (h != null) {
+                    String s = h.getValue();
+                    int cut = s.indexOf("url=");
+                    if (cut >= 0) {
+                        redirect = s.substring(cut + 4);
+                    }
+                }
+
+                if (redirect == null) {
+                    response = get.getResponseBodyAsString();
+                    log.debug("Job " + mrJobId + " get status check result.\n");
+                } else {
+                    url = redirect;
+                    log.debug("Job " + mrJobId + " check redirect url " + url + ".\n");
+                }
+            } finally {
+                get.releaseConnection();
+            }
+        }
+
+        return response;
+    }
+
+    private static Protocol EASY_HTTPS = null;
+
+    private static void registerEasyHttps() {
+        // by pass all https issue
+        if (EASY_HTTPS == null) {
+            EASY_HTTPS = new Protocol("https", (ProtocolSocketFactory) new DefaultSslProtocolSocketFactory(), 443);
+            Protocol.registerProtocol("https", EASY_HTTPS);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
new file mode 100644
index 0000000..b62fd21
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.common;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.reflect.Constructor;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.constant.JobStepStatusEnum;
+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.ExecutableState;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.job.execution.Output;
+
+import com.google.common.base.Preconditions;
+
+/**
+ */
+public class MapReduceExecutable extends AbstractExecutable {
+
+    private static final String KEY_MR_JOB = "MR_JOB_CLASS";
+    private static final String KEY_PARAMS = "MR_JOB_PARAMS";
+    private static final String KEY_COUNTER_SAVEAS = "MR_COUNTER_SAVEAS";
+    
+    public static final String MAP_REDUCE_WAIT_TIME = "mapReduceWaitTime";
+
+    public MapReduceExecutable() {
+        super();
+    }
+
+    @Override
+    protected void onExecuteStart(ExecutableContext executableContext) {
+        final Output output = executableManager.getOutput(getId());
+        if (output.getExtra().containsKey(START_TIME)) {
+            final String mrJobId = output.getExtra().get(ExecutableConstants.MR_JOB_ID);
+            if (mrJobId == null) {
+                executableManager.updateJobOutput(getId(), ExecutableState.RUNNING, null, null);
+                return;
+            }
+            try {
+                Job job = new Cluster(new Configuration()).getJob(JobID.forName(mrJobId));
+                if (job.getJobState() == JobStatus.State.FAILED) {
+                    //remove previous mr job info
+                    super.onExecuteStart(executableContext);
+                } else {
+                    executableManager.updateJobOutput(getId(), ExecutableState.RUNNING, null, null);
+                }
+            } catch (IOException e) {
+                logger.warn("error get hadoop status");
+                super.onExecuteStart(executableContext);
+            } catch (InterruptedException e) {
+                logger.warn("error get hadoop status");
+                super.onExecuteStart(executableContext);
+            }
+        } else {
+            super.onExecuteStart(executableContext);
+        }
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final String mapReduceJobClass = getMapReduceJobClass();
+        String params = getMapReduceParams();
+        Preconditions.checkNotNull(mapReduceJobClass);
+        Preconditions.checkNotNull(params);
+        try {
+            Job job;
+            final Map<String, String> extra = executableManager.getOutput(getId()).getExtra();
+            if (extra.containsKey(ExecutableConstants.MR_JOB_ID)) {
+                job = new Cluster(new Configuration()).getJob(JobID.forName(extra.get(ExecutableConstants.MR_JOB_ID)));
+                logger.info("mr_job_id:" + extra.get(ExecutableConstants.MR_JOB_ID + " resumed"));
+            } else {
+                final Constructor<? extends AbstractHadoopJob> constructor = ClassUtil.forName(mapReduceJobClass, AbstractHadoopJob.class).getConstructor();
+                final AbstractHadoopJob hadoopJob = constructor.newInstance();
+                hadoopJob.setAsync(true); // so the ToolRunner.run() returns right away
+                logger.info("parameters of the MapReduceExecutable:");
+                logger.info(params);
+                String[] args = params.trim().split("\\s+");
+                try {
+                    //for async mr job, ToolRunner just return 0;
+                    ToolRunner.run(hadoopJob, args);
+                } catch (Exception ex) {
+                    StringBuilder log = new StringBuilder();
+                    logger.error("error execute " + this.toString(), ex);
+                    StringWriter stringWriter = new StringWriter();
+                    ex.printStackTrace(new PrintWriter(stringWriter));
+                    log.append(stringWriter.toString()).append("\n");
+                    log.append("result code:").append(2);
+                    return new ExecuteResult(ExecuteResult.State.ERROR, log.toString());
+                }
+                job = hadoopJob.getJob();
+            }
+            final StringBuilder output = new StringBuilder();
+            final HadoopCmdOutput hadoopCmdOutput = new HadoopCmdOutput(job, output);
+
+            final String restStatusCheckUrl = getRestStatusCheckUrl(job, context.getConfig());
+            if (restStatusCheckUrl == null) {
+                logger.error("restStatusCheckUrl is null");
+                return new ExecuteResult(ExecuteResult.State.ERROR, "restStatusCheckUrl is null");
+            }
+            String mrJobId = hadoopCmdOutput.getMrJobId();
+            HadoopStatusChecker statusChecker = new HadoopStatusChecker(restStatusCheckUrl, mrJobId, output);
+            JobStepStatusEnum status = JobStepStatusEnum.NEW;
+            while (!isDiscarded()) {
+                JobStepStatusEnum newStatus = statusChecker.checkStatus();
+                if (status == JobStepStatusEnum.KILLED) {
+                    executableManager.updateJobOutput(getId(), ExecutableState.ERROR, Collections.<String, String>emptyMap(), "killed by admin");
+                    return new ExecuteResult(ExecuteResult.State.FAILED, "killed by admin");
+                }
+                if (status == JobStepStatusEnum.WAITING && (newStatus == JobStepStatusEnum.FINISHED || newStatus == JobStepStatusEnum.ERROR || newStatus == JobStepStatusEnum.RUNNING)) {
+                    final long waitTime = System.currentTimeMillis() - getStartTime();
+                    setMapReduceWaitTime(waitTime);
+                }
+                status = newStatus;
+                executableManager.addJobInfo(getId(), hadoopCmdOutput.getInfo());
+                if (status.isComplete()) {
+                    final Map<String, String> info = hadoopCmdOutput.getInfo();
+                    readCounters(hadoopCmdOutput, info);
+                    executableManager.addJobInfo(getId(), info);
+
+                    if (status == JobStepStatusEnum.FINISHED) {
+                        return new ExecuteResult(ExecuteResult.State.SUCCEED, output.toString());
+                    } else {
+                        return new ExecuteResult(ExecuteResult.State.FAILED, output.toString());
+                    }
+                }
+                Thread.sleep(context.getConfig().getYarnStatusCheckIntervalSeconds() * 1000);
+            }
+            //TODO kill discarded mr job using "hadoop job -kill " + mrJobId
+
+            return new ExecuteResult(ExecuteResult.State.DISCARDED, output.toString());
+
+        } catch (ReflectiveOperationException e) {
+            logger.error("error getMapReduceJobClass, class name:" + getParam(KEY_MR_JOB), e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        } catch (Exception e) {
+            logger.error("error execute " + this.toString(), e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+
+    private void readCounters(final HadoopCmdOutput hadoopCmdOutput, final Map<String, String> info) {
+        hadoopCmdOutput.updateJobCounter();
+        info.put(ExecutableConstants.SOURCE_RECORDS_COUNT, hadoopCmdOutput.getMapInputRecords());
+        info.put(ExecutableConstants.SOURCE_RECORDS_SIZE, hadoopCmdOutput.getHdfsBytesRead());
+        info.put(ExecutableConstants.HDFS_BYTES_WRITTEN, hadoopCmdOutput.getHdfsBytesWritten());
+        
+        String saveAs = getParam(KEY_COUNTER_SAVEAS);
+        if (saveAs != null) {
+            String[] saveAsNames = saveAs.split(",");
+            saveCounterAs(hadoopCmdOutput.getMapInputRecords(), saveAsNames, 0, info);
+            saveCounterAs(hadoopCmdOutput.getHdfsBytesRead(), saveAsNames, 1, info);
+            saveCounterAs(hadoopCmdOutput.getHdfsBytesWritten(), saveAsNames, 2, info);
+        }
+    }
+
+    private void saveCounterAs(String counter, String[] saveAsNames, int i, Map<String, String> info) {
+        if (saveAsNames.length > i && StringUtils.isBlank(saveAsNames[i]) == false) {
+            info.put(saveAsNames[i].trim(), counter);
+        }
+    }
+
+    private String getRestStatusCheckUrl(Job job, KylinConfig config) {
+        final String yarnStatusCheckUrl = config.getYarnStatusCheckUrl();
+        if (yarnStatusCheckUrl != null) {
+            return yarnStatusCheckUrl;
+        } else {
+            logger.info(KylinConfig.KYLIN_JOB_YARN_APP_REST_CHECK_URL + " is not set, read from job configuration");
+        }
+        String rmWebHost = job.getConfiguration().get("yarn.resourcemanager.webapp.address");
+        if (StringUtils.isEmpty(rmWebHost)) {
+            return null;
+        }
+        if (rmWebHost.startsWith("http://") || rmWebHost.startsWith("https://")) {
+            //do nothing
+        } else {
+            rmWebHost = "http://" + rmWebHost;
+        }
+        logger.info("yarn.resourcemanager.webapp.address:" + rmWebHost);
+        return rmWebHost + "/ws/v1/cluster/apps/${job_id}?anonymous=true";
+    }
+
+    public long getMapReduceWaitTime() {
+        return getExtraInfoAsLong(MAP_REDUCE_WAIT_TIME, 0L);
+    }
+
+    public void setMapReduceWaitTime(long t) {
+        addExtraInfo(MAP_REDUCE_WAIT_TIME, t + "");
+    }
+
+    public void setMapReduceJobClass(Class<? extends AbstractHadoopJob> clazzName) {
+        setParam(KEY_MR_JOB, clazzName.getName());
+    }
+
+    public String getMapReduceJobClass() throws ExecuteException {
+        return getParam(KEY_MR_JOB);
+    }
+
+    public void setMapReduceParams(String param) {
+        setParam(KEY_PARAMS, param);
+    }
+
+    public String getMapReduceParams() {
+        return getParam(KEY_PARAMS);
+    }
+    
+    public String getCounterSaveAs() {
+        return getParam(KEY_COUNTER_SAVEAS);
+    }
+    
+    public void setCounterSaveAs(String value) {
+        setParam(KEY_COUNTER_SAVEAS, value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidJob.java
new file mode 100644
index 0000000..2e716eb
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidJob.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * @author honma
+ * 
+ */
+
+public class BaseCuboidJob extends CuboidJob {
+    public BaseCuboidJob() {
+        this.setMapperClass(HiveToBaseCuboidMapper.class);
+    }
+
+    public static void main(String[] args) throws Exception {
+        CuboidJob job = new BaseCuboidJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
new file mode 100644
index 0000000..9bc84ff
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
@@ -0,0 +1,205 @@
+package org.apache.kylin.engine.mr.steps;
+
+import com.google.common.collect.Lists;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.BytesSplitter;
+import org.apache.kylin.common.util.SplittedBytes;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+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.SegmentStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ */
+public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VALUEIN, Text, Text> {
+    protected static final Logger logger = LoggerFactory.getLogger(HiveToBaseCuboidMapper.class);
+    public static final byte[] HIVE_NULL = Bytes.toBytes("\\N");
+    public static final byte[] ONE = Bytes.toBytes("1");
+    protected String cubeName;
+    protected String segmentName;
+    protected Cuboid baseCuboid;
+    protected CubeInstance cube;
+    protected CubeDesc cubeDesc;
+    protected CubeSegment cubeSegment;
+    protected List<byte[]> nullBytes;
+    protected CubeJoinedFlatTableDesc intermediateTableDesc;
+    protected String intermediateTableRowDelimiter;
+    protected byte byteRowDelimiter;
+    protected int counter;
+    protected Object[] measures;
+    protected byte[][] keyBytesBuf;
+    protected BytesSplitter bytesSplitter;
+    protected AbstractRowKeyEncoder rowKeyEncoder;
+    protected MeasureCodec measureCodec;
+    private int errorRecordCounter;
+    private Text outputKey = new Text();
+    private Text outputValue = new Text();
+    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME);
+        intermediateTableRowDelimiter = context.getConfiguration().get(BatchConstants.CFG_CUBE_INTERMEDIATE_TABLE_ROW_DELIMITER, Character.toString(BatchConstants.INTERMEDIATE_TABLE_ROW_DELIMITER));
+        if (Bytes.toBytes(intermediateTableRowDelimiter).length > 1) {
+            throw new RuntimeException("Expected delimiter byte length is 1, but got " + Bytes.toBytes(intermediateTableRowDelimiter).length);
+        }
+
+        byteRowDelimiter = Bytes.toBytes(intermediateTableRowDelimiter)[0];
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        cube = CubeManager.getInstance(config).getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+        cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+
+        intermediateTableDesc = new CubeJoinedFlatTableDesc(cube.getDescriptor(), cubeSegment);
+
+        bytesSplitter = new BytesSplitter(200, 4096);
+        rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid);
+
+        measureCodec = new MeasureCodec(cubeDesc.getMeasures());
+        measures = new Object[cubeDesc.getMeasures().size()];
+
+        int colCount = cubeDesc.getRowkey().getRowKeyColumns().length;
+        keyBytesBuf = new byte[colCount][];
+
+        initNullBytes();
+    }
+
+    private void initNullBytes() {
+        nullBytes = Lists.newArrayList();
+        nullBytes.add(HIVE_NULL);
+        String[] nullStrings = cubeDesc.getNullStrings();
+        if (nullStrings != null) {
+            for (String s : nullStrings) {
+                nullBytes.add(Bytes.toBytes(s));
+            }
+        }
+    }
+
+    private boolean isNull(byte[] v) {
+        for (byte[] nullByte : nullBytes) {
+            if (Bytes.equals(v, nullByte))
+                return true;
+        }
+        return false;
+    }
+
+    private byte[] buildKey(SplittedBytes[] splitBuffers) {
+        int[] rowKeyColumnIndexes = intermediateTableDesc.getRowKeyColumnIndexes();
+        for (int i = 0; i < baseCuboid.getColumns().size(); i++) {
+            int index = rowKeyColumnIndexes[i];
+            keyBytesBuf[i] = Arrays.copyOf(splitBuffers[index].value, splitBuffers[index].length);
+            if (isNull(keyBytesBuf[i])) {
+                keyBytesBuf[i] = null;
+            }
+        }
+        return rowKeyEncoder.encode(keyBytesBuf);
+    }
+
+    private void buildValue(SplittedBytes[] splitBuffers) {
+
+        for (int i = 0; i < measures.length; i++) {
+            byte[] valueBytes = getValueBytes(splitBuffers, i);
+            measures[i] = measureCodec.getSerializer(i).valueOf(valueBytes);
+        }
+
+        valueBuf.clear();
+        measureCodec.encode(measures, valueBuf);
+    }
+
+    private byte[] getValueBytes(SplittedBytes[] splitBuffers, int measureIdx) {
+        MeasureDesc desc = cubeDesc.getMeasures().get(measureIdx);
+        FunctionDesc func = desc.getFunction();
+        ParameterDesc paramDesc = func.getParameter();
+        int[] flatTableIdx = intermediateTableDesc.getMeasureColumnIndexes()[measureIdx];
+
+        byte[] result = null;
+
+        // constant
+        if (flatTableIdx == null) {
+            result = Bytes.toBytes(paramDesc.getValue());
+        }
+        // column values
+        else {
+            // for multiple columns, their values are joined
+            for (int i = 0; i < flatTableIdx.length; i++) {
+                SplittedBytes split = splitBuffers[flatTableIdx[i]];
+                if (result == null) {
+                    result = Arrays.copyOf(split.value, split.length);
+                } else {
+                    byte[] newResult = new byte[result.length + split.length];
+                    System.arraycopy(result, 0, newResult, 0, result.length);
+                    System.arraycopy(split.value, 0, newResult, result.length, split.length);
+                    result = newResult;
+                }
+            }
+        }
+
+        if (func.isCount() || func.isHolisticCountDistinct()) {
+            // note for holistic count distinct, this value will be ignored
+            result = ONE;
+        }
+
+        if (isNull(result)) {
+            result = null;
+        }
+
+        return result;
+    }
+
+    protected void outputKV(Context context) throws IOException, InterruptedException {
+        intermediateTableDesc.sanityCheck(bytesSplitter);
+
+        byte[] rowKey = buildKey(bytesSplitter.getSplitBuffers());
+        outputKey.set(rowKey, 0, rowKey.length);
+
+        buildValue(bytesSplitter.getSplitBuffers());
+        outputValue.set(valueBuf.array(), 0, valueBuf.position());
+        context.write(outputKey, outputValue);
+    }
+
+    protected void handleErrorRecord(BytesSplitter bytesSplitter, Exception ex) throws IOException {
+
+        System.err.println("Insane record: " + bytesSplitter);
+        ex.printStackTrace(System.err);
+
+        errorRecordCounter++;
+        if (errorRecordCounter > BatchConstants.ERROR_RECORD_THRESHOLD) {
+            if (ex instanceof IOException)
+                throw (IOException) ex;
+            else if (ex instanceof RuntimeException)
+                throw (RuntimeException) ex;
+            else
+                throw new RuntimeException("", ex);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
new file mode 100644
index 0000000..4b4c815
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.cli.DictionaryGeneratorCLI;
+import org.apache.kylin.dict.DistinctColumnValuesProvider;
+import org.apache.kylin.engine.mr.DFSFileTable;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.source.ReadableTable;
+
+/**
+ * @author ysong1
+ * 
+ */
+
+public class CreateDictionaryJob extends AbstractHadoopJob {
+
+    private int returnCode = 0;
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            parseOptions(options, args);
+
+            final String cubeName = getOptionValue(OPTION_CUBE_NAME);
+            final String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+            final String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
+
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+            DictionaryGeneratorCLI.processSegment(config, cubeName, segmentName, new DistinctColumnValuesProvider() {
+                @Override
+                public ReadableTable getDistinctValuesFor(TblColRef col) {
+                    return new DFSFileTable(factColumnsInputPath + "/" + col.getName(), -1);
+                }
+            });
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+
+        return returnCode;
+    }
+    
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new CreateDictionaryJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
new file mode 100644
index 0000000..88edfe1
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+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.cube.cuboid.CuboidCLI;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.MRUtil;
+import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.job.exception.JobException;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * @author ysong1
+ */
+public class CuboidJob extends AbstractHadoopJob {
+
+    protected static final Logger logger = LoggerFactory.getLogger(CuboidJob.class);
+    private static final String MAPRED_REDUCE_TASKS = "mapred.reduce.tasks";
+
+    @SuppressWarnings("rawtypes")
+    private Class<? extends Mapper> mapperClass;
+
+    @Override
+    public int run(String[] args) throws Exception {
+        if (this.mapperClass == null)
+            throw new Exception("Mapper class is not set!");
+        
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_NCUBOID_LEVEL);
+            options.addOption(OPTION_INPUT_FORMAT);
+            parseOptions(options, args);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            int nCuboidLevel = Integer.parseInt(getOptionValue(OPTION_NCUBOID_LEVEL));
+            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            CubeManager cubeMgr = CubeManager.getInstance(config);
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            logger.info("Starting: " + job.getJobName());
+
+            setJobClasspath(job);
+
+            // Mapper
+            configureMapperInputFormat(cube.getSegment(segmentName, SegmentStatusEnum.NEW));
+            job.setMapperClass(this.mapperClass);
+            job.setMapOutputKeyClass(Text.class);
+            job.setMapOutputValueClass(Text.class);
+            job.setCombinerClass(CuboidReducer.class); // for base cuboid shuffle skew, some rowkey aggregates far more records than others
+
+            // Reducer
+            job.setReducerClass(CuboidReducer.class);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(Text.class);
+
+            FileOutputFormat.setOutputPath(job, output);
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+            // add metadata to distributed cache
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            setReduceTaskNum(job, config, cubeName, nCuboidLevel);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            logger.error("error in CuboidJob", e);
+            printUsage(options);
+            throw e;
+        }
+    }
+
+    private void configureMapperInputFormat(CubeSegment cubeSeg) throws IOException {
+        String input = getOptionValue(OPTION_INPUT_PATH);
+        
+        if (StringUtils.isBlank(input)) {
+            // base cuboid case
+            IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSeg).getFlatTableInputFormat();
+            flatTableInputFormat.configureJob(job);
+        }
+        else {
+            // n-dimension cuboid case
+            FileInputFormat.setInputPaths(job, new Path(input));
+            if (hasOption(OPTION_INPUT_FORMAT) && ("textinputformat".equalsIgnoreCase(getOptionValue(OPTION_INPUT_FORMAT)))) {
+                job.setInputFormatClass(TextInputFormat.class);
+            } else {
+                job.setInputFormatClass(SequenceFileInputFormat.class);
+            }
+        }
+    }
+
+    protected void setReduceTaskNum(Job job, KylinConfig config, String cubeName, int level) throws ClassNotFoundException, IOException, InterruptedException, JobException {
+        Configuration jobConf = job.getConfiguration();
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+
+        CubeDesc cubeDesc = CubeManager.getInstance(config).getCube(cubeName).getDescriptor();
+
+        double perReduceInputMB = kylinConfig.getDefaultHadoopJobReducerInputMB();
+        double reduceCountRatio = kylinConfig.getDefaultHadoopJobReducerCountRatio();
+
+        // total map input MB
+        double totalMapInputMB = this.getTotalMapInputMB();
+
+        // output / input ratio
+        int preLevelCuboids, thisLevelCuboids;
+        if (level == 0) { // base cuboid
+            preLevelCuboids = thisLevelCuboids = 1;
+        } else { // n-cuboid
+            int[] allLevelCount = CuboidCLI.calculateAllLevelCount(cubeDesc);
+            preLevelCuboids = allLevelCount[level - 1];
+            thisLevelCuboids = allLevelCount[level];
+        }
+
+        // total reduce input MB
+        double totalReduceInputMB = totalMapInputMB * thisLevelCuboids / preLevelCuboids;
+
+        // number of reduce tasks
+        int numReduceTasks = (int) Math.round(totalReduceInputMB / perReduceInputMB * reduceCountRatio);
+
+        // adjust reducer number for cube which has DISTINCT_COUNT measures for
+        // better performance
+        if (cubeDesc.hasHolisticCountDistinctMeasures()) {
+            numReduceTasks = numReduceTasks * 4;
+        }
+
+        // at least 1 reducer
+        numReduceTasks = Math.max(1, numReduceTasks);
+        // no more than 5000 reducer by default
+        numReduceTasks = Math.min(kylinConfig.getHadoopJobMaxReducerNumber(), numReduceTasks);
+
+        jobConf.setInt(MAPRED_REDUCE_TASKS, numReduceTasks);
+
+        logger.info("Having total map input MB " + Math.round(totalMapInputMB));
+        logger.info("Having level " + level + ", pre-level cuboids " + preLevelCuboids + ", this level cuboids " + thisLevelCuboids);
+        logger.info("Having per reduce MB " + perReduceInputMB + ", reduce count ratio " + reduceCountRatio);
+        logger.info("Setting " + MAPRED_REDUCE_TASKS + "=" + numReduceTasks);
+    }
+
+    /**
+     * @param mapperClass
+     *            the mapperClass to set
+     */
+    @SuppressWarnings("rawtypes")
+    public void setMapperClass(Class<? extends Mapper> mapperClass) {
+        this.mapperClass = mapperClass;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
new file mode 100644
index 0000000..eab967e
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.metadata.measure.MeasureAggregators;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.KylinReducer;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CuboidReducer extends KylinReducer<Text, Text, Text, Text> {
+
+    private static final Logger logger = LoggerFactory.getLogger(CuboidReducer.class);
+
+    private String cubeName;
+    private CubeDesc cubeDesc;
+    private List<MeasureDesc> measuresDescs;
+
+    private MeasureCodec codec;
+    private MeasureAggregators aggs;
+
+    private int counter;
+    private Object[] input;
+    private Object[] result;
+
+    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+    private Text outputValue = new Text();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        cubeDesc = CubeManager.getInstance(config).getCube(cubeName).getDescriptor();
+        measuresDescs = cubeDesc.getMeasures();
+
+        codec = new MeasureCodec(measuresDescs);
+        aggs = new MeasureAggregators(measuresDescs);
+
+        input = new Object[measuresDescs.size()];
+        result = new Object[measuresDescs.size()];
+    }
+
+    @Override
+    public void reduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+
+        aggs.reset();
+
+        for (Text value : values) {
+            codec.decode(ByteBuffer.wrap(value.getBytes(), 0, value.getLength()), input);
+            aggs.aggregate(input);
+        }
+        aggs.collectStates(result);
+
+        valueBuf.clear();
+        codec.encode(result, valueBuf);
+
+        outputValue.set(valueBuf.array(), 0, valueBuf.position());
+        context.write(key, outputValue);
+
+        counter++;
+        if (counter % BatchConstants.COUNTER_MAX == 0) {
+            logger.info("Handled " + counter + " records!");
+        }
+    }
+
+}



[18/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsCombiner.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsCombiner.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsCombiner.java
new file mode 100644
index 0000000..e7dfb9a
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsCombiner.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+import java.util.HashSet;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.engine.mr.KylinReducer;
+
+/**
+ * @author yangli9
+ */
+public class FactDistinctColumnsCombiner extends KylinReducer<LongWritable, Text, LongWritable, Text> {
+
+    private Text outputValue = new Text();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+    }
+
+    @Override
+    public void reduce(LongWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+
+        if(key.get() >= 0) {
+            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);
+            }
+        } else {
+            // for hll, each key only has one output, no need to do local combine;
+            outputValue.set(values.iterator().next().getBytes());
+            context.write(key, outputValue);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
new file mode 100644
index 0000000..b51eb15
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.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.steps;
+
+import java.io.IOException;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+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.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.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.MRUtil;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class FactDistinctColumnsJob extends AbstractHadoopJob {
+    protected static final Logger log = LoggerFactory.getLogger(FactDistinctColumnsJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_SEGMENT_NAME);
+            options.addOption(OPTION_STATISTICS_ENABLED);
+            options.addOption(OPTION_STATISTICS_OUTPUT);
+            options.addOption(OPTION_STATISTICS_SAMPLING_PERCENT);
+            parseOptions(options, args);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME);
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            
+            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+            String statistics_enabled = getOptionValue(OPTION_STATISTICS_ENABLED);
+            String statistics_output = getOptionValue(OPTION_STATISTICS_OUTPUT);
+            String statistics_sampling_percent = getOptionValue(OPTION_STATISTICS_SAMPLING_PERCENT);
+
+            // ----------------------------------------------------------------------------
+            // add metadata to distributed cache
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+            job.getConfiguration().set(BatchConstants.CFG_STATISTICS_ENABLED, statistics_enabled);
+            job.getConfiguration().set(BatchConstants.CFG_STATISTICS_OUTPUT, statistics_output);
+            job.getConfiguration().set(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT, statistics_sampling_percent);
+            log.info("Starting: " + job.getJobName());
+
+            setJobClasspath(job);
+
+            setupMapper(cube.getSegment(segmentName, SegmentStatusEnum.NEW));
+            setupReducer(output);
+
+            // CubeSegment seg = cubeMgr.getCube(cubeName).getTheOnlySegment();
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            return waitForCompletion(job);
+
+        } catch (Exception e) {
+            logger.error("error in FactDistinctColumnsJob", e);
+            printUsage(options);
+            throw e;
+        }
+
+    }
+
+    private void setupMapper(CubeSegment cubeSeg) throws IOException {
+        IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSeg).getFlatTableInputFormat();
+        flatTableInputFormat.configureJob(job);
+
+        job.setMapperClass(FactDistinctHiveColumnsMapper.class);
+        job.setCombinerClass(FactDistinctColumnsCombiner.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(Text.class);
+    }
+
+    private void setupReducer(Path output) throws IOException {
+        job.setReducerClass(FactDistinctColumnsReducer.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 {
+        FactDistinctColumnsJob job = new FactDistinctColumnsJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
new file mode 100644
index 0000000..592c45d
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
@@ -0,0 +1,89 @@
+package org.apache.kylin.engine.mr.steps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.RowKeyDesc;
+import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.kylin.engine.mr.MRUtil;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+
+/**
+ */
+public class FactDistinctColumnsMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VALUEIN, LongWritable, Text> {
+
+    protected String cubeName;
+    protected CubeInstance cube;
+    protected CubeSegment cubeSeg;
+    protected CubeDesc cubeDesc;
+    protected long baseCuboidId;
+    protected List<TblColRef> columns;
+    protected ArrayList<Integer> factDictCols;
+    protected IMRTableInputFormat flatTableInputFormat;
+
+    protected LongWritable outputKey = new LongWritable();
+    protected Text outputValue = new Text();
+    protected int errorRecordCounter = 0;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        bindCurrentConfiguration(conf);
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
+        cube = CubeManager.getInstance(config).getCube(cubeName);
+        cubeSeg = cube.getSegment(conf.get(BatchConstants.CFG_CUBE_SEGMENT_NAME), SegmentStatusEnum.NEW);
+        cubeDesc = cube.getDescriptor();
+        baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        columns = Cuboid.findById(cubeDesc, baseCuboidId).getColumns();
+
+        factDictCols = new ArrayList<Integer>();
+        RowKeyDesc rowKey = cubeDesc.getRowkey();
+        DictionaryManager dictMgr = DictionaryManager.getInstance(config);
+        for (int i = 0; i < columns.size(); i++) {
+            TblColRef col = columns.get(i);
+            if (!rowKey.isUseDictionary(col))
+                continue;
+
+            String scanTable = (String) dictMgr.decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0];
+            if (cubeDesc.getModel().isFactTable(scanTable)) {
+                factDictCols.add(i);
+            }
+        }
+        
+        flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSeg).getFlatTableInputFormat();
+    }
+
+    protected void handleErrorRecord(String[] record, Exception ex) throws IOException {
+
+        System.err.println("Insane record: " + Arrays.toString(record));
+        ex.printStackTrace(System.err);
+
+        errorRecordCounter++;
+        if (errorRecordCounter > BatchConstants.ERROR_RECORD_THRESHOLD) {
+            if (ex instanceof IOException)
+                throw (IOException) ex;
+            else if (ex instanceof RuntimeException)
+                throw (RuntimeException) ex;
+            else
+                throw new RuntimeException("", ex);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
new file mode 100644
index 0000000..08140e5
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.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.steps;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.*;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.KylinReducer;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+/**
+ * @author yangli9
+ */
+public class FactDistinctColumnsReducer extends KylinReducer<LongWritable, Text, NullWritable, Text> {
+
+    private List<TblColRef> columnList = new ArrayList<TblColRef>();
+    private boolean collectStatistics = false;
+    private String statisticsOutput = null;
+    private List<Long> baseCuboidRowCountInMappers;
+    protected Map<Long, HyperLogLogPlusCounter> cuboidHLLMap = null;
+    protected long baseCuboidId;
+    protected CubeDesc cubeDesc;
+    private long totalRowsBeforeMerge = 0;
+    private int SAMPING_PERCENTAGE = 5;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+        Configuration conf = context.getConfiguration();
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+        String cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+
+        baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        columnList = baseCuboid.getColumns();
+        collectStatistics = Boolean.parseBoolean(conf.get(BatchConstants.CFG_STATISTICS_ENABLED));
+        statisticsOutput = conf.get(BatchConstants.CFG_STATISTICS_OUTPUT);
+
+        if (collectStatistics) {
+            baseCuboidRowCountInMappers = Lists.newArrayList();
+            cuboidHLLMap = Maps.newHashMap();
+            SAMPING_PERCENTAGE = Integer.parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT, "5"));
+        }
+    }
+
+    @Override
+    public void reduce(LongWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+
+        if (key.get() >= 0) {
+            TblColRef col = columnList.get((int) 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, col.getName()));
+
+            try {
+                for (ByteArray value : set) {
+                    out.write(value.array(), value.offset(), value.length());
+                    out.write('\n');
+                }
+            } finally {
+                out.close();
+            }
+        } else {
+            // for hll
+            long cuboidId = 0 - key.get();
+
+            for (Text value : values) {
+                HyperLogLogPlusCounter hll = new HyperLogLogPlusCounter(14);
+                ByteArray byteArray = new ByteArray(value.getBytes());
+                hll.readRegisters(byteArray.asBuffer());
+
+                totalRowsBeforeMerge += hll.getCountEstimate();
+
+                if (cuboidId == baseCuboidId) {
+                    baseCuboidRowCountInMappers.add(hll.getCountEstimate());
+                }
+
+                if (cuboidHLLMap.get(cuboidId) != null) {
+                    cuboidHLLMap.get(cuboidId).merge(hll);
+                } else {
+                    cuboidHLLMap.put(cuboidId, hll);
+                }
+            }
+        }
+
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+
+        //output the hll info;
+        if (collectStatistics) {
+            writeMapperAndCuboidStatistics(context); // for human check
+            writeCuboidStatistics(context.getConfiguration(), new Path(statisticsOutput), cuboidHLLMap, SAMPING_PERCENTAGE); // for CreateHTableJob
+        }
+    }
+
+    private void writeMapperAndCuboidStatistics(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        FileSystem fs = FileSystem.get(conf);
+        FSDataOutputStream out = fs.create(new Path(statisticsOutput, BatchConstants.CFG_STATISTICS_CUBE_ESTIMATION));
+
+        try {
+            String msg;
+
+            List<Long> allCuboids = new ArrayList<Long>();
+            allCuboids.addAll(cuboidHLLMap.keySet());
+            Collections.sort(allCuboids);
+
+            msg = "Total cuboid number: \t" + allCuboids.size();
+            writeLine(out, msg);
+            msg = "Samping percentage: \t" + SAMPING_PERCENTAGE;
+            writeLine(out, msg);
+
+            writeLine(out, "The following statistics are collected based sampling data.");
+            for (int i = 0; i < baseCuboidRowCountInMappers.size(); i++) {
+                if (baseCuboidRowCountInMappers.get(i) > 0) {
+                    msg = "Base Cuboid in Mapper " + i + " row count: \t " + baseCuboidRowCountInMappers.get(i);
+                    writeLine(out, msg);
+                }
+            }
+
+            long grantTotal = 0;
+            for (long i : allCuboids) {
+                grantTotal += cuboidHLLMap.get(i).getCountEstimate();
+                msg = "Cuboid " + i + " row count is: \t " + cuboidHLLMap.get(i).getCountEstimate();
+                writeLine(out, msg);
+            }
+
+            msg = "Sum of all the cube segments (before merge) is: \t " + totalRowsBeforeMerge;
+            writeLine(out, msg);
+
+            msg = "After merge, the cube has row count: \t " + grantTotal;
+            writeLine(out, msg);
+
+            if (grantTotal > 0) {
+                msg = "The compaction factor is: \t" + totalRowsBeforeMerge / grantTotal;
+                writeLine(out, msg);
+            }
+
+        } finally {
+            out.close();
+        }
+    }
+
+    private void writeLine(FSDataOutputStream out, String msg) throws IOException {
+        out.write(msg.getBytes());
+        out.write('\n');
+
+    }
+
+    public static void writeCuboidStatistics(Configuration conf, Path outputPath, Map<Long, HyperLogLogPlusCounter> cuboidHLLMap, int samplingPercentage) throws IOException {
+        Path seqFilePath = new Path(outputPath, BatchConstants.CFG_STATISTICS_CUBOID_ESTIMATION);
+        SequenceFile.Writer writer = SequenceFile.createWriter(conf,
+                SequenceFile.Writer.file(seqFilePath), SequenceFile.Writer.keyClass(LongWritable.class),
+                SequenceFile.Writer.valueClass(BytesWritable.class));
+
+        List<Long> allCuboids = new ArrayList<Long>();
+        allCuboids.addAll(cuboidHLLMap.keySet());
+        Collections.sort(allCuboids);
+
+        // persist the sample percentage with key 0
+        writer.append(new LongWritable(0l), new BytesWritable(Bytes.toBytes(samplingPercentage)));
+        ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+        try {
+            for (long i : allCuboids) {
+                valueBuf.clear();
+                cuboidHLLMap.get(i).writeRegisters(valueBuf);
+                valueBuf.flip();
+                writer.append(new LongWritable(i), new BytesWritable(valueBuf.array(), valueBuf.limit()));
+            }
+        } finally {
+            writer.close();
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctHiveColumnsMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctHiveColumnsMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctHiveColumnsMapper.java
new file mode 100644
index 0000000..8ea7fd3
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctHiveColumnsMapper.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+
+import com.google.common.collect.Lists;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+
+/**
+ * @author yangli9
+ */
+public class FactDistinctHiveColumnsMapper<KEYIN> extends FactDistinctColumnsMapperBase<KEYIN, Object> {
+
+    private CubeJoinedFlatTableDesc intermediateTableDesc;
+
+    protected boolean collectStatistics = false;
+    protected CuboidScheduler cuboidScheduler = null;
+    protected int nRowKey;
+    private Integer[][] allCuboidsBitSet = null;
+    private HyperLogLogPlusCounter[] allCuboidsHLL = null;
+    private Long[] cuboidIds;
+    private HashFunction hf = null;
+    private int rowCount = 0;
+    private int SAMPING_PERCENTAGE = 5;
+    private ByteArray[] row_hashcodes = null;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.setup(context);
+        intermediateTableDesc = new CubeJoinedFlatTableDesc(cubeDesc, null);
+        collectStatistics = Boolean.parseBoolean(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_ENABLED));
+        if (collectStatistics) {
+            SAMPING_PERCENTAGE = Integer.parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT, "5"));
+            cuboidScheduler = new CuboidScheduler(cubeDesc);
+            nRowKey = cubeDesc.getRowkey().getRowKeyColumns().length;
+
+            List<Long> cuboidIdList = Lists.newArrayList();
+            List<Integer[]> allCuboidsBitSetList = Lists.newArrayList();
+            addCuboidBitSet(baseCuboidId, allCuboidsBitSetList, cuboidIdList);
+
+            allCuboidsBitSet = allCuboidsBitSetList.toArray(new Integer[cuboidIdList.size()][]);
+            cuboidIds = cuboidIdList.toArray(new Long[cuboidIdList.size()]);
+
+            allCuboidsHLL = new HyperLogLogPlusCounter[cuboidIds.length];
+            for (int i = 0; i < cuboidIds.length; i++) {
+                allCuboidsHLL[i] = new HyperLogLogPlusCounter(14);
+            }
+
+            hf = Hashing.murmur3_32();
+            row_hashcodes = new ByteArray[nRowKey];
+            for (int i = 0; i < nRowKey; i++) {
+                row_hashcodes[i] = new ByteArray();
+            }
+        }
+    }
+
+    private void addCuboidBitSet(long cuboidId, List<Integer[]> allCuboidsBitSet, List<Long> allCuboids) {
+        allCuboids.add(cuboidId);
+        BitSet bitSet = BitSet.valueOf(new long[]{cuboidId});
+        Integer[] indice = new Integer[bitSet.cardinality()];
+
+        long mask = Long.highestOneBit(baseCuboidId);
+        int position = 0;
+        for (int i = 0; i < nRowKey; i++) {
+            if ((mask & cuboidId) > 0) {
+                indice[position] = i;
+                position++;
+            }
+            mask = mask >> 1;
+        }
+
+        allCuboidsBitSet.add(indice);
+        Collection<Long> children = cuboidScheduler.getSpanningCuboid(cuboidId);
+        for (Long childId : children) {
+            addCuboidBitSet(childId, allCuboidsBitSet, allCuboids);
+        }
+    }
+
+    @Override
+    public void map(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
+        String[] row = flatTableInputFormat.parseMapperInput(record);
+        try {
+            for (int i : factDictCols) {
+                outputKey.set((long) i);
+                String fieldValue = row[intermediateTableDesc.getRowKeyColumnIndexes()[i]];
+                if (fieldValue == null)
+                    continue;
+                byte[] bytes = Bytes.toBytes(fieldValue);
+                outputValue.set(bytes, 0, bytes.length);
+                context.write(outputKey, outputValue);
+            }
+        } catch (Exception ex) {
+            handleErrorRecord(row, ex);
+        }
+
+        if (collectStatistics && rowCount < SAMPING_PERCENTAGE) {
+            putRowKeyToHLL(row);
+        }
+
+        if (rowCount++ == 100)
+            rowCount = 0;
+    }
+
+    private void putRowKeyToHLL(String[] row) {
+
+        //generate hash for each row key column
+        for (int i = 0; i < nRowKey; i++) {
+            Hasher hc = hf.newHasher();
+            String colValue = row[intermediateTableDesc.getRowKeyColumnIndexes()[i]];
+            if (colValue != null) {
+                row_hashcodes[i].set(hc.putString(colValue).hash().asBytes());
+            } else {
+                row_hashcodes[i].set(hc.putInt(0).hash().asBytes());
+            }
+        }
+
+        // user the row key column hash to get a consolidated hash for each cuboid
+        for (int i = 0, n = allCuboidsBitSet.length; i < n; i++) {
+            Hasher hc = hf.newHasher();
+            for (int position = 0; position < allCuboidsBitSet[i].length; position++) {
+                hc.putBytes(row_hashcodes[allCuboidsBitSet[i][position]].array());
+            }
+
+            allCuboidsHLL[i].add(hc.hash().asBytes());
+        }
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        if (collectStatistics) {
+            ByteBuffer hllBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+            // output each cuboid's hll to reducer, key is 0 - cuboidId
+            HyperLogLogPlusCounter hll;
+            for (int i = 0; i < cuboidIds.length; i++) {
+                hll = allCuboidsHLL[i];
+                outputKey.set(0 - cuboidIds[i]);
+                hllBuf.clear();
+                hll.writeRegisters(hllBuf);
+                outputValue.set(hllBuf.array(), 0, hllBuf.position());
+                context.write(outputKey, outputValue);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapper.java
new file mode 100644
index 0000000..266debe
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapper.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.steps;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+
+import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.MRUtil;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class HiveToBaseCuboidMapper<KEYIN> extends BaseCuboidMapperBase<KEYIN, Object> {
+    
+    private IMRTableInputFormat flatTableInputFormat;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.setup(context);
+        flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSegment).getFlatTableInputFormat();
+    }
+
+    @Override
+    public void map(KEYIN key, Object value, Context context) throws IOException, InterruptedException {
+        counter++;
+        if (counter % BatchConstants.COUNTER_MAX == 0) {
+            logger.info("Handled " + counter + " records!");
+        }
+        
+
+        try {
+            //put a record into the shared bytesSplitter
+            String[] row = flatTableInputFormat.parseMapperInput(value);
+            bytesSplitter.setBuffers(convertUTF8Bytes(row));
+            //take care of the data in bytesSplitter
+            outputKV(context);
+
+        } catch (Exception ex) {
+            handleErrorRecord(bytesSplitter, ex);
+        }
+    }
+
+    private byte[][] convertUTF8Bytes(String[] row) throws UnsupportedEncodingException {
+        byte[][] result = new byte[row.length][];
+        for (int i = 0; i < row.length; i++) {
+            result[i] = row[i].getBytes("UTF-8");
+        }
+        return result;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
new file mode 100644
index 0000000..685243c
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.ToolRunner;
+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.ByteArrayWritable;
+import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.IMROutput2.IMRStorageOutputFormat;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.MRUtil;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class InMemCuboidJob extends AbstractHadoopJob {
+
+    protected static final Logger logger = LoggerFactory.getLogger(InMemCuboidJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_JOB_FLOW_ID);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_NAME);
+            parseOptions(options, args);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            CubeManager cubeMgr = CubeManager.getInstance(config);
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            CubeSegment cubeSeg = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            logger.info("Starting: " + job.getJobName());
+            
+            setJobClasspath(job);
+            
+            // add metadata to distributed cache
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+            long timeout = 1000*60*60L; // 1 hour
+            job.getConfiguration().set("mapred.task.timeout", String.valueOf(timeout));
+            
+            // set input
+            IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSeg).getFlatTableInputFormat();
+            flatTableInputFormat.configureJob(job);
+            
+            // set mapper
+            job.setMapperClass(InMemCuboidMapper.class);
+            job.setMapOutputKeyClass(ByteArrayWritable.class);
+            job.setMapOutputValueClass(ByteArrayWritable.class);
+            
+            // set output
+            IMRStorageOutputFormat storageOutputFormat = MRUtil.getBatchCubingOutputSide2(cubeSeg).getStorageOutputFormat();
+            storageOutputFormat.configureOutput(InMemCuboidReducer.class, getOptionValue(OPTION_JOB_FLOW_ID), job);
+            
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            logger.error("error in CuboidJob", e);
+            printUsage(options);
+            throw e;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        InMemCuboidJob job = new InMemCuboidJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
new file mode 100644
index 0000000..e9340de
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
@@ -0,0 +1,122 @@
+package org.apache.kylin.engine.mr.steps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.inmemcubing.DoggedCubeBuilder;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.DimensionDesc;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+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.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.Maps;
+
+/**
+ */
+public class InMemCuboidMapper<KEYIN> extends KylinMapper<KEYIN, Object, ByteArrayWritable, ByteArrayWritable> {
+
+    private static final Log logger = LogFactory.getLog(InMemCuboidMapper.class);
+    private CubeInstance cube;
+    private CubeDesc cubeDesc;
+    private CubeSegment cubeSegment;
+    private IMRTableInputFormat flatTableInputFormat;
+
+    private int counter;
+    private BlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(10000);
+    private Future<?> future;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+        Configuration conf = context.getConfiguration();
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+        String cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
+        cube = CubeManager.getInstance(config).getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+        String segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME);
+        cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
+        flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSegment).getFlatTableInputFormat();
+
+        Map<TblColRef, Dictionary<?>> dictionaryMap = Maps.newHashMap();
+
+        for (DimensionDesc dim : cubeDesc.getDimensions()) {
+            // dictionary
+            for (TblColRef col : dim.getColumnRefs()) {
+                if (cubeDesc.getRowkey().isUseDictionary(col)) {
+                    Dictionary<?> dict = cubeSegment.getDictionary(col);
+                    if (dict == null) {
+                        logger.warn("Dictionary for " + col + " was not found.");
+                    }
+
+                    dictionaryMap.put(col, cubeSegment.getDictionary(col));
+                }
+            }
+        }
+
+        DoggedCubeBuilder cubeBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap);
+        ExecutorService executorService = Executors.newSingleThreadExecutor();
+        future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new MapContextGTRecordWriter(context, cubeDesc, cubeSegment)));
+
+    }
+
+    @Override
+    public void map(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
+        // put each row to the queue
+        String[] row = flatTableInputFormat.parseMapperInput(record);
+        List<String> rowAsList = Arrays.asList(row);
+        
+        while (!future.isDone()) {
+            if (queue.offer(rowAsList, 1, TimeUnit.SECONDS)) {
+                counter++;
+                if (counter % BatchConstants.COUNTER_MAX == 0) {
+                    logger.info("Handled " + counter + " records!");
+                }
+                break;
+            }
+        }
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        logger.info("Totally handled " + counter + " records!");
+
+        while (!future.isDone()) {
+            if (queue.offer(new ArrayList<String>(0), 1, TimeUnit.SECONDS)) {
+                break;
+            }
+        }
+        
+        try {
+            future.get();
+        } catch (Exception e) {
+            throw new IOException("Failed to build cube in mapper " + context.getTaskAttemptID().getTaskID().getId(), e);
+        }
+        queue.clear();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
new file mode 100644
index 0000000..004eb17
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
@@ -0,0 +1,95 @@
+package org.apache.kylin.engine.mr.steps;
+
+import java.io.IOException;
+import java.util.List;
+
+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.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.IMROutput2.IMRStorageOutputFormat;
+import org.apache.kylin.engine.mr.KylinReducer;
+import org.apache.kylin.engine.mr.MRUtil;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.metadata.measure.MeasureAggregators;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ */
+public class InMemCuboidReducer extends KylinReducer<ByteArrayWritable, ByteArrayWritable, Object, Object> {
+
+    private static final Logger logger = LoggerFactory.getLogger(InMemCuboidReducer.class);
+
+    private IMRStorageOutputFormat storageOutputFormat;
+    private MeasureCodec codec;
+    private MeasureAggregators aggs;
+
+    private int counter;
+    private Object[] input;
+    private Object[] result;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+        
+        String cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        String segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME);
+        boolean isMerge = Boolean.parseBoolean(context.getConfiguration().get(BatchConstants.CFG_IS_MERGE));
+
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        CubeDesc cubeDesc = cube.getDescriptor();
+        CubeSegment cubeSeg = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
+        if (isMerge)
+            storageOutputFormat = MRUtil.getBatchMergeOutputSide2(cubeSeg).getStorageOutputFormat();
+        else
+            storageOutputFormat = MRUtil.getBatchCubingOutputSide2(cubeSeg).getStorageOutputFormat();
+
+        List<MeasureDesc> measuresDescs = Lists.newArrayList();
+        for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
+            for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
+                for (MeasureDesc measure : colDesc.getMeasures()) {
+                    measuresDescs.add(measure);
+                }
+            }
+        }
+
+        codec = new MeasureCodec(measuresDescs);
+        aggs = new MeasureAggregators(measuresDescs);
+
+        input = new Object[measuresDescs.size()];
+        result = new Object[measuresDescs.size()];
+    }
+
+    @Override
+    public void reduce(ByteArrayWritable key, Iterable<ByteArrayWritable> values, Context context) throws IOException, InterruptedException {
+
+        aggs.reset();
+
+        for (ByteArrayWritable value : values) {
+            codec.decode(value.asBuffer(), input);
+            aggs.aggregate(input);
+        }
+        aggs.collectStates(result);
+        
+        storageOutputFormat.doReducerOutput(key, result, context);
+        
+        counter++;
+        if (counter % BatchConstants.COUNTER_MAX == 0) {
+            logger.info("Handled " + counter + " records!");
+        }
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionJob.java
new file mode 100644
index 0000000..686f2b2
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionJob.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.steps;
+
+///*
+// * Copyright 2013-2014 eBay Software Foundation
+// *
+// * Licensed under the Apache License, Version 2.0 (the "License");
+// * you may not use this file except in compliance with the License.
+// * You may obtain a copy of the License at
+// *
+// *   http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT 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.index.cube;
+//
+//import org.apache.commons.cli.Options;
+//import org.apache.hadoop.fs.FileSystem;
+//import org.apache.hadoop.fs.Path;
+//import org.apache.hadoop.io.LongWritable;
+//import org.apache.hadoop.io.Text;
+//import org.apache.hadoop.mapreduce.Job;
+//import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+//import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+//import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+//import org.apache.hadoop.util.ToolRunner;
+//
+//import org.apache.kylin.cube.CubeInstance;
+//import org.apache.kylin.cube.CubeManager;
+//import org.apache.kylin.cube.cuboid.Cuboid;
+//import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
+//import org.apache.kylin.cube.kv.RowKeyEncoder;
+//import org.apache.kylin.index.AbstractHadoopJob;
+//import org.apache.kylin.metadata.model.cube.CubeDesc;
+//
+///**
+// * @author xjiang
+// *
+// */
+//
+//public class KeyDistributionJob extends AbstractHadoopJob {
+//
+//    public static final String JOB_TITLE = "Kylin Row Key Distribution Job";
+//    public static final String KEY_HEADER_LENGTH = "key_header_length";
+//    public static final String KEY_COLUMN_PERCENTAGE = "key_column_percentage";
+//    public static final String KEY_SPLIT_NUMBER = "key_split_number";
+//
+//    /* (non-Javadoc)
+//     * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
+//     */
+//    @Override
+//    public int run(String[] args) throws Exception {
+//        Options options = new Options();
+//
+//        try {
+//            options.addOption(OPTION_INPUT_PATH);
+//            options.addOption(OPTION_OUTPUT_PATH);
+//            options.addOption(OPTION_METADATA_URL);
+//            options.addOption(OPTION_CUBE_NAME);
+//            options.addOption(OPTION_KEY_COLUMN_PERCENTAGE);
+//            options.addOption(OPTION_KEY_SPLIT_NUMBER);
+//            parseOptions(options, args);
+//
+//            // start job
+//            String jobName = JOB_TITLE + getOptionsAsString();
+//            System.out.println("Starting: " + jobName);
+//            Job job = Job.getInstanceFromEnv(getConf(), jobName);
+//
+//            // set job configuration - basic 
+//            setJobClasspath(job);
+//            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+//
+//            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+//            FileOutputFormat.setOutputPath(job, output);
+//            //job.getConfiguration().set("dfs.block.size", "67108864");
+//
+//            // set job configuration - key prefix size & key split number
+//            String keyColumnPercentage = getOptionValue(OPTION_KEY_COLUMN_PERCENTAGE);
+//            job.getConfiguration().set(KEY_COLUMN_PERCENTAGE, keyColumnPercentage);
+//            String metadataUrl = validateMetadataUrl(getOptionValue(OPTION_METADATA_URL));
+//            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+//            int keyHeaderLen = getKeyHeaderLength(metadataUrl, cubeName);
+//            job.getConfiguration().set(KEY_HEADER_LENGTH, String.valueOf(keyHeaderLen));
+//            job.getConfiguration().set(KEY_SPLIT_NUMBER, getOptionValue(OPTION_KEY_SPLIT_NUMBER));
+//
+//            // Mapper
+//            job.setInputFormatClass(SequenceFileInputFormat.class);
+//            job.setMapperClass(KeyDistributionMapper.class);
+//            job.setMapOutputKeyClass(Text.class);
+//            job.setMapOutputValueClass(LongWritable.class);
+//
+//            // Combiner, not needed any more as mapper now does the groping
+//            //job.setCombinerClass(KeyDistributionCombiner.class);
+//
+//            // Reducer - only one
+//            job.setReducerClass(KeyDistributionReducer.class);
+//            // use sequence file as output
+//            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+//            // key is text
+//            job.setOutputKeyClass(Text.class);
+//            // value is long
+//            job.setOutputValueClass(LongWritable.class);
+//            job.setNumReduceTasks(1);
+//
+//            FileSystem fs = FileSystem.get(job.getConfiguration());
+//            if (fs.exists(output))
+//                fs.delete(output, true);
+//
+//            return waitForCompletion(job);
+//        } catch (Exception e) {
+//            printUsage(options);
+//            e.printStackTrace(System.err);
+//            return 2;
+//        }
+//    }
+//
+//    private int getKeyHeaderLength(String metadataUrl, String cubeName) {
+//        CubeManager cubeMgr = CubeManager.getInstanceFromEnv(metadataUrl);
+//        CubeInstance cubeInstance = cubeMgr.getCube(cubeName);
+//        CubeDesc cubeDesc = cubeInstance.getDescriptor();
+//        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+//        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+//        RowKeyEncoder rowKeyEncoder =
+//                (RowKeyEncoder) AbstractRowKeyEncoder.createInstance(cubeInstance.getTheOnlySegment(),
+//                        baseCuboid);
+//
+//        return rowKeyEncoder.getHeaderLength();
+//
+//    }
+//
+//    public static void main(String[] args) throws Exception {
+//        int exitCode = ToolRunner.run(new KeyDistributionJob(), args);
+//        System.exit(exitCode);
+//    }
+// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionMapper.java
new file mode 100644
index 0000000..e08a56b
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionMapper.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+///*
+// * Copyright 2013-2014 eBay Software Foundation
+// *
+// * Licensed under the Apache License, Version 2.0 (the "License");
+// * you may not use this file except in compliance with the License.
+// * You may obtain a copy of the License at
+// *
+// *   http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT 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.index.cube;
+//
+//import java.io.IOException;
+//
+//import org.apache.hadoop.io.LongWritable;
+//import org.apache.hadoop.io.Text;
+//import org.apache.hadoop.mapreduce.Mapper;
+//
+///**
+// * @author xjiang
+// *
+// */
+//public class KeyDistributionMapper extends KylinMapper<Text, Text, Text, LongWritable> {
+//
+//    private int headerLength;
+//
+//    private Text currentKey;
+//    private long outputLong;
+//    private Text outputKey;
+//    private LongWritable outputValue;
+//    private int columnPercentage;
+//    private int allRowCount;
+//
+//    @Override
+//    protected void setup(Context context) throws IOException {
+//super.publishConfiguration(context.getConfiguration());
+
+//        String percentStr = context.getConfiguration().get(KeyDistributionJob.KEY_COLUMN_PERCENTAGE);
+//        this.columnPercentage = Integer.valueOf(percentStr).intValue();
+//        if (this.columnPercentage <= 0 || this.columnPercentage >= 100) {
+//            this.columnPercentage = 20;
+//        }
+//        String headerLenStr = context.getConfiguration().get(KeyDistributionJob.KEY_HEADER_LENGTH);
+//        this.headerLength = Integer.valueOf(headerLenStr).intValue();
+//
+//        currentKey = new Text();
+//        outputLong = 0;
+//        outputKey = new Text();
+//        outputValue = new LongWritable(1);
+//        allRowCount = 0;
+//    }
+//
+//    @Override
+//    protected void cleanup(Context context) throws IOException, InterruptedException {
+//        emit(context); // emit the last holding record
+//
+//        byte[] zerokey = new byte[] { 0 };
+//        outputKey.set(zerokey);
+//        outputValue.set(allRowCount);
+//        context.write(outputKey, outputValue);
+//    }
+//
+//    @Override
+//    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+//        byte[] bytes = key.getBytes();
+//        int columnLength = bytes.length - this.headerLength;
+//        int columnPrefixLen = columnLength * this.columnPercentage / 100;
+//        if (columnPrefixLen == 0 && columnLength > 0) {
+//            columnPrefixLen = 1;
+//        }
+//        if (columnPrefixLen > 0) {
+//            currentKey.set(bytes, 0, this.headerLength + columnPrefixLen);
+//        } else {
+//            currentKey.set(bytes);
+//        }
+//
+//        allRowCount++;
+//
+//        if (outputKey.getLength() == 0) { // first record
+//            outputKey.set(currentKey);
+//            outputLong = 1;
+//        } else if (outputKey.equals(currentKey)) { // same key, note input is sorted
+//            outputLong++;
+//        } else { // the next key
+//            emit(context);
+//            outputKey.set(currentKey);
+//            outputLong = 1;
+//        }
+//    }
+//
+//    private void emit(Context context) throws IOException, InterruptedException {
+//        if (outputLong == 0)
+//            return;
+//
+//        outputValue.set(outputLong);
+//        context.write(outputKey, outputValue);
+//    }
+// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionReducer.java
new file mode 100644
index 0000000..ed3f966
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KeyDistributionReducer.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+///*
+// * Copyright 2013-2014 eBay Software Foundation
+// *
+// * Licensed under the Apache License, Version 2.0 (the "License");
+// * you may not use this file except in compliance with the License.
+// * You may obtain a copy of the License at
+// *
+// *   http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT 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.index.cube;
+//
+//import java.io.IOException;
+//
+//import org.apache.hadoop.io.LongWritable;
+//import org.apache.hadoop.io.Text;
+//import org.apache.hadoop.mapreduce.Reducer;
+//import org.apache.hadoop.util.StringUtils;
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+//
+///**
+// * @author xjiang
+// *
+// */
+//public class KeyDistributionReducer extends KylinReducer<Text, LongWritable, Text, LongWritable> {
+//
+//    private static final Logger logger = LoggerFactory.getLogger(KeyDistributionReducer.class);
+//
+//    private LongWritable outputValue;
+//    private boolean isTotalCount;
+//    private long totalCount;
+//    private int splitNumber;
+//    private long splitQuota;
+//    private long splitRemain;
+//
+//    @Override
+//    protected void setup(Context context) throws IOException, InterruptedException {
+//        super.publishConfiguration(context.getConfiguration());
+
+//        String splitStr = context.getConfiguration().get(KeyDistributionJob.KEY_SPLIT_NUMBER);
+//        splitNumber = Integer.valueOf(splitStr).intValue();
+//        outputValue = new LongWritable();
+//        isTotalCount = true;
+//        totalCount = 0;
+//        splitQuota = 0;
+//        splitRemain = 0;
+//    }
+//
+//    @Override
+//    protected void cleanup(Context context) throws IOException, InterruptedException {
+//        logger.info("---------------");
+//        long splitCount = splitQuota - splitRemain;
+//        logger.info("Total Count = " + totalCount + ", Left Count = " + splitCount);
+//    }
+//
+//    @Override
+//    public void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException,
+//            InterruptedException {
+//
+//        // calculate split quota
+//        if (isTotalCount) {
+//            for (LongWritable count : values) {
+//                totalCount += count.get();
+//            }
+//            splitQuota = totalCount / splitNumber;
+//            splitRemain = splitQuota;
+//            isTotalCount = false;
+//            return;
+//        }
+//
+//        // output key when split quota is used up 
+//        for (LongWritable count : values) {
+//            splitRemain -= count.get();
+//        }
+//        if (splitRemain <= 0) {
+//            long splitCount = splitQuota - splitRemain;
+//            String hexKey = StringUtils.byteToHexString(key.getBytes());
+//            logger.info(hexKey + "\t\t" + splitCount);
+//
+//            outputValue.set(splitCount);
+//            context.write(key, outputValue);
+//            splitRemain = splitQuota;
+//        }
+//
+//    }
+// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MapContextGTRecordWriter.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MapContextGTRecordWriter.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MapContextGTRecordWriter.java
new file mode 100644
index 0000000..4585dd1
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MapContextGTRecordWriter.java
@@ -0,0 +1,96 @@
+package org.apache.kylin.engine.mr.steps;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.inmemcubing.ICuboidWriter;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.metadata.model.TblColRef;
+
+/**
+ */
+public class MapContextGTRecordWriter implements ICuboidWriter {
+
+    private static final Log logger = LogFactory.getLog(MapContextGTRecordWriter.class);
+    protected MapContext<?, ?, ByteArrayWritable, ByteArrayWritable> mapContext;
+    private Long lastCuboidId;
+    protected CubeSegment cubeSegment;
+    protected CubeDesc cubeDesc;
+
+    private int bytesLength;
+    private int dimensions;
+    private int measureCount;
+    private byte[] keyBuf;
+    private int[] measureColumnsIndex;
+    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+    private ByteArrayWritable outputKey = new ByteArrayWritable();
+    private ByteArrayWritable outputValue = new ByteArrayWritable();
+    private long cuboidRowCount = 0;
+
+    public MapContextGTRecordWriter(MapContext<?, ?, ByteArrayWritable, ByteArrayWritable> mapContext, CubeDesc cubeDesc, CubeSegment cubeSegment) {
+        this.mapContext = mapContext;
+        this.cubeDesc = cubeDesc;
+        this.cubeSegment = cubeSegment;
+        this.measureCount = cubeDesc.getMeasures().size();
+
+    }
+
+    @Override
+    public void write(long cuboidId, GTRecord record) throws IOException {
+
+        if (lastCuboidId == null || !lastCuboidId.equals(cuboidId)) {
+            // output another cuboid
+            initVariables(cuboidId);
+            if (lastCuboidId != null) {
+                logger.info("Cuboid " + lastCuboidId + " has " + cuboidRowCount + " rows");
+                cuboidRowCount = 0;
+            }
+        }
+
+        cuboidRowCount++;
+        int offSet = RowConstants.ROWKEY_CUBOIDID_LEN;
+        for (int x = 0; x < dimensions; x++) {
+            System.arraycopy(record.get(x).array(), record.get(x).offset(), keyBuf, offSet, record.get(x).length());
+            offSet += record.get(x).length();
+        }
+
+        //output measures
+        valueBuf.clear();
+        record.exportColumns(measureColumnsIndex, valueBuf);
+
+        outputKey.set(keyBuf, 0, offSet);
+        outputValue.set(valueBuf.array(), 0, valueBuf.position());
+        try {
+            mapContext.write(outputKey, outputValue);
+        } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private void initVariables(Long cuboidId) {
+        bytesLength = RowConstants.ROWKEY_CUBOIDID_LEN;
+        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidId);
+        for (TblColRef column : cuboid.getColumns()) {
+            bytesLength += cubeSegment.getColumnLength(column);
+        }
+
+        keyBuf = new byte[bytesLength];
+        dimensions = BitSet.valueOf(new long[]{cuboidId}).cardinality();
+        measureColumnsIndex = new int[measureCount];
+        for (int i = 0; i < measureCount; i++) {
+            measureColumnsIndex[i] = dimensions + i;
+        }
+
+        System.arraycopy(Bytes.toBytes(cuboidId), 0, keyBuf, 0, RowConstants.ROWKEY_CUBOIDID_LEN);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageJob.java
new file mode 100644
index 0000000..6525d93
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageJob.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+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.ByteArrayWritable;
+import org.apache.kylin.engine.mr.IMROutput2.IMRStorageInputFormat;
+import org.apache.kylin.engine.mr.IMROutput2.IMRStorageOutputFormat;
+import org.apache.kylin.engine.mr.MRUtil;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+
+/**
+ */
+public class MergeCuboidFromStorageJob extends CuboidJob {
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_JOB_FLOW_ID);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_NAME);
+            parseOptions(options, args);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            String segmentName = getOptionValue(OPTION_SEGMENT_NAME).toUpperCase();
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            CubeManager cubeMgr = CubeManager.getInstance(config);
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            CubeSegment cubeSeg = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
+
+            Configuration conf = this.getConf();
+
+            // start job
+            String jobName = getOptionValue(OPTION_JOB_NAME);
+            System.out.println("Starting: " + jobName);
+            job = Job.getInstance(conf, jobName);
+
+            setJobClasspath(job);
+            
+            // add metadata to distributed cache
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+            job.getConfiguration().set(BatchConstants.CFG_IS_MERGE, "true");
+
+            // configure mapper input
+            IMRStorageInputFormat storageInputFormat = MRUtil.getBatchMergeInputSide2(cubeSeg).getStorageInputFormat();
+            storageInputFormat.configureInput(MergeCuboidFromStorageMapper.class, ByteArrayWritable.class, ByteArrayWritable.class, job);
+
+            // configure reducer output
+            IMRStorageOutputFormat storageOutputFormat = MRUtil.getBatchMergeOutputSide2(cubeSeg).getStorageOutputFormat();
+            storageOutputFormat.configureOutput(InMemCuboidReducer.class, getOptionValue(OPTION_JOB_FLOW_ID), job);
+            
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            logger.error("error in MergeCuboidFromHBaseJob", e);
+            printUsage(options);
+            throw e;
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
new file mode 100644
index 0000000..2281432
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.common.util.SplittedBytes;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.common.RowKeySplitter;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.IMROutput2.IMRStorageInputFormat;
+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.metadata.measure.MeasureCodec;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+/**
+ * @author shaoshi
+ */
+public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, ByteArrayWritable, ByteArrayWritable> {
+
+    private static final Logger logger = LoggerFactory.getLogger(MergeCuboidFromStorageMapper.class);
+    
+    private KylinConfig config;
+    private String cubeName;
+    private String segmentName;
+    private CubeManager cubeManager;
+    private CubeInstance cube;
+    private CubeDesc cubeDesc;
+    private CubeSegment mergedCubeSegment;
+    private CubeSegment sourceCubeSegment; // Must be unique during a mapper's life cycle
+    private IMRStorageInputFormat storageInputFormat;
+
+    private ByteArrayWritable outputKey = new ByteArrayWritable();
+    private byte[] newKeyBuf;
+    private RowKeySplitter rowKeySplitter;
+
+    private HashMap<TblColRef, Boolean> dictsNeedMerging = new HashMap<TblColRef, Boolean>();
+
+    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+    private MeasureCodec codec;
+    private ByteArrayWritable outputValue = new ByteArrayWritable();
+
+    private Boolean checkNeedMerging(TblColRef col) throws IOException {
+        Boolean ret = dictsNeedMerging.get(col);
+        if (ret != null)
+            return ret;
+        else {
+            ret = cubeDesc.getRowkey().isUseDictionary(col);
+            if (ret) {
+                String dictTable = (String) DictionaryManager.getInstance(config).decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0];
+                ret = cubeDesc.getFactTable().equalsIgnoreCase(dictTable);
+            }
+            dictsNeedMerging.put(col, ret);
+            return ret;
+        }
+    }
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+        config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME).toUpperCase();
+
+        cubeManager = CubeManager.getInstance(config);
+        cube = cubeManager.getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+        mergedCubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
+        storageInputFormat = MRUtil.getBatchMergeInputSide2(mergedCubeSegment).getStorageInputFormat();
+
+        newKeyBuf = new byte[256];// size will auto-grow
+
+        sourceCubeSegment = storageInputFormat.findSourceSegment(context, cube);
+        logger.info(sourceCubeSegment.toString());
+
+        this.rowKeySplitter = new RowKeySplitter(sourceCubeSegment, 65, 255);
+
+        List<MeasureDesc> measuresDescs = Lists.newArrayList();
+        for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
+            for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
+                for (MeasureDesc measure : colDesc.getMeasures()) {
+                    measuresDescs.add(measure);
+                }
+            }
+        }
+        codec = new MeasureCodec(measuresDescs);
+    }
+
+    @Override
+    public void map(Object inKey, Object inValue, Context context) throws IOException, InterruptedException {
+        Pair<ByteArrayWritable, Object[]> pair = storageInputFormat.parseMapperInput(inKey, inValue);
+        ByteArrayWritable key = pair.getFirst();
+        Object[] value = pair.getSecond();
+        
+        Preconditions.checkState(key.offset() == 0);
+        
+        long cuboidID = rowKeySplitter.split(key.array(), key.length());
+        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidID);
+
+        SplittedBytes[] splittedByteses = rowKeySplitter.getSplitBuffers();
+        int bufOffset = 0;
+        BytesUtil.writeLong(cuboidID, newKeyBuf, bufOffset, RowConstants.ROWKEY_CUBOIDID_LEN);
+        bufOffset += RowConstants.ROWKEY_CUBOIDID_LEN;
+
+        for (int i = 0; i < cuboid.getColumns().size(); ++i) {
+            TblColRef col = cuboid.getColumns().get(i);
+
+            if (this.checkNeedMerging(col)) {
+                // if dictionary on fact table column, needs rewrite
+                DictionaryManager dictMgr = DictionaryManager.getInstance(config);
+                Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(col));
+                Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(col));
+
+                while (sourceDict.getSizeOfValue() > newKeyBuf.length - bufOffset || mergedDict.getSizeOfValue() > newKeyBuf.length - bufOffset) {
+                    byte[] oldBuf = newKeyBuf;
+                    newKeyBuf = new byte[2 * newKeyBuf.length];
+                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
+                }
+
+                int idInSourceDict = BytesUtil.readUnsigned(splittedByteses[i + 1].value, 0, splittedByteses[i + 1].length);
+
+                int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBuf, bufOffset);
+                int idInMergedDict;
+                if (size < 0) {
+                    idInMergedDict = mergedDict.nullId();
+                } else {
+                    idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBuf, bufOffset, size);
+                }
+                BytesUtil.writeUnsigned(idInMergedDict, newKeyBuf, bufOffset, mergedDict.getSizeOfId());
+
+                bufOffset += mergedDict.getSizeOfId();
+            } else {
+                // keep as it is
+                while (splittedByteses[i + 1].length > newKeyBuf.length - bufOffset) {
+                    byte[] oldBuf = newKeyBuf;
+                    newKeyBuf = new byte[2 * newKeyBuf.length];
+                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
+                }
+
+                System.arraycopy(splittedByteses[i + 1].value, 0, newKeyBuf, bufOffset, splittedByteses[i + 1].length);
+                bufOffset += splittedByteses[i + 1].length;
+            }
+        }
+        byte[] newKey = Arrays.copyOf(newKeyBuf, bufOffset);
+        outputKey.set(newKey, 0, newKey.length);
+
+        valueBuf.clear();
+        codec.encode(value, valueBuf);
+        outputValue.set(valueBuf.array(), 0, valueBuf.position());
+        
+        context.write(outputKey, outputValue);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidJob.java
new file mode 100644
index 0000000..ceebfc1
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidJob.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.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+
+/**
+ * @author ysong1
+ */
+public class MergeCuboidJob extends CuboidJob {
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            parseOptions(options, args);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            String segmentName = getOptionValue(OPTION_SEGMENT_NAME).toUpperCase();
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            CubeManager cubeMgr = CubeManager.getInstance(config);
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+
+            // start job
+            String jobName = getOptionValue(OPTION_JOB_NAME);
+            System.out.println("Starting: " + jobName);
+            job = Job.getInstance(getConf(), jobName);
+
+            setJobClasspath(job);
+
+            // set inputs
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            FileOutputFormat.setOutputPath(job, output);
+
+            // Mapper
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapperClass(MergeCuboidMapper.class);
+            job.setMapOutputKeyClass(Text.class);
+            job.setMapOutputValueClass(Text.class);
+
+            // Reducer - only one
+            job.setReducerClass(CuboidReducer.class);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(Text.class);
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+
+            // add metadata to distributed cache
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            setReduceTaskNum(job, config, cubeName, 0);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            logger.error("error in MergeCuboidJob", e);
+            printUsage(options);
+            throw e;
+        }
+    }
+
+}



[20/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory2.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory2.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory2.java
new file mode 100644
index 0000000..9e26378
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory2.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.metadata.model.IBuildable;
+
+/**
+ */
+public class StorageFactory2 {
+    
+    private static final IStorage dft = (IStorage) ClassUtil.newInstance("org.apache.kylin.storage.hbase.HBaseStorage");
+    
+    public static <T> T createEngineAdapter(IBuildable buildable, Class<T> engineInterface) {
+        return dft.adaptToBuildEngine(engineInterface);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java b/core-storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
new file mode 100644
index 0000000..c6b1a18
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.tuple;
+
+import java.math.BigDecimal;
+import java.util.Date;
+import java.util.List;
+
+import net.sf.ehcache.pool.sizeof.annotations.IgnoreSizeOf;
+
+import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.metadata.measure.DoubleMutable;
+import org.apache.kylin.metadata.measure.LongMutable;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.tuple.ITuple;
+
+/**
+ * @author xjiang
+ */
+public class Tuple implements ITuple {
+
+    @IgnoreSizeOf
+    private final TupleInfo info;
+    private final Object[] values;
+
+    public Tuple(TupleInfo info) {
+        this.info = info;
+        this.values = new Object[info.size()];
+    }
+
+    public List<String> getAllFields() {
+        return info.getAllFields();
+    }
+
+    public List<TblColRef> getAllColumns() {
+        return info.getAllColumns();
+    }
+
+    public Object[] getAllValues() {
+        return values;
+    }
+
+    @Override
+    public ITuple makeCopy() {
+        Tuple ret = new Tuple(this.info);
+        for (int i = 0; i < this.values.length; ++i) {
+            ret.values[i] = this.values[i];
+        }
+        return ret;
+    }
+
+    public TupleInfo getInfo() {
+        return info;
+    }
+
+    public String getFieldName(TblColRef col) {
+        return info.getFieldName(col);
+    }
+
+    public TblColRef getFieldColumn(String fieldName) {
+        return info.getColumn(fieldName);
+    }
+
+    public Object getValue(String fieldName) {
+        int index = info.getFieldIndex(fieldName);
+        return values[index];
+    }
+
+    public Object getValue(TblColRef col) {
+        int index = info.getColumnIndex(col);
+        return values[index];
+    }
+
+    public String getDataTypeName(int idx) {
+        return info.getDataTypeName(idx);
+    }
+
+    public void setDimensionValue(String fieldName, String fieldValue) {
+        setDimensionValue(info.getFieldIndex(fieldName), fieldValue);
+    }
+
+    public void setDimensionValue(int idx, String fieldValue) {
+        Object objectValue = convertOptiqCellValue(fieldValue, getDataTypeName(idx));
+        values[idx] = objectValue;
+    }
+
+    public void setMeasureValue(String fieldName, Object fieldValue) {
+        setMeasureValue(info.getFieldIndex(fieldName), fieldValue);
+    }
+
+    public void setMeasureValue(int idx, Object fieldValue) {
+        fieldValue = convertWritableToJava(fieldValue);
+
+        String dataType = getDataTypeName(idx);
+        // special handling for BigDecimal, allow double be aggregated as
+        // BigDecimal during cube build for best precision
+        if ("double".equals(dataType) && fieldValue instanceof BigDecimal) {
+            fieldValue = ((BigDecimal) fieldValue).doubleValue();
+        } else if ("integer".equals(dataType) && !(fieldValue instanceof Integer)) {
+            fieldValue = ((Number) fieldValue).intValue();
+        } else if ("float".equals(dataType) && fieldValue instanceof BigDecimal) {
+            fieldValue = ((BigDecimal) fieldValue).floatValue();
+        }
+        values[idx] = fieldValue;
+    }
+
+    private Object convertWritableToJava(Object o) {
+        if (o instanceof LongMutable)
+            o = ((LongMutable) o).get();
+        else if (o instanceof DoubleMutable)
+            o = ((DoubleMutable) o).get();
+        return o;
+    }
+
+    public boolean hasColumn(TblColRef column) {
+        return info.hasColumn(column);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        for (String field : info.getAllFields()) {
+            sb.append(field);
+            sb.append("=");
+            sb.append(getValue(field));
+            sb.append(",");
+        }
+        return sb.toString();
+    }
+
+    public static long epicDaysToMillis(int days) {
+        return 1L * days * (1000 * 3600 * 24);
+    }
+
+    public static int dateToEpicDays(String strValue) {
+        Date dateValue = DateFormat.stringToDate(strValue); // NOTE: forces GMT timezone
+        long millis = dateValue.getTime();
+        return (int) (millis / (1000 * 3600 * 24));
+    }
+
+    public static long getTs(ITuple row, TblColRef partitionCol) {
+        //ts column type differentiate
+        if (partitionCol.getDatatype().equals("date")) {
+            return Tuple.epicDaysToMillis(Integer.valueOf(row.getValue(partitionCol).toString()));
+        } else {
+            return Long.valueOf(row.getValue(partitionCol).toString());
+        }
+    }
+
+    public static Object convertOptiqCellValue(String strValue, String dataTypeName) {
+        if (strValue == null)
+            return null;
+
+        if ((strValue.equals("") || strValue.equals("\\N")) && !dataTypeName.equals("string"))
+            return null;
+
+        // TODO use data type enum instead of string comparison
+        if ("date".equals(dataTypeName)) {
+            // convert epoch time
+            return dateToEpicDays(strValue);// Optiq expects Integer instead of Long. by honma
+        } else if ("timestamp".equals(dataTypeName) || "datetime".equals(dataTypeName)) {
+            return Long.valueOf(DateFormat.stringToMillis(strValue));
+        } else if ("tinyint".equals(dataTypeName)) {
+            return Byte.valueOf(strValue);
+        } else if ("short".equals(dataTypeName) || "smallint".equals(dataTypeName)) {
+            return Short.valueOf(strValue);
+        } else if ("integer".equals(dataTypeName)) {
+            return Integer.valueOf(strValue);
+        } else if ("long".equals(dataTypeName) || "bigint".equals(dataTypeName)) {
+            return Long.valueOf(strValue);
+        } else if ("double".equals(dataTypeName)) {
+            return Double.valueOf(strValue);
+        } else if ("decimal".equals(dataTypeName)) {
+            return new BigDecimal(strValue);
+        } else if ("float".equals(dataTypeName)){
+            return Float.valueOf(strValue);
+        } else {
+            return strValue;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-storage/src/main/java/org/apache/kylin/storage/tuple/TupleInfo.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/tuple/TupleInfo.java b/core-storage/src/main/java/org/apache/kylin/storage/tuple/TupleInfo.java
new file mode 100644
index 0000000..735cc64
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/tuple/TupleInfo.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.tuple;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.metadata.model.TblColRef;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class TupleInfo {
+
+    private final Map<String, Integer> fieldMap;
+    private final Map<TblColRef, Integer> columnMap;
+    
+    private final List<String> fields;
+    private final List<TblColRef> columns;
+    private final List<String> dataTypeNames;
+
+    public TupleInfo() {
+        fieldMap = new HashMap<String, Integer>();
+        columnMap = new HashMap<TblColRef, Integer>();
+        fields = new ArrayList<String>();
+        columns = new ArrayList<TblColRef>();
+        dataTypeNames = new ArrayList<String>();
+    }
+
+    public TblColRef getColumn(String fieldName) {
+        int idx = getFieldIndex(fieldName);
+        return columns.get(idx);
+    }
+
+    public int getColumnIndex(TblColRef col) {
+        return columnMap.get(col);
+    }
+
+    public String getDataTypeName(int index) {
+        return dataTypeNames.get(index);
+    }
+
+    public int getFieldIndex(String fieldName) {
+        return fieldMap.get(fieldName);
+    }
+    
+    public boolean hasField(String fieldName) {
+        return fieldMap.containsKey(fieldName);
+    }
+
+    public String getFieldName(TblColRef col) {
+        int idx = columnMap.get(col);
+        return fields.get(idx);
+    }
+
+    public boolean hasColumn(TblColRef col) {
+        return columnMap.containsKey(col);
+    }
+
+    public void setField(String fieldName, TblColRef col, int index) {
+        fieldMap.put(fieldName, index);
+
+        if (col != null)
+            columnMap.put(col, index);
+
+        if (fields.size() > index)
+            fields.set(index, fieldName);
+        else
+            fields.add(index, fieldName);
+
+        if (columns.size() > index)
+            columns.set(index, col);
+        else
+            columns.add(index, col);
+        
+        if (dataTypeNames.size() > index)
+            dataTypeNames.set(index, col.getType().getName());
+        else
+            dataTypeNames.add(index, col.getType().getName());
+    }
+
+    public List<String> getAllFields() {
+        return fields;
+    }
+
+    public List<TblColRef> getAllColumns() {
+        return columns;
+    }
+
+    public int size() {
+        return fields.size();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/pom.xml
----------------------------------------------------------------------
diff --git a/engine-mr/pom.xml b/engine-mr/pom.xml
index a226e80..2d6f157 100644
--- a/engine-mr/pom.xml
+++ b/engine-mr/pom.xml
@@ -79,18 +79,33 @@
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-mapreduce-client-app</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-api</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.mrunit</groupId>
+            <artifactId>mrunit</artifactId>
+            <classifier>hadoop2</classifier>
+            <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..6b5cfa4
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
+import org.apache.kylin.engine.mr.IMROutput.IMRBatchCubingOutputSide;
+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.job.constant.ExecutableConstants;
+
+public class BatchCubingJobBuilder extends JobBuilderSupport {
+
+    private final IMRBatchCubingInputSide inputSide;
+    private final IMRBatchCubingOutputSide outputSide;
+
+    public BatchCubingJobBuilder(CubeSegment newSegment, String submitter) {
+        super(newSegment, submitter);
+        this.inputSide = MRUtil.getBatchCubingInputSide(seg);
+        this.outputSide = MRUtil.getBatchCubingOutputSide(seg);
+    }
+
+    public CubingJob build() {
+        final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
+        final String jobId = result.getId();
+        final String cuboidRootPath = getCuboidRootPath(jobId);
+
+        // Phase 1: Create Flat Table
+        inputSide.addStepPhase1_CreateFlatTable(result);
+
+        // Phase 2: Build Dictionary
+        result.addTask(createFactDistinctColumnsStep(jobId));
+        result.addTask(createBuildDictionaryStep(jobId));
+
+        // Phase 3: Build Cube
+        final int groupRowkeyColumnsCount = seg.getCubeDesc().getRowkey().getNCuboidBuildLevels();
+        final int totalRowkeyColumnsCount = seg.getCubeDesc().getRowkey().getRowKeyColumns().length;
+        final String[] cuboidOutputTempPath = getCuboidOutputPaths(cuboidRootPath, totalRowkeyColumnsCount, groupRowkeyColumnsCount);
+        // base cuboid step
+        result.addTask(createBaseCuboidStep(cuboidOutputTempPath, jobId));
+        // n dim cuboid steps
+        for (int i = 1; i <= groupRowkeyColumnsCount; i++) {
+            int dimNum = totalRowkeyColumnsCount - i;
+            result.addTask(createNDimensionCuboidStep(cuboidOutputTempPath, dimNum, totalRowkeyColumnsCount));
+        }
+        outputSide.addStepPhase3_BuildCube(result, cuboidRootPath);
+
+        // Phase 4: Update Metadata & Cleanup
+        result.addTask(createUpdateCubeInfoAfterBuildStep(jobId));
+        inputSide.addStepPhase4_Cleanup(result);
+        outputSide.addStepPhase4_Cleanup(result);
+
+        return result;
+    }
+
+    private MapReduceExecutable createBaseCuboidStep(String[] cuboidOutputTempPath, String jobId) {
+        // base cuboid job
+        MapReduceExecutable baseCuboidStep = new MapReduceExecutable();
+
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd, seg);
+
+        baseCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_BASE_CUBOID);
+
+        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "segmentname", seg.getName());
+        appendExecCmdParameters(cmd, "input", ""); // marks flat table input
+        appendExecCmdParameters(cmd, "output", cuboidOutputTempPath[0]);
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Base_Cuboid_Builder_" + seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "level", "0");
+
+        baseCuboidStep.setMapReduceParams(cmd.toString());
+        baseCuboidStep.setMapReduceJobClass(BaseCuboidJob.class);
+        baseCuboidStep.setCounterSaveAs(CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES);
+        return baseCuboidStep;
+    }
+
+    private MapReduceExecutable createNDimensionCuboidStep(String[] cuboidOutputTempPath, int dimNum, int totalRowkeyColumnCount) {
+        // ND cuboid job
+        MapReduceExecutable ndCuboidStep = new MapReduceExecutable();
+
+        ndCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_N_D_CUBOID + " : " + dimNum + "-Dimension");
+        StringBuilder cmd = new StringBuilder();
+
+        appendMapReduceParameters(cmd, seg);
+        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().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, "level", "" + (totalRowkeyColumnCount - dimNum));
+
+        ndCuboidStep.setMapReduceParams(cmd.toString());
+        ndCuboidStep.setMapReduceJobClass(NDCuboidJob.class);
+        return ndCuboidStep;
+    }
+
+    private String[] getCuboidOutputPaths(String cuboidRootPath, int totalRowkeyColumnCount, int groupRowkeyColumnsCount) {
+        String[] paths = new String[groupRowkeyColumnsCount + 1];
+        for (int i = 0; i <= groupRowkeyColumnsCount; i++) {
+            int dimNum = totalRowkeyColumnCount - i;
+            if (dimNum == totalRowkeyColumnCount) {
+                paths[i] = cuboidRootPath + "base_cuboid";
+            } else {
+                paths[i] = cuboidRootPath + dimNum + "d_cuboid";
+            }
+        }
+        return paths;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..abdabd8
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
+import org.apache.kylin.engine.mr.IMROutput2.IMRBatchCubingOutputSide2;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.steps.InMemCuboidJob;
+import org.apache.kylin.engine.mr.steps.SaveStatisticsStep;
+import org.apache.kylin.job.constant.ExecutableConstants;
+
+public class BatchCubingJobBuilder2 extends JobBuilderSupport {
+
+    private final IMRBatchCubingInputSide inputSide;
+    private final IMRBatchCubingOutputSide2 outputSide;
+
+    public BatchCubingJobBuilder2(CubeSegment newSegment, String submitter) {
+        super(newSegment, submitter);
+        this.inputSide = MRUtil.getBatchCubingInputSide(seg);
+        this.outputSide = MRUtil.getBatchCubingOutputSide2(seg);
+    }
+
+    public CubingJob build() {
+        final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
+        final String jobId = result.getId();
+
+        // Phase 1: Create Flat Table
+        inputSide.addStepPhase1_CreateFlatTable(result);
+
+        // Phase 2: Build Dictionary
+        result.addTask(createFactDistinctColumnsStepWithStats(jobId));
+        result.addTask(createBuildDictionaryStep(jobId));
+        result.addTask(createSaveStatisticsStep(jobId));
+        outputSide.addStepPhase2_BuildDictionary(result);
+
+        // Phase 3: Build Cube
+        result.addTask(createInMemCubingStep(jobId));
+        outputSide.addStepPhase3_BuildCube(result);
+
+        // Phase 4: Update Metadata & Cleanup
+        result.addTask(createUpdateCubeInfoAfterBuildStep(jobId));
+        inputSide.addStepPhase4_Cleanup(result);
+        outputSide.addStepPhase4_Cleanup(result);
+
+        return result;
+    }
+
+    private SaveStatisticsStep createSaveStatisticsStep(String jobId) {
+        SaveStatisticsStep result = new SaveStatisticsStep();
+        result.setName(ExecutableConstants.STEP_NAME_SAVE_STATISTICS);
+        result.setCubeName(seg.getCubeInstance().getName());
+        result.setSegmentId(seg.getUuid());
+        result.setStatisticsPath(getStatisticsPath(jobId));
+        return result;
+    }
+
+    private MapReduceExecutable createInMemCubingStep(String jobId) {
+        // base cuboid job
+        MapReduceExecutable cubeStep = new MapReduceExecutable();
+
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd, seg);
+
+        cubeStep.setName(ExecutableConstants.STEP_NAME_BUILD_IN_MEM_CUBE);
+
+        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "segmentname", seg.getName());
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Cube_Builder_" + seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "jobflowid", jobId);
+
+        cubeStep.setMapReduceParams(cmd.toString());
+        cubeStep.setMapReduceJobClass(InMemCuboidJob.class);
+        cubeStep.setCounterSaveAs(CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES + "," + CubingJob.CUBE_SIZE_BYTES);
+        return cubeStep;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..3f6201c
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr;
+
+import java.util.List;
+
+import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.IMROutput.IMRBatchMergeOutputSide;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.steps.MergeCuboidJob;
+import org.apache.kylin.job.constant.ExecutableConstants;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class BatchMergeJobBuilder extends JobBuilderSupport {
+
+    private final IMRBatchMergeOutputSide outputSide;
+
+    public BatchMergeJobBuilder(CubeSegment mergeSegment, String submitter) {
+        super(mergeSegment, submitter);
+        this.outputSide = MRUtil.getBatchMergeOutputSide(seg);
+    }
+
+    public CubingJob build() {
+        final CubingJob result = CubingJob.createMergeJob(seg, submitter, config);
+        final String jobId = result.getId();
+        final String cuboidRootPath = getCuboidRootPath(jobId);
+
+        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(seg);
+        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();
+        for (CubeSegment merging : mergingSegments) {
+            mergingSegmentIds.add(merging.getUuid());
+            mergingCuboidPaths.add(getCuboidRootPath(merging) + "*");
+        }
+
+        // Phase 1: Merge Dictionary
+        result.addTask(createMergeDictionaryStep(mergingSegmentIds));
+
+        // Phase 2: Merge Cube Files
+        String formattedPath = StringUtil.join(mergingCuboidPaths, ",");
+        result.addTask(createMergeCuboidDataStep(seg, formattedPath, cuboidRootPath));
+        outputSide.addStepPhase2_BuildCube(result, cuboidRootPath);
+
+        // Phase 3: Update Metadata & Cleanup
+        result.addTask(createUpdateCubeInfoAfterMergeStep(mergingSegmentIds, jobId));
+        outputSide.addStepPhase3_Cleanup(result);
+
+        return result;
+    }
+
+    private MapReduceExecutable createMergeCuboidDataStep(CubeSegment seg, String inputPath, String outputPath) {
+        MapReduceExecutable mergeCuboidDataStep = new MapReduceExecutable();
+        mergeCuboidDataStep.setName(ExecutableConstants.STEP_NAME_MERGE_CUBOID);
+        StringBuilder cmd = new StringBuilder();
+
+        appendMapReduceParameters(cmd, seg);
+        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "segmentname", seg.getName());
+        appendExecCmdParameters(cmd, "input", inputPath);
+        appendExecCmdParameters(cmd, "output", outputPath);
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Merge_Cuboid_" + seg.getCubeInstance().getName() + "_Step");
+
+        mergeCuboidDataStep.setMapReduceParams(cmd.toString());
+        mergeCuboidDataStep.setMapReduceJobClass(MergeCuboidJob.class);
+        return mergeCuboidDataStep;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..c0fe759
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.steps.MergeCuboidFromStorageJob;
+import org.apache.kylin.engine.mr.steps.MergeStatisticsStep;
+import org.apache.kylin.job.constant.ExecutableConstants;
+
+import java.util.List;
+
+public class BatchMergeJobBuilder2 extends JobBuilderSupport {
+
+    private final IMROutput2.IMRBatchMergeOutputSide2 outputSide;
+    
+    public BatchMergeJobBuilder2(CubeSegment mergeSegment, String submitter) {
+        super(mergeSegment, submitter);
+        this.outputSide = MRUtil.getBatchMergeOutputSide2(seg);
+    }
+
+    public CubingJob build() {
+        final CubingJob result = CubingJob.createMergeJob(seg, submitter, config);
+        final String jobId = result.getId();
+
+        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(seg);
+        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();
+        for (CubeSegment merging : mergingSegments) {
+            mergingSegmentIds.add(merging.getUuid());
+            mergingHTables.add(merging.getStorageLocationIdentifier());
+        }
+
+        // Phase 1: Merge Dictionary
+        result.addTask(createMergeDictionaryStep(mergingSegmentIds));
+        result.addTask(createMergeStatisticsStep(seg, mergingSegmentIds, getStatisticsPath(jobId)));
+        outputSide.addStepPhase1_MergeDictionary(result);
+
+        // Phase 2: Merge Cube
+        String formattedTables = StringUtil.join(mergingHTables, ",");
+        result.addTask(createMergeCuboidDataFromStorageStep(formattedTables, jobId));
+        outputSide.addStepPhase2_BuildCube(result);
+
+        // Phase 3: Update Metadata & Cleanup
+        result.addTask(createUpdateCubeInfoAfterMergeStep(mergingSegmentIds, jobId));
+        outputSide.addStepPhase3_Cleanup(result);
+
+        return result;
+    }
+
+    private MergeStatisticsStep createMergeStatisticsStep(CubeSegment seg, List<String> mergingSegmentIds, String mergedStatisticsFolder) {
+        MergeStatisticsStep result = new MergeStatisticsStep();
+        result.setName(ExecutableConstants.STEP_NAME_MERGE_STATISTICS);
+        result.setCubeName(seg.getCubeInstance().getName());
+        result.setSegmentId(seg.getUuid());
+        result.setMergingSegmentIds(mergingSegmentIds);
+        result.setMergedStatisticsPath(mergedStatisticsFolder);
+        return result;
+    }
+
+    private MapReduceExecutable createMergeCuboidDataFromStorageStep(String inputTableNames, String jobId) {
+        MapReduceExecutable mergeCuboidDataStep = new MapReduceExecutable();
+        mergeCuboidDataStep.setName(ExecutableConstants.STEP_NAME_MERGE_CUBOID);
+        StringBuilder cmd = new StringBuilder();
+
+        appendMapReduceParameters(cmd, seg);
+        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "segmentname", seg.getName());
+        appendExecCmdParameters(cmd, "jobname", "Kylin_Merge_Cuboid_" + seg.getCubeInstance().getName() + "_Step");
+        appendExecCmdParameters(cmd, "jobflowid", jobId);
+
+        mergeCuboidDataStep.setMapReduceParams(cmd.toString());
+        mergeCuboidDataStep.setMapReduceJobClass(MergeCuboidFromStorageJob.class);
+        mergeCuboidDataStep.setCounterSaveAs(",," + CubingJob.CUBE_SIZE_BYTES);
+        return mergeCuboidDataStep;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/ByteArrayWritable.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/ByteArrayWritable.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/ByteArrayWritable.java
new file mode 100644
index 0000000..37a8841
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/ByteArrayWritable.java
@@ -0,0 +1,166 @@
+package org.apache.kylin.engine.mr;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.kylin.common.util.Bytes;
+
+public class ByteArrayWritable implements WritableComparable<ByteArrayWritable> {
+
+    private byte[] data;
+    private int offset;
+    private int length;
+
+    public ByteArrayWritable() {
+        this(null, 0, 0);
+    }
+
+    public ByteArrayWritable(int capacity) {
+        this(new byte[capacity], 0, capacity);
+    }
+
+    public ByteArrayWritable(byte[] data) {
+        this(data, 0, data == null ? 0 : data.length);
+    }
+
+    public ByteArrayWritable(byte[] data, int offset, int length) {
+        this.data = data;
+        this.offset = offset;
+        this.length = length;
+    }
+
+    public byte[] array() {
+        return data;
+    }
+
+    public int offset() {
+        return offset;
+    }
+
+    public int length() {
+        return length;
+    }
+
+    public void set(byte[] array) {
+        set(array, 0, array.length);
+    }
+
+    public void set(byte[] array, int offset, int length) {
+        this.data = array;
+        this.offset = offset;
+        this.length = length;
+    }
+
+    public ByteBuffer asBuffer() {
+        if (data == null)
+            return null;
+        else if (offset == 0 && length == data.length)
+            return ByteBuffer.wrap(data);
+        else
+            return ByteBuffer.wrap(data, offset, length).slice();
+    }
+
+    @Override
+    public int hashCode() {
+        if (data == null)
+            return 0;
+        else
+            return Bytes.hashCode(data, offset, length);
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        out.writeInt(this.length);
+        out.write(this.data, this.offset, this.length);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        this.length = in.readInt();
+        this.data = new byte[this.length];
+        in.readFully(this.data, 0, this.length);
+        this.offset = 0;
+    }
+
+    // Below methods copied from BytesWritable
+    /**
+     * Define the sort order of the BytesWritable.
+     * @param that The other bytes writable
+     * @return Positive if left is bigger than right, 0 if they are equal, and
+     *         negative if left is smaller than right.
+     */
+    public int compareTo(ByteArrayWritable that) {
+        return WritableComparator.compareBytes(this.data, this.offset, this.length, that.data, that.offset, that.length);
+    }
+
+    /**
+     * Compares the bytes in this object to the specified byte array
+     * @param that
+     * @return Positive if left is bigger than right, 0 if they are equal, and
+     *         negative if left is smaller than right.
+     */
+    public int compareTo(final byte[] that) {
+        return WritableComparator.compareBytes(this.data, this.offset, this.length, that, 0, that.length);
+    }
+
+    /**
+     * @see java.lang.Object#equals(java.lang.Object)
+     */
+    @Override
+    public boolean equals(Object right_obj) {
+        if (right_obj instanceof byte[]) {
+            return compareTo((byte[]) right_obj) == 0;
+        }
+        if (right_obj instanceof ByteArrayWritable) {
+            return compareTo((ByteArrayWritable) right_obj) == 0;
+        }
+        return false;
+    }
+
+    /**
+     * @see java.lang.Object#toString()
+     */
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder(3 * this.length);
+        final int endIdx = this.offset + this.length;
+        for (int idx = this.offset; idx < endIdx; idx++) {
+            sb.append(' ');
+            String num = Integer.toHexString(0xff & this.data[idx]);
+            // if it is only one digit, add a leading 0.
+            if (num.length() < 2) {
+                sb.append('0');
+            }
+            sb.append(num);
+        }
+        return sb.length() > 0 ? sb.substring(1) : "";
+    }
+
+    /** A Comparator optimized for ByteArrayWritable.
+     */
+    public static class Comparator extends WritableComparator {
+        private BytesWritable.Comparator comparator = new BytesWritable.Comparator();
+
+        /** constructor */
+        public Comparator() {
+            super(ByteArrayWritable.class);
+        }
+
+        /**
+         * @see org.apache.hadoop.io.WritableComparator#compare(byte[], int, int, byte[], int, int)
+         */
+        @Override
+        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+            return comparator.compare(b1, s1, l1, b2, s2, l2);
+        }
+    }
+
+    static { // register this comparator
+        WritableComparator.define(ByteArrayWritable.class, new Comparator());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
new file mode 100644
index 0000000..7251730
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecutableState;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.job.execution.Output;
+
+/**
+ */
+public class CubingJob extends DefaultChainedExecutable {
+
+    // KEYS of Output.extraInfo map, info passed across job steps
+    public static final String SOURCE_RECORD_COUNT = "sourceRecordCount";
+    public static final String SOURCE_SIZE_BYTES = "sourceSizeBytes";
+    public static final String CUBE_SIZE_BYTES = "byteSizeBytes";
+    public static final String MAP_REDUCE_WAIT_TIME = "mapReduceWaitTime";
+    
+    private static final String CUBE_INSTANCE_NAME = "cubeName";
+    private static final String SEGMENT_ID = "segmentId";
+    
+    public static CubingJob createBuildJob(CubeSegment seg, String submitter, JobEngineConfig config) {
+        return initCubingJob(seg, "BUILD", submitter, config);
+    }
+    
+    public static CubingJob createMergeJob(CubeSegment seg, String submitter, JobEngineConfig config) {
+        return initCubingJob(seg, "MERGE", submitter, config);
+    }
+    
+    private static CubingJob initCubingJob(CubeSegment seg, String jobType, String submitter, JobEngineConfig config) {
+        CubingJob result = new CubingJob();
+        SimpleDateFormat format = new SimpleDateFormat("z yyyy-MM-dd HH:mm:ss");
+        format.setTimeZone(TimeZone.getTimeZone(config.getTimeZone()));
+        result.setCubeName(seg.getCubeInstance().getName());
+        result.setSegmentId(seg.getUuid());
+        result.setName(seg.getCubeInstance().getName() + " - " + seg.getName() + " - " + jobType + " - " + format.format(new Date(System.currentTimeMillis())));
+        result.setSubmitter(submitter);
+        result.setNotifyList(seg.getCubeInstance().getDescriptor().getNotifyList());
+        return result;
+    }
+
+    public CubingJob() {
+        super();
+    }
+    
+    void setCubeName(String name) {
+        setParam(CUBE_INSTANCE_NAME, name);
+    }
+
+    public String getCubeName() {
+        return getParam(CUBE_INSTANCE_NAME);
+    }
+
+    void setSegmentId(String segmentId) {
+        setParam(SEGMENT_ID, segmentId);
+    }
+
+    public String getSegmentId() {
+        return getParam(SEGMENT_ID);
+    }
+
+    @Override
+    protected Pair<String, String> formatNotifications(ExecutableContext context, ExecutableState state) {
+        CubeInstance cubeInstance = CubeManager.getInstance(context.getConfig()).getCube(getCubeName());
+        final Output output = jobService.getOutput(getId());
+        String logMsg;
+        state = output.getState();
+        if (state != ExecutableState.ERROR &&
+                !cubeInstance.getDescriptor().getStatusNeedNotify().contains(state.toString().toLowerCase())) {
+            logger.info("state:" + state + " no need to notify users");
+            return null;
+        }
+        switch (state) {
+            case ERROR:
+                logMsg = output.getVerboseMsg();
+                break;
+            case DISCARDED:
+                logMsg = "job has been discarded";
+                break;
+            case SUCCEED:
+                logMsg = "job has succeeded";
+                break;
+            default:
+                return null;
+        }
+        if (logMsg == null) {
+            logMsg = "no error message";
+        }
+        String content = ExecutableConstants.NOTIFY_EMAIL_TEMPLATE;
+        content = content.replaceAll("\\$\\{job_name\\}", getName());
+        content = content.replaceAll("\\$\\{result\\}", state.toString());
+        content = content.replaceAll("\\$\\{cube_name\\}", getCubeName());
+        content = content.replaceAll("\\$\\{start_time\\}", new Date(getStartTime()).toString());
+        content = content.replaceAll("\\$\\{duration\\}", getDuration() / 60000 + "mins");
+        content = content.replaceAll("\\$\\{mr_waiting\\}", getMapReduceWaitTime() / 60000 + "mins");
+        content = content.replaceAll("\\$\\{last_update_time\\}", new Date(getLastModified()).toString());
+        content = content.replaceAll("\\$\\{submitter\\}", getSubmitter());
+        content = content.replaceAll("\\$\\{error_log\\}", logMsg);
+
+        try {
+            InetAddress inetAddress = InetAddress.getLocalHost();
+            content = content.replaceAll("\\$\\{job_engine\\}", inetAddress.getCanonicalHostName());
+        } catch (UnknownHostException e) {
+            logger.warn(e.getLocalizedMessage(), e);
+        }
+
+        String title = "["+ state.toString() + "] - [Kylin Cube Build Job]-" + getCubeName();
+        return Pair.of(title, content);
+    }
+
+    @Override
+    protected void onExecuteFinished(ExecuteResult result, ExecutableContext executableContext) {
+        long time = 0L;
+        for (AbstractExecutable task: getTasks()) {
+            final ExecutableState status = task.getStatus();
+            if (status != ExecutableState.SUCCEED) {
+                break;
+            }
+            if (task instanceof MapReduceExecutable) {
+                time += ((MapReduceExecutable) task).getMapReduceWaitTime();
+            }
+        }
+        setMapReduceWaitTime(time);
+        super.onExecuteFinished(result, executableContext);
+    }
+
+    public long getMapReduceWaitTime() {
+        return getExtraInfoAsLong(MAP_REDUCE_WAIT_TIME, 0L);
+    }
+
+    public void setMapReduceWaitTime(long t) {
+        addExtraInfo(MAP_REDUCE_WAIT_TIME, t + "");
+    }
+    
+    public long findSourceRecordCount() {
+        return Long.parseLong(findExtraInfo(SOURCE_RECORD_COUNT, "0"));
+    }
+    
+    public long findSourceSizeBytes() {
+        return Long.parseLong(findExtraInfo(SOURCE_SIZE_BYTES, "0"));
+    }
+    
+    public long findCubeSizeBytes() {
+        return Long.parseLong(findExtraInfo(CUBE_SIZE_BYTES, "0"));
+    }
+    
+    private String findExtraInfo(String key, String dft) {
+        for (AbstractExecutable child : getTasks()) {
+            Output output = executableManager.getOutput(child.getId());
+            String value = output.getExtra().get(key);
+            if (value != null)
+                return value;
+        }
+        return dft;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..0c39398
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.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;
+
+/**
+ * Any ITableSource that wishes to serve as input of MapReduce build engine must adapt to this interface.
+ */
+public interface IMRInput {
+
+    /** Return a helper to participate in batch cubing job flow. */
+    public IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg);
+
+    /** Return an InputFormat that reads from specified table. */
+    public IMRTableInputFormat getTableInputFormat(TableDesc table);
+    
+    /**
+     * Utility that configures mapper to read from a table.
+     */
+    public interface IMRTableInputFormat {
+        
+        /** Configure the InputFormat of given job. */
+        public void configureJob(Job job);
+        
+        /** Parse a mapper input object into column values. */
+        public String[] parseMapperInput(Object mapperInput);
+    }
+    
+    /**
+     * Participate the batch cubing flow as the input side. Responsible for creating
+     * intermediate flat table (Phase 1) and clean up any leftover (Phase 4).
+     * 
+     * - Phase 1: Create Flat Table
+     * - Phase 2: Build Dictionary (with FlatTableInputFormat)
+     * - Phase 3: Build Cube (with FlatTableInputFormat)
+     * - Phase 4: Update Metadata & Cleanup
+     */
+    public interface IMRBatchCubingInputSide {
+        
+        /** Return an InputFormat that reads from the intermediate flat table */
+        public IMRTableInputFormat getFlatTableInputFormat();
+        
+        /** Add step that creates an intermediate flat table as defined by CubeJoinedFlatTableDesc */
+        public void addStepPhase1_CreateFlatTable(DefaultChainedExecutable jobFlow);
+        
+        /** Add step that does necessary clean up, like delete the intermediate flat table */
+        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..bc6ee1f
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+
+public interface IMROutput {
+
+    /** Return a helper to participate in batch cubing job flow. */
+    public IMRBatchCubingOutputSide getBatchCubingOutputSide(CubeSegment seg);
+    
+    /**
+     * Participate the batch cubing flow as the output side. Responsible for saving
+     * the cuboid output to storage (Phase 3).
+     * 
+     * - Phase 1: Create Flat Table
+     * - Phase 2: Build Dictionary
+     * - Phase 3: Build Cube
+     * - Phase 4: Update Metadata & Cleanup
+     */
+    public interface IMRBatchCubingOutputSide {
+        
+        /**
+         * Add step that saves cuboid output from HDFS to storage.
+         * 
+         * The cuboid output is a directory of sequence files, where key takes format "CUBOID,D1,D2,..,Dn", 
+         * value takes format "M1,M2,..,Mm". CUBOID is 8 bytes cuboid ID; Dx is dimension value with
+         * dictionary encoding; Mx is measure value serialization form.
+         */
+        public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow, String cuboidRootPath);
+        
+        /** Add step that does any necessary clean up. */
+        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow);
+    }
+    
+    /** Return a helper to participate in batch merge job flow. */
+    public IMRBatchMergeOutputSide getBatchMergeOutputSide(CubeSegment seg);
+    
+    /**
+     * Participate the batch cubing flow as the output side. Responsible for saving
+     * the cuboid output to storage (Phase 2).
+     * 
+     * - Phase 1: Merge Dictionary
+     * - Phase 2: Merge Cube
+     * - Phase 3: Update Metadata & Cleanup
+     */
+    public interface IMRBatchMergeOutputSide {
+        
+        /**
+         * Add step that saves cuboid output from HDFS to storage.
+         * 
+         * The cuboid output is a directory of sequence files, where key takes format "CUBOID,D1,D2,..,Dn", 
+         * value takes format "M1,M2,..,Mm". CUBOID is 8 bytes cuboid ID; Dx is dimension value with
+         * dictionary encoding; Mx is measure value serialization form.
+         */
+        public void addStepPhase2_BuildCube(DefaultChainedExecutable jobFlow, String cuboidRootPath);
+        
+        /** Add step that does any necessary clean up. */
+        public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
new file mode 100644
index 0000000..974e2fc
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
@@ -0,0 +1,88 @@
+package org.apache.kylin.engine.mr;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+
+public interface IMROutput2 {
+
+    /** Return a helper to participate in batch cubing job flow. */
+    public IMRBatchCubingOutputSide2 getBatchCubingOutputSide(CubeSegment seg);
+
+    /**
+     * Participate the batch cubing flow as the output side.
+     * 
+     * - Phase 1: Create Flat Table
+     * - Phase 2: Build Dictionary
+     * - Phase 3: Build Cube (with StorageOutputFormat)
+     * - Phase 4: Update Metadata & Cleanup
+     */
+    public interface IMRBatchCubingOutputSide2 {
+
+        public IMRStorageOutputFormat getStorageOutputFormat();
+
+        /** Add step that executes after build dictionary and before build cube. */
+        public void addStepPhase2_BuildDictionary(DefaultChainedExecutable jobFlow);
+
+        /** Add step that executes after build cube. */
+        public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow);
+
+        /** Add step that does any necessary clean up. */
+        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow);
+    }
+
+    public IMRBatchMergeInputSide2 getBatchMergeInputSide(CubeSegment seg);
+
+    public interface IMRBatchMergeInputSide2 {
+        public IMRStorageInputFormat getStorageInputFormat();
+    }
+
+    @SuppressWarnings("rawtypes")
+    public interface IMRStorageInputFormat {
+        
+        public void configureInput(Class<? extends Mapper> mapper, Class<? extends WritableComparable> outputKeyClass, Class<? extends Writable> outputValueClass, Job job) throws IOException;
+        
+        public CubeSegment findSourceSegment(Mapper.Context context, CubeInstance cubeInstance) throws IOException;
+        
+        public Pair<ByteArrayWritable, Object[]> parseMapperInput(Object inKey, Object inValue);
+    }
+
+    /** Return a helper to participate in batch merge job flow. */
+    public IMRBatchMergeOutputSide2 getBatchMergeOutputSide(CubeSegment seg);
+
+    /**
+     * Participate the batch merge flow as the output side.
+     * 
+     * - Phase 1: Merge Dictionary
+     * - Phase 2: Merge Cube (with StorageInputFormat & StorageOutputFormat)
+     * - Phase 3: Update Metadata & Cleanup
+     */
+    public interface IMRBatchMergeOutputSide2 {
+
+        public IMRStorageOutputFormat getStorageOutputFormat();
+
+        /** Add step that executes after merge dictionary and before merge cube. */
+        public void addStepPhase1_MergeDictionary(DefaultChainedExecutable jobFlow);
+
+        /** Add step that executes after merge cube. */
+        public void addStepPhase2_BuildCube(DefaultChainedExecutable jobFlow);
+
+        /** Add step that does any necessary clean up. */
+        public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow);
+    }
+
+    @SuppressWarnings("rawtypes")
+    public interface IMRStorageOutputFormat {
+        public void configureOutput(Class<? extends Reducer> reducer, String jobFlowId, Job job) throws IOException;
+        
+        public void doReducerOutput(ByteArrayWritable key, Object[] value, Reducer.Context context) throws IOException, InterruptedException;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..5e23531
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr;
+
+import java.io.IOException;
+import java.util.List;
+
+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.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.job.constant.ExecutableConstants;
+import org.apache.kylin.job.engine.JobEngineConfig;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Hold reusable steps for builders.
+ */
+public class JobBuilderSupport {
+
+    final protected JobEngineConfig config;
+    final protected CubeSegment seg;
+    final protected String submitter;
+
+    public JobBuilderSupport(CubeSegment seg, String submitter) {
+        Preconditions.checkNotNull(seg, "segment cannot be null");
+        this.config = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
+        this.seg = seg;
+        this.submitter = submitter;
+    }
+    
+    public MapReduceExecutable createFactDistinctColumnsStep(String jobId) {
+        return createFactDistinctColumnsStep(jobId, false);
+    }
+    
+    public MapReduceExecutable createFactDistinctColumnsStepWithStats(String jobId) {
+        return createFactDistinctColumnsStep(jobId, true);
+    }
+    
+    private MapReduceExecutable createFactDistinctColumnsStep(String jobId, boolean withStats) {
+        MapReduceExecutable result = new MapReduceExecutable();
+        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());
+        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");
+
+        result.setMapReduceParams(cmd.toString());
+        return result;
+    }
+
+    public HadoopShellExecutable createBuildDictionaryStep(String jobId) {
+        // base cuboid job
+        HadoopShellExecutable buildDictionaryStep = new HadoopShellExecutable();
+        buildDictionaryStep.setName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY);
+        StringBuilder cmd = new StringBuilder();
+        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "segmentname", seg.getName());
+        appendExecCmdParameters(cmd, "input", getFactDistinctColumnsPath(jobId));
+
+        buildDictionaryStep.setJobParams(cmd.toString());
+        buildDictionaryStep.setJobClass(CreateDictionaryJob.class);
+        return buildDictionaryStep;
+    }
+
+    public UpdateCubeInfoAfterBuildStep createUpdateCubeInfoAfterBuildStep(String jobId) {
+        final UpdateCubeInfoAfterBuildStep updateCubeInfoStep = new UpdateCubeInfoAfterBuildStep();
+        updateCubeInfoStep.setName(ExecutableConstants.STEP_NAME_UPDATE_CUBE_INFO);
+        updateCubeInfoStep.setCubeName(seg.getCubeInstance().getName());
+        updateCubeInfoStep.setSegmentId(seg.getUuid());
+        updateCubeInfoStep.setCubingJobId(jobId);
+        return updateCubeInfoStep;
+    }
+
+    public MergeDictionaryStep createMergeDictionaryStep(List<String> mergingSegmentIds) {
+        MergeDictionaryStep result = new MergeDictionaryStep();
+        result.setName(ExecutableConstants.STEP_NAME_MERGE_DICTIONARY);
+        result.setCubeName(seg.getCubeInstance().getName());
+        result.setSegmentId(seg.getUuid());
+        result.setMergingSegmentIds(mergingSegmentIds);
+        return result;
+    }
+    
+    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.setSegmentId(seg.getUuid());
+        result.setMergingSegmentIds(mergingSegmentIds);
+        result.setCubingJobId(jobId);
+        return result;
+    }
+
+    // ============================================================================
+
+    public String getJobWorkingDir(String jobId) {
+        return getJobWorkingDir(config, jobId);
+    }
+    
+    public String getCuboidRootPath(String jobId) {
+        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/cuboid/";
+    }
+    
+    public String getCuboidRootPath(CubeSegment seg) {
+        return getCuboidRootPath(seg.getLastBuildJobID());
+    }
+    
+    public void appendMapReduceParameters(StringBuilder buf, CubeSegment seg) {
+        try {
+            String jobConf = config.getHadoopJobConfFilePath(seg.getCubeDesc().getModel().getCapacity());
+            if (jobConf != null && jobConf.length() > 0) {
+                buf.append(" -conf ").append(jobConf);
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+    
+    public String getFactDistinctColumnsPath(String jobId) {
+        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/fact_distinct_columns";
+    }
+
+
+    public String getStatisticsPath(String jobId) {
+        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/statistics";
+    }
+
+    // ============================================================================
+    // static methods also shared by other job flow participant
+    // ----------------------------------------------------------------------------
+
+    public static String getJobWorkingDir(JobEngineConfig conf, String jobId) {
+        return conf.getHdfsWorkingDirectory() + "/" + "kylin-" + jobId;
+    }
+
+    public static StringBuilder appendExecCmdParameters(StringBuilder buf, String paraName, String paraValue) {
+        return buf.append(" -").append(paraName).append(" ").append(paraValue);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
new file mode 100644
index 0000000..61328c9
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.IBatchCubingEngine;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+
+public class MRBatchCubingEngine implements IBatchCubingEngine {
+
+    @Override
+    public DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter) {
+        return new BatchCubingJobBuilder(newSegment, submitter).build();
+    }
+
+    @Override
+    public DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter) {
+        return new BatchMergeJobBuilder(mergeSegment, submitter).build();
+    }
+    
+    @Override
+    public Class<?> getSourceInterface() {
+        return IMRInput.class;
+    }
+
+    @Override
+    public Class<?> getStorageInterface() {
+        return IMROutput.class;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
new file mode 100644
index 0000000..57ec128
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.IBatchCubingEngine;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+
+public class MRBatchCubingEngine2 implements IBatchCubingEngine {
+
+    @Override
+    public DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter) {
+        return new BatchCubingJobBuilder2(newSegment, submitter).build();
+    }
+
+    @Override
+    public DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter) {
+        return new BatchMergeJobBuilder2(mergeSegment, submitter).build();
+    }
+    
+    @Override
+    public Class<?> getSourceInterface() {
+        return IMRInput.class;
+    }
+
+    @Override
+    public Class<?> getStorageInterface() {
+        return IMROutput2.class;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..dc0533e
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
@@ -0,0 +1,55 @@
+package org.apache.kylin.engine.mr;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
+import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.IMROutput.IMRBatchCubingOutputSide;
+import org.apache.kylin.engine.mr.IMROutput.IMRBatchMergeOutputSide;
+import org.apache.kylin.engine.mr.IMROutput2.IMRBatchCubingOutputSide2;
+import org.apache.kylin.engine.mr.IMROutput2.IMRBatchMergeInputSide2;
+import org.apache.kylin.engine.mr.IMROutput2.IMRBatchMergeOutputSide2;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.source.TableSourceFactory;
+import org.apache.kylin.storage.StorageFactory2;
+
+public class MRUtil {
+
+    public static IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg) {
+        return TableSourceFactory.createEngineAdapter(seg, IMRInput.class).getBatchCubingInputSide(seg);
+    }
+
+    public static IMRTableInputFormat getTableInputFormat(String tableName) {
+        return getTableInputFormat(getTableDesc(tableName));
+    }
+
+    public static IMRTableInputFormat getTableInputFormat(TableDesc tableDesc) {
+        return TableSourceFactory.createEngineAdapter(tableDesc, IMRInput.class).getTableInputFormat(tableDesc);
+    }
+
+    private static TableDesc getTableDesc(String tableName) {
+        return MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(tableName);
+    }
+
+    public static IMRBatchCubingOutputSide getBatchCubingOutputSide(CubeSegment seg) {
+        return StorageFactory2.createEngineAdapter(seg, IMROutput.class).getBatchCubingOutputSide(seg);
+    }
+
+    public static IMRBatchMergeOutputSide getBatchMergeOutputSide(CubeSegment seg) {
+        return StorageFactory2.createEngineAdapter(seg, IMROutput.class).getBatchMergeOutputSide(seg);
+    }
+
+    public static IMRBatchCubingOutputSide2 getBatchCubingOutputSide2(CubeSegment seg) {
+        return StorageFactory2.createEngineAdapter(seg, IMROutput2.class).getBatchCubingOutputSide(seg);
+    }
+    
+    public static IMRBatchMergeInputSide2 getBatchMergeInputSide2(CubeSegment seg) {
+        return StorageFactory2.createEngineAdapter(seg, IMROutput2.class).getBatchMergeInputSide(seg);
+    }
+    
+    public static IMRBatchMergeOutputSide2 getBatchMergeOutputSide2(CubeSegment seg) {
+        return StorageFactory2.createEngineAdapter(seg, IMROutput2.class).getBatchMergeOutputSide(seg);
+    }
+    
+}


[25/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
new file mode 100644
index 0000000..85ac47a
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
@@ -0,0 +1,661 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.common.util.MemoryBudgetController;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.gridtable.CubeGridTable;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.gridtable.GTAggregateScanner;
+import org.apache.kylin.gridtable.GTBuilder;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.GridTable;
+import org.apache.kylin.gridtable.IGTScanner;
+import org.apache.kylin.metadata.measure.DoubleMutable;
+import org.apache.kylin.metadata.measure.LongMutable;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * Build a cube (many cuboids) in memory. Calculating multiple cuboids at the same time as long as memory permits.
+ * Assumes base cuboid fits in memory or otherwise OOM exception will occur.
+ */
+public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
+
+    private static Logger logger = LoggerFactory.getLogger(InMemCubeBuilder.class);
+    private static final LongMutable ONE = new LongMutable(1l);
+
+    private final CuboidScheduler cuboidScheduler;
+    private final long baseCuboidId;
+    private final int totalCuboidCount;
+    private final CubeJoinedFlatTableDesc intermediateTableDesc;
+    private final MeasureCodec measureCodec;
+    private final String[] metricsAggrFuncs;
+    private final int[] hbaseMeasureRefIndex;
+    private final MeasureDesc[] measureDescs;
+    private final int measureCount;
+
+    private MemoryBudgetController memBudget;
+    private Thread[] taskThreads;
+    private Throwable[] taskThreadExceptions;
+    private LinkedBlockingQueue<CuboidTask> taskPending;
+    private AtomicInteger taskCuboidCompleted = new AtomicInteger(0);
+
+    private CuboidResult baseResult;
+    private Object[] totalSumForSanityCheck;
+    private ICuboidCollector resultCollector;
+
+    public InMemCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<?>> dictionaryMap) {
+        super(cubeDesc, dictionaryMap);
+        this.cuboidScheduler = new CuboidScheduler(cubeDesc);
+        this.baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        this.totalCuboidCount = cuboidScheduler.getCuboidCount();
+        this.intermediateTableDesc = new CubeJoinedFlatTableDesc(cubeDesc, null);
+        this.measureCodec = new MeasureCodec(cubeDesc.getMeasures());
+
+        Map<String, Integer> measureIndexMap = Maps.newHashMap();
+        List<String> metricsAggrFuncsList = Lists.newArrayList();
+        measureCount = cubeDesc.getMeasures().size();
+
+        List<MeasureDesc> measureDescsList = Lists.newArrayList();
+        hbaseMeasureRefIndex = new int[measureCount];
+        int measureRef = 0;
+        for (HBaseColumnFamilyDesc familyDesc : cubeDesc.getHbaseMapping().getColumnFamily()) {
+            for (HBaseColumnDesc hbaseColDesc : familyDesc.getColumns()) {
+                for (MeasureDesc measure : hbaseColDesc.getMeasures()) {
+                    for (int j = 0; j < measureCount; j++) {
+                        if (cubeDesc.getMeasures().get(j).equals(measure)) {
+                            measureDescsList.add(measure);
+                            hbaseMeasureRefIndex[measureRef] = j;
+                            break;
+                        }
+                    }
+                    measureRef++;
+                }
+            }
+        }
+
+        for (int i = 0; i < measureCount; i++) {
+            MeasureDesc measureDesc = measureDescsList.get(i);
+            metricsAggrFuncsList.add(measureDesc.getFunction().getExpression());
+            measureIndexMap.put(measureDesc.getName(), i);
+        }
+        this.metricsAggrFuncs = metricsAggrFuncsList.toArray(new String[metricsAggrFuncsList.size()]);
+        this.measureDescs = cubeDesc.getMeasures().toArray(new MeasureDesc[measureCount]);
+    }
+
+    private GridTable newGridTableByCuboidID(long cuboidID) throws IOException {
+        GTInfo info = CubeGridTable.newGTInfo(cubeDesc, cuboidID, dictionaryMap);
+
+        // Below several store implementation are very similar in performance. The ConcurrentDiskStore is the simplest.
+        // MemDiskStore store = new MemDiskStore(info, memBudget == null ? MemoryBudgetController.ZERO_BUDGET : memBudget);
+        // MemDiskStore store = new MemDiskStore(info, MemoryBudgetController.ZERO_BUDGET);
+        ConcurrentDiskStore store = new ConcurrentDiskStore(info);
+
+        GridTable gridTable = new GridTable(info, store);
+        return gridTable;
+    }
+
+    private Pair<ImmutableBitSet, ImmutableBitSet> getDimensionAndMetricColumnBitSet(long cuboidId) {
+        BitSet bitSet = BitSet.valueOf(new long[] { cuboidId });
+        BitSet dimension = new BitSet();
+        dimension.set(0, bitSet.cardinality());
+        BitSet metrics = new BitSet();
+        metrics.set(bitSet.cardinality(), bitSet.cardinality() + this.measureCount);
+        return new Pair<ImmutableBitSet, ImmutableBitSet>(new ImmutableBitSet(dimension), new ImmutableBitSet(metrics));
+    }
+
+    @Override
+    public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
+        ConcurrentNavigableMap<Long, CuboidResult> result = build(input);
+        for (CuboidResult cuboidResult : result.values()) {
+            outputCuboid(cuboidResult.cuboidId, cuboidResult.table, output);
+            cuboidResult.table.close();
+        }
+    }
+
+    ConcurrentNavigableMap<Long, CuboidResult> build(BlockingQueue<List<String>> input) throws IOException {
+        final ConcurrentNavigableMap<Long, CuboidResult> result = new ConcurrentSkipListMap<Long, CuboidResult>();
+        build(input, new ICuboidCollector() {
+            @Override
+            public void collect(CuboidResult cuboidResult) {
+                result.put(cuboidResult.cuboidId, cuboidResult);
+            }
+        });
+        return result;
+    }
+
+    interface ICuboidCollector {
+        void collect(CuboidResult result);
+    }
+
+    static class CuboidResult {
+        public long cuboidId;
+        public GridTable table;
+        public int nRows;
+        public long timeSpent;
+        public int aggrCacheMB;
+
+        public CuboidResult(long cuboidId, GridTable table, int nRows, long timeSpent, int aggrCacheMB) {
+            this.cuboidId = cuboidId;
+            this.table = table;
+            this.nRows = nRows;
+            this.timeSpent = timeSpent;
+            this.aggrCacheMB = aggrCacheMB;
+        }
+    }
+
+    private void build(BlockingQueue<List<String>> input, ICuboidCollector collector) throws IOException {
+        long startTime = System.currentTimeMillis();
+        logger.info("In Mem Cube Build start, " + cubeDesc.getName());
+
+        // multiple threads to compute cuboid in parallel
+        taskPending = new LinkedBlockingQueue<>();
+        taskCuboidCompleted.set(0);
+        taskThreads = prepareTaskThreads();
+        taskThreadExceptions = new Throwable[taskThreadCount];
+
+        // build base cuboid
+        resultCollector = collector;
+        totalSumForSanityCheck = null;
+        baseResult = createBaseCuboid(input);
+        if (baseResult.nRows == 0)
+            return;
+
+        // plan memory budget
+        makeMemoryBudget();
+
+        // kick off N-D cuboid tasks and output
+        addChildTasks(baseResult);
+        start(taskThreads);
+
+        // wait complete
+        join(taskThreads);
+
+        long endTime = System.currentTimeMillis();
+        logger.info("In Mem Cube Build end, " + cubeDesc.getName() + ", takes " + (endTime - startTime) + " ms");
+
+        throwExceptionIfAny();
+    }
+
+    public void abort() {
+        interrupt(taskThreads);
+    }
+
+    private void start(Thread... threads) {
+        for (Thread t : threads)
+            t.start();
+    }
+
+    private void interrupt(Thread... threads) {
+        for (Thread t : threads)
+            t.interrupt();
+    }
+
+    private void join(Thread... threads) throws IOException {
+        try {
+            for (Thread t : threads)
+                t.join();
+        } catch (InterruptedException e) {
+            throw new IOException("interrupted while waiting task and output complete", e);
+        }
+    }
+
+    private void throwExceptionIfAny() throws IOException {
+        ArrayList<Throwable> errors = new ArrayList<Throwable>();
+        for (int i = 0; i < taskThreadCount; i++) {
+            Throwable t = taskThreadExceptions[i];
+            if (t != null)
+                errors.add(t);
+        }
+        if (errors.isEmpty()) {
+            return;
+        } else if (errors.size() == 1) {
+            Throwable t = errors.get(0);
+            if (t instanceof IOException)
+                throw (IOException) t;
+            else
+                throw new IOException(t);
+        } else {
+            for (Throwable t : errors)
+                logger.error("Exception during in-mem cube build", t);
+            throw new IOException(errors.size() + " exceptions during in-mem cube build, cause set to the first, check log for more", errors.get(0));
+        }
+    }
+
+    private Thread[] prepareTaskThreads() {
+        Thread[] result = new Thread[taskThreadCount];
+        for (int i = 0; i < taskThreadCount; i++) {
+            result[i] = new CuboidTaskThread(i);
+        }
+        return result;
+    }
+
+    public boolean isAllCuboidDone() {
+        return taskCuboidCompleted.get() == totalCuboidCount;
+    }
+
+    private class CuboidTaskThread extends Thread {
+        private int id;
+
+        CuboidTaskThread(int id) {
+            super("CuboidTask-" + id);
+            this.id = id;
+        }
+
+        @Override
+        public void run() {
+            try {
+                while (!isAllCuboidDone()) {
+                    CuboidTask task = null;
+                    while (task == null && taskHasNoException()) {
+                        task = taskPending.poll(15, TimeUnit.SECONDS);
+                    }
+                    // if task error occurs
+                    if (task == null)
+                        break;
+
+                    CuboidResult newCuboid = buildCuboid(task.parent, task.childCuboidId);
+                    addChildTasks(newCuboid);
+
+                    if (isAllCuboidDone()) {
+                        for (Thread t : taskThreads) {
+                            if (t != Thread.currentThread())
+                                t.interrupt();
+                        }
+                    }
+                }
+            } catch (Throwable ex) {
+                if (!isAllCuboidDone()) {
+                    logger.error("task thread exception", ex);
+                    taskThreadExceptions[id] = ex;
+                }
+            }
+        }
+    }
+
+    private boolean taskHasNoException() {
+        for (int i = 0; i < taskThreadExceptions.length; i++)
+            if (taskThreadExceptions[i] != null)
+                return false;
+        return true;
+    }
+
+    private void addChildTasks(CuboidResult parent) {
+        List<Long> children = cuboidScheduler.getSpanningCuboid(parent.cuboidId);
+        for (Long child : children) {
+            taskPending.add(new CuboidTask(parent, child));
+        }
+    }
+
+    private int getSystemAvailMB() {
+        Runtime.getRuntime().gc();
+        try {
+            Thread.sleep(500);
+        } catch (InterruptedException e) {
+            logger.error("", e);
+        }
+        return MemoryBudgetController.getSystemAvailMB();
+    }
+
+    private void makeMemoryBudget() {
+        int systemAvailMB = getSystemAvailMB();
+        logger.info("System avail " + systemAvailMB + " MB");
+        int reserve = Math.max(reserveMemoryMB, baseResult.aggrCacheMB / 3);
+        logger.info("Reserve " + reserve + " MB for system basics");
+
+        int budget = systemAvailMB - reserve;
+        if (budget < baseResult.aggrCacheMB) {
+            // make sure we have base aggr cache as minimal
+            budget = baseResult.aggrCacheMB;
+            logger.warn("!!! System avail memory (" + systemAvailMB + " MB) is less than base aggr cache (" + baseResult.aggrCacheMB + " MB) + minimal reservation (" + reserve + " MB), consider increase JVM heap -Xmx");
+        }
+
+        logger.info("Memory Budget is " + budget + " MB");
+        memBudget = new MemoryBudgetController(budget);
+    }
+
+    private CuboidResult createBaseCuboid(BlockingQueue<List<String>> input) throws IOException {
+        GridTable baseCuboid = newGridTableByCuboidID(baseCuboidId);
+        GTBuilder baseBuilder = baseCuboid.rebuild();
+        IGTScanner baseInput = new InputConverter(baseCuboid.getInfo(), input);
+
+        int mbBefore = getSystemAvailMB();
+        int mbAfter = 0;
+
+        Pair<ImmutableBitSet, ImmutableBitSet> dimensionMetricsBitSet = getDimensionAndMetricColumnBitSet(baseCuboidId);
+        GTScanRequest req = new GTScanRequest(baseCuboid.getInfo(), null, dimensionMetricsBitSet.getFirst(), dimensionMetricsBitSet.getSecond(), metricsAggrFuncs, null);
+        GTAggregateScanner aggregationScanner = new GTAggregateScanner(baseInput, req);
+
+        long startTime = System.currentTimeMillis();
+        logger.info("Calculating cuboid " + baseCuboidId);
+
+        int count = 0;
+        for (GTRecord r : aggregationScanner) {
+            if (mbAfter == 0) {
+                mbAfter = getSystemAvailMB();
+            }
+            baseBuilder.write(r);
+            count++;
+        }
+        aggregationScanner.close();
+        baseBuilder.close();
+
+        long timeSpent = System.currentTimeMillis() - startTime;
+        logger.info("Cuboid " + baseCuboidId + " has " + count + " rows, build takes " + timeSpent + "ms");
+
+        int mbBaseAggrCacheOnHeap = mbAfter == 0 ? 0 : mbBefore - mbAfter;
+        int mbEstimateBaseAggrCache = (int) (aggregationScanner.getEstimateSizeOfAggrCache() / MemoryBudgetController.ONE_MB);
+        int mbBaseAggrCache = Math.max((int) (mbBaseAggrCacheOnHeap * 1.1), mbEstimateBaseAggrCache);
+        mbBaseAggrCache = Math.max(mbBaseAggrCache, 10); // let it be 10 MB at least
+        logger.info("Base aggr cache is " + mbBaseAggrCache + " MB (heap " + mbBaseAggrCacheOnHeap + " MB, estimate " + mbEstimateBaseAggrCache + " MB)");
+
+        return updateCuboidResult(baseCuboidId, baseCuboid, count, timeSpent, mbBaseAggrCache);
+    }
+
+    private CuboidResult updateCuboidResult(long cuboidId, GridTable table, int nRows, long timeSpent, int aggrCacheMB) {
+        if (aggrCacheMB <= 0) {
+            aggrCacheMB = (int) Math.ceil(1.0 * nRows / baseResult.nRows * baseResult.aggrCacheMB);
+        }
+
+        CuboidResult result = new CuboidResult(cuboidId, table, nRows, timeSpent, aggrCacheMB);
+        taskCuboidCompleted.incrementAndGet();
+
+        resultCollector.collect(result);
+        return result;
+    }
+
+    private CuboidResult buildCuboid(CuboidResult parent, long cuboidId) throws IOException {
+        final String consumerName = "AggrCache@Cuboid " + cuboidId;
+        MemoryBudgetController.MemoryConsumer consumer = new MemoryBudgetController.MemoryConsumer() {
+            @Override
+            public int freeUp(int mb) {
+                return 0; // cannot free up on demand
+            }
+
+            @Override
+            public String toString() {
+                return consumerName;
+            }
+        };
+
+        // reserve memory for aggregation cache, can't be larger than the parent
+        memBudget.reserveInsist(consumer, parent.aggrCacheMB);
+        try {
+            return aggregateCuboid(parent, cuboidId);
+        } finally {
+            memBudget.reserve(consumer, 0);
+        }
+    }
+
+    private CuboidResult aggregateCuboid(CuboidResult parent, long cuboidId) throws IOException {
+        Pair<ImmutableBitSet, ImmutableBitSet> columnBitSets = getDimensionAndMetricColumnBitSet(parent.cuboidId);
+        ImmutableBitSet parentDimensions = columnBitSets.getFirst();
+        ImmutableBitSet measureColumns = columnBitSets.getSecond();
+        ImmutableBitSet childDimensions = parentDimensions;
+
+        long mask = Long.highestOneBit(parent.cuboidId);
+        long childCuboidId = cuboidId;
+        long parentCuboidIdActualLength = Long.SIZE - Long.numberOfLeadingZeros(parent.cuboidId);
+        int index = 0;
+        for (int i = 0; i < parentCuboidIdActualLength; i++) {
+            if ((mask & parent.cuboidId) > 0) {
+                if ((mask & childCuboidId) == 0) {
+                    // this dim will be aggregated
+                    childDimensions = childDimensions.set(index, false);
+                }
+                index++;
+            }
+            mask = mask >> 1;
+        }
+
+        return scanAndAggregateGridTable(parent.table, cuboidId, childDimensions, measureColumns);
+    }
+
+    private CuboidResult scanAndAggregateGridTable(GridTable gridTable, long cuboidId, ImmutableBitSet aggregationColumns, ImmutableBitSet measureColumns) throws IOException {
+        long startTime = System.currentTimeMillis();
+        logger.info("Calculating cuboid " + cuboidId);
+
+        GTScanRequest req = new GTScanRequest(gridTable.getInfo(), null, aggregationColumns, measureColumns, metricsAggrFuncs, null);
+        GTAggregateScanner scanner = (GTAggregateScanner) gridTable.scan(req);
+        GridTable newGridTable = newGridTableByCuboidID(cuboidId);
+        GTBuilder builder = newGridTable.rebuild();
+
+        ImmutableBitSet allNeededColumns = aggregationColumns.or(measureColumns);
+
+        GTRecord newRecord = new GTRecord(newGridTable.getInfo());
+        int count = 0;
+        try {
+            for (GTRecord record : scanner) {
+                count++;
+                for (int i = 0; i < allNeededColumns.trueBitCount(); i++) {
+                    int c = allNeededColumns.trueBitAt(i);
+                    newRecord.set(i, record.get(c));
+                }
+                builder.write(newRecord);
+            }
+
+            // disable sanity check for performance
+            sanityCheck(scanner.getTotalSumForSanityCheck());
+        } finally {
+            scanner.close();
+            builder.close();
+        }
+
+        long timeSpent = System.currentTimeMillis() - startTime;
+        logger.info("Cuboid " + cuboidId + " has " + count + " rows, build takes " + timeSpent + "ms");
+
+        return updateCuboidResult(cuboidId, newGridTable, count, timeSpent, 0);
+    }
+
+    //@SuppressWarnings("unused")
+    private void sanityCheck(Object[] totalSum) {
+        // double sum introduces error and causes result not exactly equal
+        for (int i = 0; i < totalSum.length; i++) {
+            if (totalSum[i] instanceof DoubleMutable) {
+                totalSum[i] = Math.round(((DoubleMutable) totalSum[i]).get());
+            }
+        }
+
+        if (totalSumForSanityCheck == null) {
+            totalSumForSanityCheck = totalSum;
+            return;
+        }
+        if (Arrays.equals(totalSumForSanityCheck, totalSum) == false) {
+            throw new IllegalStateException();
+        }
+    }
+
+    // ===========================================================================
+
+    private static class CuboidTask implements Comparable<CuboidTask> {
+        final CuboidResult parent;
+        final long childCuboidId;
+
+        CuboidTask(CuboidResult parent, long childCuboidId) {
+            this.parent = parent;
+            this.childCuboidId = childCuboidId;
+        }
+
+        @Override
+        public int compareTo(CuboidTask o) {
+            long comp = this.childCuboidId - o.childCuboidId;
+            return comp < 0 ? -1 : (comp > 0 ? 1 : 0);
+        }
+    }
+
+    // ============================================================================
+
+    private class InputConverter implements IGTScanner {
+        GTInfo info;
+        GTRecord record;
+        BlockingQueue<List<String>> input;
+
+        public InputConverter(GTInfo info, BlockingQueue<List<String>> input) {
+            this.info = info;
+            this.input = input;
+            this.record = new GTRecord(info);
+        }
+
+        @Override
+        public Iterator<GTRecord> iterator() {
+            return new Iterator<GTRecord>() {
+
+                List<String> currentObject = null;
+
+                @Override
+                public boolean hasNext() {
+                    try {
+                        currentObject = input.take();
+                    } catch (InterruptedException e) {
+                        throw new RuntimeException(e);
+                    }
+                    return currentObject != null && currentObject.size() > 0;
+                }
+
+                @Override
+                public GTRecord next() {
+                    if (currentObject.size() == 0)
+                        throw new IllegalStateException();
+
+                    buildGTRecord(currentObject, record);
+                    return record;
+                }
+
+                @Override
+                public void remove() {
+                    throw new UnsupportedOperationException();
+                }
+            };
+        }
+
+        @Override
+        public void close() throws IOException {
+        }
+
+        @Override
+        public GTInfo getInfo() {
+            return info;
+        }
+
+        @Override
+        public int getScannedRowCount() {
+            return 0;
+        }
+
+        @Override
+        public int getScannedRowBlockCount() {
+            return 0;
+        }
+
+        private void buildGTRecord(List<String> row, GTRecord record) {
+            Object[] dimensions = buildKey(row);
+            Object[] metricsValues = buildValue(row);
+            Object[] recordValues = new Object[dimensions.length + metricsValues.length];
+            System.arraycopy(dimensions, 0, recordValues, 0, dimensions.length);
+            System.arraycopy(metricsValues, 0, recordValues, dimensions.length, metricsValues.length);
+            record.setValues(recordValues);
+        }
+
+        private Object[] buildKey(List<String> row) {
+            int keySize = intermediateTableDesc.getRowKeyColumnIndexes().length;
+            Object[] key = new Object[keySize];
+
+            for (int i = 0; i < keySize; i++) {
+                key[i] = row.get(intermediateTableDesc.getRowKeyColumnIndexes()[i]);
+            }
+
+            return key;
+        }
+
+        private Object[] buildValue(List<String> row) {
+
+            Object[] values = new Object[measureCount];
+            MeasureDesc measureDesc = null;
+
+            for (int position = 0; position < hbaseMeasureRefIndex.length; position++) {
+                int i = hbaseMeasureRefIndex[position];
+                measureDesc = measureDescs[i];
+
+                Object value = null;
+                int[] flatTableIdx = intermediateTableDesc.getMeasureColumnIndexes()[i];
+                FunctionDesc function = cubeDesc.getMeasures().get(i).getFunction();
+                if (function.isCount() || function.isHolisticCountDistinct()) {
+                    // note for holistic count distinct, this value will be ignored
+                    value = ONE;
+                } else if (flatTableIdx == null) {
+                    value = measureCodec.getSerializer(i).valueOf(measureDesc.getFunction().getParameter().getValue());
+                } else if (flatTableIdx.length == 1) {
+                    value = measureCodec.getSerializer(i).valueOf(toBytes(row.get(flatTableIdx[0])));
+                } else {
+
+                    byte[] result = null;
+                    for (int x = 0; x < flatTableIdx.length; x++) {
+                        byte[] split = toBytes(row.get(flatTableIdx[x]));
+                        if (result == null) {
+                            result = Arrays.copyOf(split, split.length);
+                        } else {
+                            byte[] newResult = new byte[result.length + split.length];
+                            System.arraycopy(result, 0, newResult, 0, result.length);
+                            System.arraycopy(split, 0, newResult, result.length, split.length);
+                            result = newResult;
+                        }
+                    }
+                    value = measureCodec.getSerializer(i).valueOf(result);
+                }
+                values[position] = value;
+            }
+            return values;
+        }
+
+        private byte[] toBytes(String v) {
+            return v == null ? null : Bytes.toBytes(v);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/MemDiskStore.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/MemDiskStore.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/MemDiskStore.java
new file mode 100644
index 0000000..d9b0ba6
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/MemDiskStore.java
@@ -0,0 +1,679 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import static org.apache.kylin.common.util.MemoryBudgetController.*;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.StandardOpenOption;
+import java.util.NoSuchElementException;
+
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.common.util.MemoryBudgetController;
+import org.apache.kylin.common.util.MemoryBudgetController.MemoryConsumer;
+import org.apache.kylin.common.util.MemoryBudgetController.NotEnoughBudgetException;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTRowBlock;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.IGTStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MemDiskStore implements IGTStore, Closeable {
+
+    private static final Logger logger = LoggerFactory.getLogger(MemDiskStore.class);
+    private static final boolean debug = true;
+
+    private static final int STREAM_BUFFER_SIZE = 8192;
+    private static final int MEM_CHUNK_SIZE_MB = 5;
+
+    private final GTInfo info;
+    private final Object lock; // all public methods that read/write object states are synchronized on this lock
+    private final MemPart memPart;
+    private final DiskPart diskPart;
+    private final boolean delOnClose;
+
+    private Writer ongoingWriter;
+
+    public MemDiskStore(GTInfo info, MemoryBudgetController budgetCtrl) throws IOException {
+        this(info, budgetCtrl, File.createTempFile("MemDiskStore", ""), true);
+    }
+
+    public MemDiskStore(GTInfo info, MemoryBudgetController budgetCtrl, File diskFile) throws IOException {
+        this(info, budgetCtrl, diskFile, false);
+    }
+
+    private MemDiskStore(GTInfo info, MemoryBudgetController budgetCtrl, File diskFile, boolean delOnClose) throws IOException {
+        this.info = info;
+        this.lock = this;
+        this.memPart = new MemPart(budgetCtrl);
+        this.diskPart = new DiskPart(diskFile);
+        this.delOnClose = delOnClose;
+
+        // in case user forget to call close()
+        if (delOnClose)
+            diskFile.deleteOnExit();
+    }
+
+    @Override
+    public GTInfo getInfo() {
+        return info;
+    }
+
+    @Override
+    public IGTStoreWriter rebuild(int shard) throws IOException {
+        return newWriter(0);
+    }
+
+    @Override
+    public IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) throws IOException {
+        return newWriter(length());
+    }
+
+    private Writer newWriter(long startOffset) throws IOException {
+        synchronized (lock) {
+            if (ongoingWriter != null)
+                throw new IllegalStateException();
+
+            ongoingWriter = new Writer(startOffset);
+            return ongoingWriter;
+        }
+    }
+
+    @Override
+    public IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) throws IOException {
+        synchronized (lock) {
+            return new Reader();
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        // synchronized inside the parts close()
+        memPart.close();
+        diskPart.close();
+    }
+
+    public long length() {
+        synchronized (lock) {
+            return Math.max(memPart.tailOffset(), diskPart.tailOffset);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "MemDiskStore@" + (info.getTableName() == null ? this.hashCode() : info.getTableName());
+    }
+
+    private class Reader implements IGTStoreScanner {
+
+        final DataInputStream din;
+        long readOffset = 0;
+        long memRead = 0;
+        long diskRead = 0;
+        int nReadCalls = 0;
+
+        GTRowBlock block = GTRowBlock.allocate(info);
+        GTRowBlock next = null;
+
+        Reader() throws IOException {
+            diskPart.openRead();
+            if (debug)
+                logger.debug(MemDiskStore.this + " read start @ " + readOffset);
+
+            InputStream in = new InputStream() {
+                byte[] tmp = new byte[1];
+                MemChunk memChunk;
+
+                @Override
+                public int read() throws IOException {
+                    int n = read(tmp, 0, 1);
+                    if (n <= 0)
+                        return -1;
+                    else
+                        return (int) tmp[0];
+                }
+
+                @Override
+                public int read(byte[] b, int off, int len) throws IOException {
+                    synchronized (lock) {
+                        nReadCalls++;
+                        if (available() <= 0)
+                            return -1;
+
+                        if (memChunk == null && memPart.headOffset() <= readOffset && readOffset < memPart.tailOffset()) {
+                            memChunk = memPart.seekMemChunk(readOffset);
+                        }
+
+                        int lenToGo = Math.min(available(), len);
+
+                        int nRead = 0;
+                        while (lenToGo > 0) {
+                            int n;
+                            if (memChunk != null) {
+                                if (memChunk.headOffset() > readOffset) {
+                                    memChunk = null;
+                                    continue;
+                                }
+                                if (readOffset >= memChunk.tailOffset()) {
+                                    memChunk = memChunk.next;
+                                    continue;
+                                }
+                                int chunkOffset = (int) (readOffset - memChunk.headOffset());
+                                n = Math.min((int) (memChunk.tailOffset() - readOffset), lenToGo);
+                                System.arraycopy(memChunk.data, chunkOffset, b, off, n);
+                                memRead += n;
+                            } else {
+                                n = diskPart.read(readOffset, b, off, lenToGo);
+                                diskRead += n;
+                            }
+                            lenToGo -= n;
+                            nRead += n;
+                            off += n;
+                            readOffset += n;
+                        }
+                        return nRead;
+                    }
+                }
+
+                @Override
+                public int available() throws IOException {
+                    synchronized (lock) {
+                        return (int) (length() - readOffset);
+                    }
+                }
+            };
+
+            din = new DataInputStream(new BufferedInputStream(in, STREAM_BUFFER_SIZE));
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (next != null)
+                return true;
+
+            try {
+                if (din.available() > 0) {
+                    block.importFrom(din);
+                    next = block;
+                }
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+
+            return next != null;
+        }
+
+        @Override
+        public GTRowBlock next() {
+            if (next == null) {
+                hasNext();
+                if (next == null)
+                    throw new NoSuchElementException();
+            }
+            GTRowBlock r = next;
+            next = null;
+            return r;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void close() throws IOException {
+            synchronized (lock) {
+                din.close();
+                diskPart.closeRead();
+                if (debug)
+                    logger.debug(MemDiskStore.this + " read end @ " + readOffset + ", " + (memRead) + " from mem, " + (diskRead) + " from disk, " + nReadCalls + " read() calls");
+            }
+        }
+
+    }
+
+    private class Writer implements IGTStoreWriter {
+
+        final DataOutputStream dout;
+        long writeOffset;
+        long memWrite = 0;
+        long diskWrite = 0;
+        int nWriteCalls;
+        boolean closed = false;
+
+        Writer(long startOffset) throws IOException {
+            writeOffset = 0; // TODO does not support append yet
+            memPart.clear();
+            diskPart.clear();
+            diskPart.openWrite(false);
+            if (debug)
+                logger.debug(MemDiskStore.this + " write start @ " + writeOffset);
+
+            memPart.activateMemWrite();
+
+            OutputStream out = new OutputStream() {
+                byte[] tmp = new byte[1];
+                boolean memPartActivated = true;
+
+                @Override
+                public void write(int b) throws IOException {
+                    tmp[0] = (byte) b;
+                    write(tmp, 0, 1);
+                }
+
+                @Override
+                public void write(byte[] bytes, int offset, int length) throws IOException {
+                    // lock inside memPart.write() and diskPartm.write()
+                    nWriteCalls++;
+                    while (length > 0) {
+                        int n;
+                        if (memPartActivated) {
+                            n = memPart.write(bytes, offset, length, writeOffset);
+                            memWrite += n;
+                            if (n == 0) {
+                                memPartActivated = false;
+                            }
+                        } else {
+                            n = diskPart.write(writeOffset, bytes, offset, length);
+                            diskWrite += n;
+                        }
+                        offset += n;
+                        length -= n;
+                        writeOffset += n;
+                    }
+                }
+            };
+            dout = new DataOutputStream(new BufferedOutputStream(out, STREAM_BUFFER_SIZE));
+        }
+
+        @Override
+        public void write(GTRowBlock block) throws IOException {
+            block.export(dout);
+        }
+
+        @Override
+        public void close() throws IOException {
+            synchronized (lock) {
+                if (!closed) {
+                    dout.close();
+                    memPart.deactivateMemWrite();
+                }
+
+                if (memPart.asyncFlusher == null) {
+                    assert writeOffset == diskPart.tailOffset;
+                    diskPart.closeWrite();
+                    ongoingWriter = null;
+                    if (debug)
+                        logger.debug(MemDiskStore.this + " write end @ " + writeOffset + ", " + (memWrite) + " to mem, " + (diskWrite) + " to disk, " + nWriteCalls + " write() calls");
+                } else {
+                    // the asyncFlusher will call this close() again later
+                }
+                closed = true;
+            }
+        }
+    }
+
+    private static class MemChunk {
+        long diskOffset;
+        int length;
+        byte[] data;
+        MemChunk next;
+
+        boolean isFull() {
+            return length == data.length;
+        }
+
+        long headOffset() {
+            return diskOffset;
+        }
+
+        long tailOffset() {
+            return diskOffset + length;
+        }
+
+        int freeSpace() {
+            return data.length - length;
+        }
+    }
+
+    private class MemPart implements Closeable, MemoryConsumer {
+
+        final MemoryBudgetController budgetCtrl;
+
+        // async flush thread checks this flag out of sync block
+        volatile boolean writeActivated;
+        MemChunk firstChunk;
+        MemChunk lastChunk;
+        int chunkCount;
+
+        Thread asyncFlusher;
+        MemChunk asyncFlushChunk;
+        long asyncFlushDiskOffset;
+        Throwable asyncFlushException;
+
+        MemPart(MemoryBudgetController budgetCtrl) {
+            this.budgetCtrl = budgetCtrl;
+        }
+
+        long headOffset() {
+            return firstChunk == null ? 0 : firstChunk.headOffset();
+        }
+
+        long tailOffset() {
+            return lastChunk == null ? 0 : lastChunk.tailOffset();
+        }
+
+        public MemChunk seekMemChunk(long diskOffset) {
+            MemChunk c = firstChunk;
+            while (c != null && c.headOffset() <= diskOffset) {
+                if (diskOffset < c.tailOffset())
+                    break;
+                c = c.next;
+            }
+            return c;
+        }
+
+        public int write(byte[] bytes, int offset, int length, long diskOffset) {
+            int needMoreMem = 0;
+
+            synchronized (lock) {
+                if (writeActivated == false)
+                    return 0;
+
+                // write is only expected at the tail
+                if (diskOffset != tailOffset())
+                    return 0;
+
+                if (chunkCount == 0 || lastChunk.isFull())
+                    needMoreMem = (chunkCount + 1) * MEM_CHUNK_SIZE_MB;
+            }
+
+            // call to budgetCtrl.reserve() must be out of synchronized block, or deadlock may happen between MemoryConsumers
+            if (needMoreMem > 0) {
+                try {
+                    budgetCtrl.reserve(this, needMoreMem);
+                } catch (NotEnoughBudgetException ex) {
+                    deactivateMemWrite();
+                    return 0;
+                }
+            }
+
+            synchronized (lock) {
+                if (needMoreMem > 0 && (chunkCount == 0 || lastChunk.isFull())) {
+                    MemChunk chunk = new MemChunk();
+                    chunk.diskOffset = diskOffset;
+                    chunk.data = new byte[ONE_MB * MEM_CHUNK_SIZE_MB - 48]; // -48 for MemChunk overhead
+                    if (chunkCount == 0) {
+                        firstChunk = lastChunk = chunk;
+                    } else {
+                        lastChunk.next = chunk;
+                        lastChunk = chunk;
+                    }
+                    chunkCount++;
+                }
+
+                int n = Math.min(lastChunk.freeSpace(), length);
+                System.arraycopy(bytes, offset, lastChunk.data, lastChunk.length, n);
+                lastChunk.length += n;
+
+                if (n > 0)
+                    asyncFlush(lastChunk, diskOffset, n);
+
+                return n;
+            }
+        }
+
+        private void asyncFlush(MemChunk lastChunk, long diskOffset, int n) {
+            if (asyncFlushChunk == null) {
+                asyncFlushChunk = lastChunk;
+                asyncFlushDiskOffset = diskOffset;
+            }
+
+            if (asyncFlusher == null) {
+                asyncFlusher = new Thread() {
+                    public void run() {
+                        asyncFlushException = null;
+                        if (debug)
+                            logger.debug(MemDiskStore.this + " async flush started @ " + asyncFlushDiskOffset);
+                        try {
+                            while (writeActivated) {
+                                flushToDisk();
+                                Thread.sleep(10);
+                            }
+                            flushToDisk();
+
+                            if (debug)
+                                logger.debug(MemDiskStore.this + " async flush ended @ " + asyncFlushDiskOffset);
+
+                            synchronized (lock) {
+                                asyncFlusher = null;
+                                asyncFlushChunk = null;
+                                if (ongoingWriter.closed) {
+                                    ongoingWriter.close(); // call writer.close() again to clean up
+                                }
+                            }
+                        } catch (Throwable ex) {
+                            asyncFlushException = ex;
+                        }
+                    }
+                };
+                asyncFlusher.start();
+            }
+        }
+
+        private void flushToDisk() throws IOException {
+            byte[] data;
+            int offset = 0;
+            int length = 0;
+            int flushedLen = 0;
+
+            while (true) {
+                data = null;
+                synchronized (lock) {
+                    asyncFlushDiskOffset += flushedLen; // bytes written in last loop
+                    //                    if (debug)
+                    //                        logger.debug(GTMemDiskStore.this + " async flush @ " + asyncFlushDiskOffset);
+                    if (asyncFlushChunk != null && asyncFlushChunk.tailOffset() == asyncFlushDiskOffset) {
+                        asyncFlushChunk = asyncFlushChunk.next;
+                    }
+                    if (asyncFlushChunk != null) {
+                        data = asyncFlushChunk.data;
+                        offset = (int) (asyncFlushDiskOffset - asyncFlushChunk.headOffset());
+                        length = asyncFlushChunk.length - offset;
+                    }
+                }
+
+                if (data == null)
+                    break;
+
+                flushedLen = diskPart.write(asyncFlushDiskOffset, data, offset, length);
+            }
+        }
+
+        @Override
+        public int freeUp(int mb) {
+            synchronized (lock) {
+                int mbReleased = 0;
+                while (chunkCount > 0 && mbReleased < mb) {
+                    if (firstChunk == asyncFlushChunk)
+                        break;
+
+                    mbReleased += MEM_CHUNK_SIZE_MB;
+                    chunkCount--;
+                    if (chunkCount == 0) {
+                        firstChunk = lastChunk = null;
+                    } else {
+                        MemChunk next = firstChunk.next;
+                        firstChunk.next = null;
+                        firstChunk = next;
+                    }
+                }
+                return mbReleased;
+            }
+        }
+
+        public void activateMemWrite() {
+            if (budgetCtrl.getTotalBudgetMB() > 0) {
+                writeActivated = true;
+                if (debug)
+                    logger.debug(MemDiskStore.this + " mem write activated");
+            }
+        }
+
+        public void deactivateMemWrite() {
+            writeActivated = false;
+            if (debug)
+                logger.debug(MemDiskStore.this + " mem write de-activated");
+        }
+
+        public void clear() {
+            chunkCount = 0;
+            firstChunk = lastChunk = null;
+            budgetCtrl.reserve(this, 0);
+        }
+
+        @Override
+        public void close() throws IOException {
+            synchronized (lock) {
+                if (asyncFlushException != null)
+                    throwAsyncException(asyncFlushException);
+            }
+            try {
+                asyncFlusher.join();
+            } catch (NullPointerException npe) {
+                // that's fine, async flusher may not present
+            } catch (InterruptedException e) {
+                logger.warn("async join interrupted", e);
+            }
+            synchronized (lock) {
+                if (asyncFlushException != null)
+                    throwAsyncException(asyncFlushException);
+
+                clear();
+            }
+        }
+
+        private void throwAsyncException(Throwable ex) throws IOException {
+            if (ex instanceof IOException)
+                throw (IOException) ex;
+            else
+                throw new IOException(ex);
+        }
+
+        @Override
+        public String toString() {
+            return MemDiskStore.this.toString();
+        }
+
+    }
+
+    private class DiskPart implements Closeable {
+        final File diskFile;
+        FileChannel writeChannel;
+        FileChannel readChannel;
+        int readerCount = 0; // allow parallel readers
+        long tailOffset;
+
+        DiskPart(File diskFile) throws IOException {
+            this.diskFile = diskFile;
+            this.tailOffset = diskFile.length();
+            if (debug)
+                logger.debug(MemDiskStore.this + " disk file " + diskFile.getAbsolutePath());
+        }
+
+        public void openRead() throws IOException {
+            if (readChannel == null) {
+                readChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.READ);
+            }
+            readerCount++;
+        }
+
+        public int read(long diskOffset, byte[] bytes, int offset, int length) throws IOException {
+            return readChannel.read(ByteBuffer.wrap(bytes, offset, length), diskOffset);
+        }
+
+        public void closeRead() throws IOException {
+            closeRead(false);
+        }
+
+        private void closeRead(boolean force) throws IOException {
+            readerCount--;
+            if (readerCount == 0 || force) {
+                if (readChannel != null) {
+                    readChannel.close();
+                    readChannel = null;
+                }
+            }
+        }
+
+        public void openWrite(boolean append) throws IOException {
+            if (append) {
+                writeChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.CREATE, StandardOpenOption.APPEND, StandardOpenOption.WRITE);
+                tailOffset = diskFile.length();
+            } else {
+                diskFile.delete();
+                writeChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+                tailOffset = 0;
+            }
+        }
+
+        public int write(long diskOffset, byte[] bytes, int offset, int length) throws IOException {
+            synchronized (lock) {
+                int n = writeChannel.write(ByteBuffer.wrap(bytes, offset, length), diskOffset);
+                tailOffset = Math.max(diskOffset + n, tailOffset);
+                return n;
+            }
+        }
+
+        public void closeWrite() throws IOException {
+            if (writeChannel != null) {
+                writeChannel.close();
+                writeChannel = null;
+            }
+        }
+
+        public void clear() throws IOException {
+            diskFile.delete();
+            tailOffset = 0;
+        }
+
+        @Override
+        public void close() throws IOException {
+            synchronized (lock) {
+                closeWrite();
+                closeRead(true);
+                if (delOnClose) {
+                    diskFile.delete();
+                }
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
new file mode 100644
index 0000000..38ccccd
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
@@ -0,0 +1,268 @@
+package org.apache.kylin.gridtable;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.SortedMap;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.metadata.measure.HLLCAggregator;
+import org.apache.kylin.metadata.measure.LDCAggregator;
+import org.apache.kylin.metadata.measure.MeasureAggregator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
+
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class GTAggregateScanner implements IGTScanner {
+
+    @SuppressWarnings("unused")
+    private static final Logger logger = LoggerFactory.getLogger(GTAggregateScanner.class);
+
+    final GTInfo info;
+    final ImmutableBitSet dimensions; // dimensions to return, can be more than group by
+    final ImmutableBitSet groupBy;
+    final ImmutableBitSet metrics;
+    final String[] metricsAggrFuncs;
+    final IGTScanner inputScanner;
+    final AggregationCache aggrCache;
+
+    public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req) {
+        if (req.hasAggregation() == false)
+            throw new IllegalStateException();
+
+        this.info = inputScanner.getInfo();
+        this.dimensions = req.getColumns().andNot(req.getAggrMetrics());
+        this.groupBy = req.getAggrGroupBy();
+        this.metrics = req.getAggrMetrics();
+        this.metricsAggrFuncs = req.getAggrMetricsFuncs();
+        this.inputScanner = inputScanner;
+        this.aggrCache = new AggregationCache();
+    }
+
+    @Override
+    public GTInfo getInfo() {
+        return info;
+    }
+
+    @Override
+    public int getScannedRowCount() {
+        return inputScanner.getScannedRowCount();
+    }
+
+    @Override
+    public int getScannedRowBlockCount() {
+        return inputScanner.getScannedRowBlockCount();
+    }
+
+    @Override
+    public void close() throws IOException {
+        inputScanner.close();
+    }
+
+    @Override
+    public Iterator<GTRecord> iterator() {
+        for (GTRecord r : inputScanner) {
+            aggrCache.aggregate(r);
+        }
+        return aggrCache.iterator();
+    }
+
+    /** return the estimate memory size of aggregation cache */
+    public long getEstimateSizeOfAggrCache() {
+        return aggrCache.esitmateMemSize();
+    }
+
+    public Object[] getTotalSumForSanityCheck() {
+        return aggrCache.calculateTotalSumSanityCheck();
+    }
+
+    class AggregationCache {
+        final SortedMap<byte[], MeasureAggregator[]> aggBufMap;
+        final int keyLength;
+        final boolean[] compareMask;
+
+        public AggregationCache() {
+            compareMask = createCompareMask();
+            keyLength = compareMask.length;
+            aggBufMap = Maps.newTreeMap(new Comparator<byte[]>() {
+                @Override
+                public int compare(byte[] o1, byte[] o2) {
+                    int result = 0;
+                    // profiler shows this check is slow
+                    // Preconditions.checkArgument(keyLength == o1.length && keyLength == o2.length);
+                    for (int i = 0; i < keyLength; ++i) {
+                        if (compareMask[i]) {
+                            int a = (o1[i] & 0xff);
+                            int b = (o2[i] & 0xff);
+                            result = a - b;
+                            if (result == 0) {
+                                continue;
+                            } else {
+                                return result;
+                            }
+                        }
+                    }
+                    return result;
+                }
+            });
+        }
+
+        private boolean[] createCompareMask() {
+            int keyLength = 0;
+            for (int i = 0; i < dimensions.trueBitCount(); i++) {
+                int c = dimensions.trueBitAt(i);
+                int l = info.codeSystem.maxCodeLength(c);
+                keyLength += l;
+            }
+
+            boolean[] mask = new boolean[keyLength];
+            int p = 0;
+            for (int i = 0; i < dimensions.trueBitCount(); i++) {
+                int c = dimensions.trueBitAt(i);
+                int l = info.codeSystem.maxCodeLength(c);
+                boolean m = groupBy.get(c) ? true : false;
+                for (int j = 0; j < l; j++) {
+                    mask[p++] = m;
+                }
+            }
+            return mask;
+        }
+
+        private byte[] createKey(GTRecord record) {
+            byte[] result = new byte[keyLength];
+            int offset = 0;
+            for (int i = 0; i < dimensions.trueBitCount(); i++) {
+                int c = dimensions.trueBitAt(i);
+                final ByteArray byteArray = record.cols[c];
+                final int columnLength = info.codeSystem.maxCodeLength(c);
+                System.arraycopy(byteArray.array(), byteArray.offset(), result, offset, byteArray.length());
+                offset += columnLength;
+            }
+            assert offset == result.length;
+            return result;
+        }
+
+        void aggregate(GTRecord r) {
+            final byte[] key = createKey(r);
+            MeasureAggregator[] aggrs = aggBufMap.get(key);
+            if (aggrs == null) {
+                aggrs = newAggregators();
+                aggBufMap.put(key, aggrs);
+            }
+            for (int i = 0; i < aggrs.length; i++) {
+                int col = metrics.trueBitAt(i);
+                Object metrics = info.codeSystem.decodeColumnValue(col, r.cols[col].asBuffer());
+                aggrs[i].aggregate(metrics);
+            }
+        }
+
+        private MeasureAggregator[] newAggregators() {
+            return info.codeSystem.newMetricsAggregators(metrics, metricsAggrFuncs);
+        }
+
+        public Object[] calculateTotalSumSanityCheck() {
+            MeasureAggregator[] totalSum = newAggregators();
+
+            // skip expensive aggregation
+            for (int i = 0; i < totalSum.length; i++) {
+                if (totalSum[i] instanceof HLLCAggregator || totalSum[i] instanceof LDCAggregator)
+                    totalSum[i] = null;
+            }
+
+            for (MeasureAggregator[] entry : aggBufMap.values()) {
+                for (int i = 0; i < totalSum.length; i++) {
+                    if (totalSum[i] != null)
+                        totalSum[i].aggregate(entry[i].getState());
+                }
+            }
+            Object[] result = new Object[totalSum.length];
+            for (int i = 0; i < totalSum.length; i++) {
+                if (totalSum[i] != null)
+                    result[i] = totalSum[i].getState();
+            }
+            return result;
+        }
+
+        public long esitmateMemSize() {
+            if (aggBufMap.isEmpty())
+                return 0;
+
+            byte[] sampleKey = aggBufMap.firstKey();
+            MeasureAggregator<?>[] sampleValue = aggBufMap.get(sampleKey);
+            return estimateSizeOfAggrCache(sampleKey, sampleValue, aggBufMap.size());
+        }
+
+        public Iterator<GTRecord> iterator() {
+            return new Iterator<GTRecord>() {
+
+                final Iterator<Entry<byte[], MeasureAggregator[]>> it = aggBufMap.entrySet().iterator();
+
+                final ByteBuffer metricsBuf = ByteBuffer.allocate(info.getMaxColumnLength(metrics));
+                final GTRecord secondRecord = new GTRecord(info);
+
+                @Override
+                public boolean hasNext() {
+                    return it.hasNext();
+                }
+
+                @Override
+                public GTRecord next() {
+                    Entry<byte[], MeasureAggregator[]> entry = it.next();
+                    create(entry.getKey(), entry.getValue());
+                    return secondRecord;
+                }
+
+                private void create(byte[] key, MeasureAggregator[] value) {
+                    int offset = 0;
+                    for (int i = 0; i < dimensions.trueBitCount(); i++) {
+                        int c = dimensions.trueBitAt(i);
+                        final int columnLength = info.codeSystem.maxCodeLength(c);
+                        secondRecord.set(c, new ByteArray(key, offset, columnLength));
+                        offset += columnLength;
+                    }
+                    metricsBuf.clear();
+                    for (int i = 0; i < value.length; i++) {
+                        int col = metrics.trueBitAt(i);
+                        int pos = metricsBuf.position();
+                        info.codeSystem.encodeColumnValue(col, value[i].getState(), metricsBuf);
+                        secondRecord.cols[col].set(metricsBuf.array(), pos, metricsBuf.position() - pos);
+                    }
+                }
+
+                @Override
+                public void remove() {
+                    throw new UnsupportedOperationException();
+                }
+            };
+        }
+    }
+
+    public static long estimateSizeOfAggrCache(byte[] keySample, MeasureAggregator<?>[] aggrSample, int size) {
+        // Aggregation cache is basically a tree map. The tree map entry overhead is
+        // - 40 according to http://java-performance.info/memory-consumption-of-java-data-types-2/
+        // - 41~52 according to AggregationCacheMemSizeTest
+        return (estimateSizeOf(keySample) + estimateSizeOf(aggrSample) + 64) * size;
+    }
+
+    public static long estimateSizeOf(MeasureAggregator[] aggrs) {
+        // size of array, AggregationCacheMemSizeTest reports 4 for [0], 12 for [1], 12 for [2], 20 for [3] etc..
+        // Memory alignment to 8 bytes
+        long est = (aggrs.length + 1) / 2 * 8 + 4 + (4 /* extra */);
+        for (MeasureAggregator aggr : aggrs) {
+            if (aggr != null)
+                est += aggr.getMemBytesEstimate();
+        }
+        return est;
+    }
+
+    public static long estimateSizeOf(byte[] bytes) {
+        // AggregationCacheMemSizeTest reports 20 for byte[10] and 20 again for byte[16]
+        // Memory alignment to 8 bytes
+        return (bytes.length + 7) / 8 * 8 + 4 + (4 /* extra */);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTBuilder.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTBuilder.java
new file mode 100644
index 0000000..7308b5c
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTBuilder.java
@@ -0,0 +1,74 @@
+package org.apache.kylin.gridtable;
+
+import java.io.Closeable;
+import java.io.Flushable;
+import java.io.IOException;
+
+import org.apache.kylin.gridtable.IGTStore.IGTStoreWriter;
+
+public class GTBuilder implements Closeable, Flushable {
+
+    @SuppressWarnings("unused")
+    final private GTInfo info;
+    final private IGTStoreWriter storeWriter;
+
+    final private GTRowBlock block;
+    final private GTRowBlock.Writer blockWriter;
+
+    private int writtenRowCount;
+    private int writtenRowBlockCount;
+
+    GTBuilder(GTInfo info, int shard, IGTStore store) throws IOException {
+        this(info, shard, store, false);
+    }
+
+    GTBuilder(GTInfo info, int shard, IGTStore store, boolean append) throws IOException {
+        this.info = info;
+
+        block = GTRowBlock.allocate(info);
+        blockWriter = block.getWriter();
+        if (append) {
+            storeWriter = store.append(shard, blockWriter);
+            if (block.isFull()) {
+                blockWriter.clearForNext();
+            }
+        } else {
+            storeWriter = store.rebuild(shard);
+        }
+    }
+
+    public void write(GTRecord r) throws IOException {
+        blockWriter.append(r);
+        writtenRowCount++;
+
+        if (block.isFull()) {
+            flush();
+        }
+    }
+
+    @Override
+    public void flush() throws IOException {
+        blockWriter.readyForFlush();
+        storeWriter.write(block);
+        writtenRowBlockCount++;
+        if (block.isFull()) {
+            blockWriter.clearForNext();
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (block.isEmpty() == false) {
+            flush();
+        }
+        storeWriter.close();
+    }
+
+    public int getWrittenRowCount() {
+        return writtenRowCount;
+    }
+
+    public int getWrittenRowBlockCount() {
+        return writtenRowBlockCount;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTFilterScanner.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTFilterScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTFilterScanner.java
new file mode 100644
index 0000000..21fb167
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTFilterScanner.java
@@ -0,0 +1,100 @@
+package org.apache.kylin.gridtable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.metadata.filter.IFilterCodeSystem;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.tuple.IEvaluatableTuple;
+
+public class GTFilterScanner implements IGTScanner {
+
+    final private IGTScanner inputScanner;
+    final private TupleFilter filter;
+    final private IEvaluatableTuple oneTuple; // avoid instance creation
+    
+    private GTRecord next = null;
+
+    public GTFilterScanner(IGTScanner inputScanner, GTScanRequest req) throws IOException {
+        this.inputScanner = inputScanner;
+        this.filter = req.getFilterPushDown();
+        this.oneTuple = new IEvaluatableTuple() {
+            @Override
+            public Object getValue(TblColRef col) {
+                return next.get(col.getColumnDesc().getZeroBasedIndex());
+            }
+        };
+
+        if (TupleFilter.isEvaluableRecursively(filter) == false)
+            throw new IllegalArgumentException();
+    }
+
+    @Override
+    public GTInfo getInfo() {
+        return inputScanner.getInfo();
+    }
+
+    @Override
+    public int getScannedRowCount() {
+        return inputScanner.getScannedRowCount();
+    }
+
+    @Override
+    public int getScannedRowBlockCount() {
+        return inputScanner.getScannedRowBlockCount();
+    }
+
+    @Override
+    public void close() throws IOException {
+        inputScanner.close();
+    }
+
+    @Override
+    public Iterator<GTRecord> iterator() {
+        return new Iterator<GTRecord>() {
+            
+            private Iterator<GTRecord> inputIterator = inputScanner.iterator();
+
+            @Override
+            public boolean hasNext() {
+                if (next != null)
+                    return true;
+
+                IFilterCodeSystem<ByteArray> filterCodeSystem = GTUtil.wrap(getInfo().codeSystem.getComparator());
+
+                while (inputIterator.hasNext()) {
+                    next = inputIterator.next();
+                    if (filter != null && filter.evaluate(oneTuple, filterCodeSystem) == false) {
+                        continue;
+                    }
+                    return true;
+                }
+                next = null;
+                return false;
+            }
+
+            @Override
+            public GTRecord next() {
+                // fetch next record
+                if (next == null) {
+                    hasNext();
+                    if (next == null)
+                        throw new NoSuchElementException();
+                }
+
+                GTRecord result = next;
+                next = null;
+                return result;
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTInfo.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTInfo.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTInfo.java
new file mode 100644
index 0000000..87d7811
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTInfo.java
@@ -0,0 +1,246 @@
+package org.apache.kylin.gridtable;
+
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class GTInfo {
+
+    public static Builder builder() {
+        return new Builder();
+    }
+
+    String tableName;
+    IGTCodeSystem codeSystem;
+
+    // column schema
+    int nColumns;
+    DataType[] colTypes;
+    ImmutableBitSet colAll;
+    ImmutableBitSet colPreferIndex;
+    transient TblColRef[] colRefs;
+
+    // grid info
+    ImmutableBitSet primaryKey; // order by, uniqueness is not required
+    ImmutableBitSet[] colBlocks; // primary key must be the first column block
+    ImmutableBitSet colBlocksAll;
+    int rowBlockSize; // 0: disable row block
+
+    // sharding
+    int nShards; // 0: no sharding
+
+    // must create from builder
+    private GTInfo() {
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public IGTCodeSystem getCodeSystem() {
+        return codeSystem;
+    }
+
+    public int getColumnCount() {
+        return nColumns;
+    }
+
+    public DataType getColumnType(int i) {
+        return colTypes[i];
+    }
+
+    public ImmutableBitSet getPrimaryKey() {
+        return primaryKey;
+    }
+
+    public boolean isShardingEnabled() {
+        return nShards > 0;
+    }
+
+    public boolean isRowBlockEnabled() {
+        return rowBlockSize > 0;
+    }
+
+    public int getRowBlockSize() {
+        return rowBlockSize;
+    }
+
+    public int getMaxRecordLength() {
+        return getMaxColumnLength(colAll);
+    }
+
+    public int getMaxColumnLength(ImmutableBitSet selectedCols) {
+        int result = 0;
+        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
+            int c = selectedCols.trueBitAt(i);
+            result += codeSystem.maxCodeLength(c);
+        }
+        return result;
+    }
+
+    public int getMaxColumnLength() {
+        int max = 0;
+        for (int i = 0; i < nColumns; i++)
+            max = Math.max(max, codeSystem.maxCodeLength(i));
+        return max;
+    }
+
+    public ImmutableBitSet selectColumnBlocks(ImmutableBitSet columns) {
+        if (columns == null)
+            columns = colAll;
+
+        BitSet result = new BitSet();
+        for (int i = 0; i < colBlocks.length; i++) {
+            ImmutableBitSet cb = colBlocks[i];
+            if (cb.intersects(columns)) {
+                result.set(i);
+            }
+        }
+        return new ImmutableBitSet(result);
+    }
+
+    public TblColRef colRef(int i) {
+        if (colRefs == null) {
+            colRefs = new TblColRef[nColumns];
+        }
+        if (colRefs[i] == null) {
+            colRefs[i] = GTUtil.tblColRef(i, colTypes[i].toString());
+        }
+        return colRefs[i];
+    }
+
+    public void validateColRef(TblColRef ref) {
+        TblColRef expected = colRef(ref.getColumnDesc().getZeroBasedIndex());
+        if (expected.equals(ref) == false)
+            throw new IllegalArgumentException();
+    }
+
+    void validate() {
+
+        if (codeSystem == null)
+            throw new IllegalStateException();
+
+        if (primaryKey.cardinality() == 0)
+            throw new IllegalStateException();
+
+        codeSystem.init(this);
+
+        validateColumnBlocks();
+    }
+
+    private void validateColumnBlocks() {
+        colAll = new ImmutableBitSet(0, nColumns);
+
+        if (colBlocks == null) {
+            colBlocks = new ImmutableBitSet[2];
+            colBlocks[0] = primaryKey;
+            colBlocks[1] = colAll.andNot(primaryKey);
+        }
+
+        colBlocksAll = new ImmutableBitSet(0, colBlocks.length);
+
+        if (colPreferIndex == null)
+            colPreferIndex = ImmutableBitSet.EMPTY;
+
+        // column blocks must not overlap
+        for (int i = 0; i < colBlocks.length; i++) {
+            for (int j = i + 1; j < colBlocks.length; j++) {
+                if (colBlocks[i].intersects(colBlocks[j]))
+                    throw new IllegalStateException();
+            }
+        }
+
+        // column block must cover all columns
+        ImmutableBitSet merge = ImmutableBitSet.EMPTY;
+        for (int i = 0; i < colBlocks.length; i++) {
+            merge = merge.or(colBlocks[i]);
+        }
+        if (merge.equals(colAll) == false)
+            throw new IllegalStateException();
+
+        // primary key must be the first column block
+        if (primaryKey.equals(colBlocks[0]) == false)
+            throw new IllegalStateException();
+
+        // drop empty column block
+        LinkedList<ImmutableBitSet> list = new LinkedList<ImmutableBitSet>(Arrays.asList(colBlocks));
+        Iterator<ImmutableBitSet> it = list.iterator();
+        while (it.hasNext()) {
+            ImmutableBitSet cb = it.next();
+            if (cb.isEmpty())
+                it.remove();
+        }
+        colBlocks = (ImmutableBitSet[]) list.toArray(new ImmutableBitSet[list.size()]);
+    }
+
+    public static class Builder {
+        final GTInfo info;
+
+        private Builder() {
+            this.info = new GTInfo();
+        }
+
+        /** optional */
+        public Builder setTableName(String name) {
+            info.tableName = name;
+            return this;
+        }
+
+        /** required */
+        public Builder setCodeSystem(IGTCodeSystem cs) {
+            info.codeSystem = cs;
+            return this;
+        }
+
+        /** required */
+        public Builder setColumns(DataType... colTypes) {
+            info.nColumns = colTypes.length;
+            info.colTypes = colTypes;
+            return this;
+        }
+
+        /** required */
+        public Builder setPrimaryKey(ImmutableBitSet primaryKey) {
+            info.primaryKey = primaryKey;
+            return this;
+        }
+
+        /** optional */
+        public Builder enableColumnBlock(ImmutableBitSet[] columnBlocks) {
+            info.colBlocks = new ImmutableBitSet[columnBlocks.length];
+            for (int i = 0; i < columnBlocks.length; i++) {
+                info.colBlocks[i] = columnBlocks[i];
+            }
+            return this;
+        }
+
+        /** optional */
+        public Builder enableRowBlock(int rowBlockSize) {
+            info.rowBlockSize = rowBlockSize;
+            return this;
+        }
+
+        /** optional */
+        public Builder enableSharding(int nShards) {
+            info.nShards = nShards;
+            return this;
+        }
+
+        /** optional */
+        public Builder setColumnPreferIndex(ImmutableBitSet colPreferIndex) {
+            info.colPreferIndex = colPreferIndex;
+            return this;
+        }
+
+        public GTInfo build() {
+            info.validate();
+            return info;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTInvertedIndex.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTInvertedIndex.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTInvertedIndex.java
new file mode 100644
index 0000000..4f99108
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTInvertedIndex.java
@@ -0,0 +1,205 @@
+package org.apache.kylin.gridtable;
+
+import it.uniroma3.mat.extendedset.intset.ConciseSet;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.LogicalTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+
+/**
+ * A thread-safe inverted index of row blocks in memory.
+ * 
+ * Note function not() must return all blocks, because index only know what block contains a value,
+ * but not sure what block does not contain a value.
+ * 
+ * @author yangli9
+ */
+public class GTInvertedIndex {
+
+    private final GTInfo info;
+    private final ImmutableBitSet colPreferIndex;
+    private final ImmutableBitSet colBlocks;
+    private final GTInvertedIndexOfColumn[] index; // for each column
+
+    private volatile int nIndexedBlocks;
+
+    public GTInvertedIndex(GTInfo info) {
+        this.info = info;
+        this.colPreferIndex = info.colPreferIndex;
+        this.colBlocks = info.selectColumnBlocks(colPreferIndex);
+
+        index = new GTInvertedIndexOfColumn[info.getColumnCount()];
+        for (int i = 0; i < colPreferIndex.trueBitCount(); i++) {
+            int c = colPreferIndex.trueBitAt(i);
+            index[c] = new GTInvertedIndexOfColumn(info.codeSystem.getComparator());
+        }
+    }
+
+    public void add(GTRowBlock block) {
+
+        @SuppressWarnings("unchecked")
+        Set<ByteArray>[] distinctValues = new Set[info.getColumnCount()];
+        for (int i = 0; i < colPreferIndex.trueBitCount(); i++) {
+            int c = colPreferIndex.trueBitAt(i);
+            distinctValues[c] = new HashSet<ByteArray>();
+        }
+
+        GTRowBlock.Reader reader = block.getReader(colBlocks);
+        GTRecord record = new GTRecord(info);
+        while (reader.hasNext()) {
+            reader.fetchNext(record);
+            for (int i = 0; i < colPreferIndex.trueBitCount(); i++) {
+                int c = colPreferIndex.trueBitAt(i);
+                distinctValues[c].add(record.get(c));
+            }
+        }
+
+        for (int i = 0; i < colPreferIndex.trueBitCount(); i++) {
+            int c = colPreferIndex.trueBitAt(i);
+            index[c].add(distinctValues[c], block.getSequenceId());
+        }
+
+        nIndexedBlocks = Math.max(nIndexedBlocks, block.seqId + 1);
+    }
+
+    public ConciseSet filter(TupleFilter filter) {
+        return filter(filter, nIndexedBlocks);
+    }
+
+    public ConciseSet filter(TupleFilter filter, int totalBlocks) {
+        // number of indexed blocks may increase as we do evaluation
+        int indexedBlocks = nIndexedBlocks;
+
+        Evaluator evaluator = new Evaluator(indexedBlocks);
+        ConciseSet r = evaluator.evaluate(filter);
+
+        // add blocks that have not been indexed
+        for (int i = indexedBlocks; i < totalBlocks; i++) {
+            r.add(i);
+        }
+
+        return r;
+    }
+
+    private class Evaluator {
+        private int indexedBlocks;
+
+        Evaluator(int indexedBlocks) {
+            this.indexedBlocks = indexedBlocks;
+        }
+
+        public ConciseSet evaluate(TupleFilter filter) {
+            if (filter == null) {
+                return all();
+            }
+
+            if (filter instanceof LogicalTupleFilter)
+                return evalLogical((LogicalTupleFilter) filter);
+
+            if (filter instanceof CompareTupleFilter)
+                return evalCompare((CompareTupleFilter) filter);
+
+            // unable to evaluate
+            return all();
+        }
+
+        @SuppressWarnings("unchecked")
+        private ConciseSet evalCompare(CompareTupleFilter filter) {
+            int col = col(filter);
+            if (index[col] == null)
+                return all();
+
+            switch (filter.getOperator()) {
+            case ISNULL:
+                return index[col].getNull();
+            case ISNOTNULL:
+                return all();
+            case EQ:
+                return index[col].getEquals((ByteArray) filter.getFirstValue());
+            case NEQ:
+                return all();
+            case IN:
+                return index[col].getIn((Iterable<ByteArray>) filter.getValues());
+            case NOTIN:
+                return all();
+            case LT:
+                return index[col].getRange(null, false, (ByteArray) filter.getFirstValue(), false);
+            case LTE:
+                return index[col].getRange(null, false, (ByteArray) filter.getFirstValue(), true);
+            case GT:
+                return index[col].getRange((ByteArray) filter.getFirstValue(), false, null, false);
+            case GTE:
+                return index[col].getRange((ByteArray) filter.getFirstValue(), true, null, false);
+            default:
+                throw new IllegalStateException("Unsupported operator " + filter.getOperator());
+            }
+        }
+
+        private ConciseSet evalLogical(LogicalTupleFilter filter) {
+            List<? extends TupleFilter> children = filter.getChildren();
+
+            switch (filter.getOperator()) {
+            case AND:
+                return evalLogicalAnd(children);
+            case OR:
+                return evalLogicalOr(children);
+            case NOT:
+                return evalLogicalNot(children);
+            default:
+                throw new IllegalStateException("Unsupported operator " + filter.getOperator());
+            }
+        }
+
+        private ConciseSet evalLogicalAnd(List<? extends TupleFilter> children) {
+            ConciseSet set = all();
+
+            for (TupleFilter c : children) {
+                ConciseSet t = evaluate(c);
+                if (t == null)
+                    continue; // because it's AND
+
+                set.retainAll(t);
+            }
+            return set;
+        }
+
+        private ConciseSet evalLogicalOr(List<? extends TupleFilter> children) {
+            ConciseSet set = new ConciseSet();
+
+            for (TupleFilter c : children) {
+                ConciseSet t = evaluate(c);
+                if (t == null)
+                    return null; // because it's OR
+
+                set.addAll(t);
+            }
+            return set;
+        }
+
+        private ConciseSet evalLogicalNot(List<? extends TupleFilter> children) {
+            return all();
+        }
+
+        private ConciseSet all() {
+            return not(new ConciseSet());
+        }
+
+        private ConciseSet not(ConciseSet set) {
+            set.add(indexedBlocks);
+            set.complement();
+            return set;
+        }
+
+        private int col(CompareTupleFilter filter) {
+            return filter.getColumn().getColumnDesc().getZeroBasedIndex();
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/gridtable/GTInvertedIndexOfColumn.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTInvertedIndexOfColumn.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTInvertedIndexOfColumn.java
new file mode 100644
index 0000000..4b094fd
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTInvertedIndexOfColumn.java
@@ -0,0 +1,115 @@
+package org.apache.kylin.gridtable;
+
+import it.uniroma3.mat.extendedset.intset.ConciseSet;
+
+import java.util.NavigableMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.kylin.common.util.ByteArray;
+
+import com.google.common.collect.Maps;
+
+public class GTInvertedIndexOfColumn {
+
+    final private IGTComparator comparator;
+    final private ReentrantReadWriteLock rwLock;
+
+    private int nBlocks;
+    private NavigableMap<ByteArray, ConciseSet> rangeIndex;
+    private ConciseSet nullIndex;
+
+    public GTInvertedIndexOfColumn(IGTComparator comparator) {
+        this.comparator = comparator;
+        this.rwLock = new ReentrantReadWriteLock();
+        this.rangeIndex = Maps.newTreeMap(comparator);
+        this.nullIndex = new ConciseSet();
+    }
+
+    public void add(Iterable<ByteArray> codes, int blockId) {
+        rwLock.writeLock().lock();
+        try {
+            for (ByteArray code : codes) {
+                if (comparator.isNull(code)) {
+                    nullIndex.add(blockId);
+                    continue;
+                }
+                ConciseSet set = rangeIndex.get(code);
+                if (set == null) {
+                    set = new ConciseSet();
+                    rangeIndex.put(code.copy(), set);
+                }
+                set.add(blockId);
+            }
+
+            if (blockId >= nBlocks) {
+                nBlocks = blockId + 1;
+            }
+
+        } finally {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    public ConciseSet getNull() {
+        rwLock.readLock().lock();
+        try {
+            return nullIndex.clone();
+        } finally {
+            rwLock.readLock().unlock();
+        }
+    }
+
+    public ConciseSet getEquals(ByteArray code) {
+        rwLock.readLock().lock();
+        try {
+            ConciseSet set = rangeIndex.get(code);
+            if (set == null)
+                return new ConciseSet();
+            else
+                return set.clone();
+        } finally {
+            rwLock.readLock().unlock();
+        }
+    }
+
+    public ConciseSet getIn(Iterable<ByteArray> codes) {
+        rwLock.readLock().lock();
+        try {
+            ConciseSet r = new ConciseSet();
+            for (ByteArray code : codes) {
+                ConciseSet set = rangeIndex.get(code);
+                if (set != null)
+                    r.addAll(set);
+            }
+            return r;
+        } finally {
+            rwLock.readLock().unlock();
+        }
+    }
+
+    public ConciseSet getRange(ByteArray from, boolean fromInclusive, ByteArray to, boolean toInclusive) {
+        rwLock.readLock().lock();
+        try {
+            ConciseSet r = new ConciseSet();
+            if (from == null && to == null) {
+                r.add(nBlocks);
+                r.complement();
+                return r;
+            }
+            NavigableMap<ByteArray, ConciseSet> subMap;
+            if (from == null) {
+                subMap = rangeIndex.headMap(to, toInclusive);
+            } else if (to == null) {
+                subMap = rangeIndex.tailMap(from, fromInclusive);
+            } else {
+                subMap = rangeIndex.subMap(from, fromInclusive, to, toInclusive);
+            }
+            for (ConciseSet set : subMap.values()) {
+                r.addAll(set);
+            }
+            return r;
+        } finally {
+            rwLock.readLock().unlock();
+        }
+    }
+}


[26/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive',
'storage-hbase'. The old job remains as an assembly project.


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

Branch: refs/heads/0.8
Commit: 2ef9831efd3ead32466c9163d6280718b56917c0
Parents: ccdc415
Author: Li, Yang <ya...@ebay.com>
Authored: Thu Jul 23 19:21:10 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Fri Jul 24 06:02:10 2015 +0800

----------------------------------------------------------------------
 core-common/pom.xml                             |   4 +
 core-cube/pom.xml                               |   5 +
 .../kylin/cube/gridtable/CubeCodeSystem.java    | 287 ++++++++
 .../kylin/cube/gridtable/CubeGridTable.java     |  70 ++
 .../gridtable/CuboidToGridTableMapping.java     | 172 +++++
 .../inmemcubing/AbstractInMemCubeBuilder.java   |  92 +++
 .../cube/inmemcubing/ConcurrentDiskStore.java   | 342 ++++++++++
 .../cube/inmemcubing/DoggedCubeBuilder.java     | 434 ++++++++++++
 .../kylin/cube/inmemcubing/ICuboidWriter.java   |  28 +
 .../cube/inmemcubing/InMemCubeBuilder.java      | 661 ++++++++++++++++++
 .../kylin/cube/inmemcubing/MemDiskStore.java    | 679 +++++++++++++++++++
 .../kylin/gridtable/GTAggregateScanner.java     | 268 ++++++++
 .../org/apache/kylin/gridtable/GTBuilder.java   |  74 ++
 .../apache/kylin/gridtable/GTFilterScanner.java | 100 +++
 .../java/org/apache/kylin/gridtable/GTInfo.java | 246 +++++++
 .../apache/kylin/gridtable/GTInvertedIndex.java | 205 ++++++
 .../gridtable/GTInvertedIndexOfColumn.java      | 115 ++++
 .../apache/kylin/gridtable/GTRawScanner.java    | 111 +++
 .../org/apache/kylin/gridtable/GTRecord.java    | 285 ++++++++
 .../org/apache/kylin/gridtable/GTRowBlock.java  | 259 +++++++
 .../kylin/gridtable/GTSampleCodeSystem.java     | 101 +++
 .../org/apache/kylin/gridtable/GTScanRange.java |  83 +++
 .../kylin/gridtable/GTScanRangePlanner.java     | 486 +++++++++++++
 .../apache/kylin/gridtable/GTScanRequest.java   | 155 +++++
 .../java/org/apache/kylin/gridtable/GTUtil.java | 221 ++++++
 .../org/apache/kylin/gridtable/GridTable.java   |  61 ++
 .../apache/kylin/gridtable/IGTCodeSystem.java   |  43 ++
 .../apache/kylin/gridtable/IGTComparator.java   |  15 +
 .../org/apache/kylin/gridtable/IGTScanner.java  |  13 +
 .../org/apache/kylin/gridtable/IGTStore.java    |  26 +
 .../apache/kylin/gridtable/UnitTestSupport.java | 101 +++
 .../gridtable/memstore/GTSimpleMemStore.java    | 112 +++
 .../inmemcubing/ConcurrentDiskStoreTest.java    |  94 +++
 .../DoggedCubeBuilderStressTest.java            |  97 +++
 .../cube/inmemcubing/DoggedCubeBuilderTest.java | 157 +++++
 .../cube/inmemcubing/InMemCubeBuilderTest.java  | 208 ++++++
 .../cube/inmemcubing/MemDiskStoreTest.java      |  98 +++
 .../inmemcubing/MemoryBudgetControllerTest.java |  98 +++
 .../gridtable/AggregationCacheMemSizeTest.java  | 214 ++++++
 .../kylin/gridtable/DictGridTableTest.java      | 381 +++++++++++
 .../kylin/gridtable/SimpleGridTableTest.java    | 195 ++++++
 .../gridtable/SimpleInvertedIndexTest.java      | 188 +++++
 core-job/pom.xml                                |  12 +
 .../apache/kylin/engine/BuildEngineFactory.java |  52 ++
 .../apache/kylin/engine/IBatchCubingEngine.java |  35 +
 .../kylin/engine/IStreamingCubingEngine.java    |   8 +
 .../java/org/apache/kylin/job/JobInstance.java  | 504 ++++++++++++++
 .../org/apache/kylin/job/JoinedFlatTable.java   | 243 +++++++
 .../java/org/apache/kylin/job/Scheduler.java    |  36 +
 .../apache/kylin/job/cmd/BaseCommandOutput.java |  29 +
 .../apache/kylin/job/cmd/ICommandOutput.java    |  44 ++
 .../org/apache/kylin/job/cmd/IJobCommand.java   |  32 +
 .../java/org/apache/kylin/job/cmd/ShellCmd.java | 104 +++
 .../apache/kylin/job/cmd/ShellCmdOutput.java    |  84 +++
 .../apache/kylin/job/common/OptionsHelper.java  |  79 +++
 .../kylin/job/common/ShellExecutable.java       | 143 ++++
 .../kylin/job/constant/ExecutableConstants.java |  80 +++
 .../kylin/job/constant/JobStatusEnum.java       |  49 ++
 .../kylin/job/constant/JobStepCmdTypeEnum.java  |  27 +
 .../kylin/job/constant/JobStepStatusEnum.java   |  51 ++
 .../org/apache/kylin/job/dao/ExecutableDao.java | 221 ++++++
 .../kylin/job/dao/ExecutableOutputPO.java       |  65 ++
 .../org/apache/kylin/job/dao/ExecutablePO.java  |  78 +++
 .../kylin/job/engine/JobEngineConfig.java       | 185 +++++
 .../kylin/job/exception/ExecuteException.java   |  41 ++
 .../exception/IllegalStateTranferException.java |  45 ++
 .../kylin/job/exception/JobException.java       |  58 ++
 .../kylin/job/exception/LockException.java      |  44 ++
 .../job/exception/PersistentException.java      |  44 ++
 .../kylin/job/exception/SchedulerException.java |  44 ++
 .../kylin/job/execution/AbstractExecutable.java | 302 +++++++++
 .../kylin/job/execution/ChainedExecutable.java  |  31 +
 .../job/execution/DefaultChainedExecutable.java | 122 ++++
 .../kylin/job/execution/DefaultOutput.java      |  97 +++
 .../apache/kylin/job/execution/Executable.java  |  42 ++
 .../kylin/job/execution/ExecutableContext.java  |  30 +
 .../kylin/job/execution/ExecutableState.java    |  82 +++
 .../kylin/job/execution/ExecuteResult.java      |  54 ++
 .../apache/kylin/job/execution/Idempotent.java  |  28 +
 .../org/apache/kylin/job/execution/Output.java  |  34 +
 .../job/impl/threadpool/DefaultContext.java     |  61 ++
 .../job/impl/threadpool/DefaultScheduler.java   | 209 ++++++
 .../kylin/job/manager/ExecutableManager.java    | 318 +++++++++
 .../apache/kylin/job/BaseTestExecutable.java    |  32 +
 .../apache/kylin/job/BasicLocalMetaTest.java    |  24 +
 .../apache/kylin/job/ErrorTestExecutable.java   |  41 ++
 .../apache/kylin/job/ExecutableManagerTest.java | 144 ++++
 .../apache/kylin/job/FailedTestExecutable.java  |  41 ++
 .../apache/kylin/job/JoinedFlatTableTest.java   |  88 +++
 .../apache/kylin/job/SelfStopExecutable.java    |  46 ++
 .../apache/kylin/job/SucceedTestExecutable.java |  41 ++
 core-storage/pom.xml                            |   6 +
 .../kylin/storage/ICachableStorageQuery.java    |  33 +
 .../java/org/apache/kylin/storage/IStorage.java |  28 +
 .../org/apache/kylin/storage/IStorageQuery.java |  36 +
 .../apache/kylin/storage/StorageContext.java    | 170 +++++
 .../apache/kylin/storage/StorageFactory2.java   |  34 +
 .../org/apache/kylin/storage/tuple/Tuple.java   | 198 ++++++
 .../apache/kylin/storage/tuple/TupleInfo.java   | 114 ++++
 engine-mr/pom.xml                               |  15 +
 .../kylin/engine/mr/BatchCubingJobBuilder.java  | 128 ++++
 .../kylin/engine/mr/BatchCubingJobBuilder2.java |  94 +++
 .../kylin/engine/mr/BatchMergeJobBuilder.java   |  88 +++
 .../kylin/engine/mr/BatchMergeJobBuilder2.java  |  99 +++
 .../kylin/engine/mr/ByteArrayWritable.java      | 166 +++++
 .../org/apache/kylin/engine/mr/CubingJob.java   | 188 +++++
 .../org/apache/kylin/engine/mr/IMRInput.java    |  69 ++
 .../org/apache/kylin/engine/mr/IMROutput.java   |  78 +++
 .../org/apache/kylin/engine/mr/IMROutput2.java  |  88 +++
 .../kylin/engine/mr/JobBuilderSupport.java      | 168 +++++
 .../kylin/engine/mr/MRBatchCubingEngine.java    |  47 ++
 .../kylin/engine/mr/MRBatchCubingEngine2.java   |  47 ++
 .../java/org/apache/kylin/engine/mr/MRUtil.java |  55 ++
 .../engine/mr/common/AbstractHadoopJob.java     | 379 +++++++++++
 .../kylin/engine/mr/common/BatchConstants.java  |  58 ++
 .../common/DefaultSslProtocolSocketFactory.java | 150 ++++
 .../mr/common/DefaultX509TrustManager.java      | 114 ++++
 .../kylin/engine/mr/common/HadoopCmdOutput.java | 105 +++
 .../engine/mr/common/HadoopShellExecutable.java |  95 +++
 .../engine/mr/common/HadoopStatusChecker.java   |  99 +++
 .../engine/mr/common/HadoopStatusGetter.java    | 114 ++++
 .../engine/mr/common/MapReduceExecutable.java   | 246 +++++++
 .../kylin/engine/mr/steps/BaseCuboidJob.java    |  39 ++
 .../engine/mr/steps/BaseCuboidMapperBase.java   | 205 ++++++
 .../engine/mr/steps/CreateDictionaryJob.java    |  75 ++
 .../apache/kylin/engine/mr/steps/CuboidJob.java | 200 ++++++
 .../kylin/engine/mr/steps/CuboidReducer.java    | 101 +++
 .../mr/steps/FactDistinctColumnsCombiner.java   |  63 ++
 .../engine/mr/steps/FactDistinctColumnsJob.java | 132 ++++
 .../mr/steps/FactDistinctColumnsMapperBase.java |  89 +++
 .../mr/steps/FactDistinctColumnsReducer.java    | 224 ++++++
 .../mr/steps/FactDistinctHiveColumnsMapper.java | 176 +++++
 .../engine/mr/steps/HiveToBaseCuboidMapper.java |  69 ++
 .../kylin/engine/mr/steps/InMemCuboidJob.java   | 103 +++
 .../engine/mr/steps/InMemCuboidMapper.java      | 122 ++++
 .../engine/mr/steps/InMemCuboidReducer.java     |  95 +++
 .../engine/mr/steps/KeyDistributionJob.java     | 157 +++++
 .../engine/mr/steps/KeyDistributionMapper.java  | 124 ++++
 .../engine/mr/steps/KeyDistributionReducer.java | 113 +++
 .../mr/steps/MapContextGTRecordWriter.java      |  96 +++
 .../mr/steps/MergeCuboidFromStorageJob.java     |  91 +++
 .../mr/steps/MergeCuboidFromStorageMapper.java  | 197 ++++++
 .../kylin/engine/mr/steps/MergeCuboidJob.java   | 100 +++
 .../engine/mr/steps/MergeCuboidMapper.java      | 192 ++++++
 .../engine/mr/steps/MergeDictionaryStep.java    | 197 ++++++
 .../engine/mr/steps/MergeStatisticsStep.java    | 188 +++++
 .../engine/mr/steps/MetadataCleanupJob.java     | 161 +++++
 .../kylin/engine/mr/steps/NDCuboidJob.java      |  39 ++
 .../kylin/engine/mr/steps/NDCuboidMapper.java   | 142 ++++
 .../mr/steps/RangeKeyDistributionJob.java       | 109 +++
 .../mr/steps/RangeKeyDistributionMapper.java    |  71 ++
 .../mr/steps/RangeKeyDistributionReducer.java   | 106 +++
 .../mr/steps/RowKeyDistributionCheckerJob.java  |  97 +++
 .../steps/RowKeyDistributionCheckerMapper.java  | 107 +++
 .../steps/RowKeyDistributionCheckerReducer.java |  51 ++
 .../engine/mr/steps/SaveStatisticsStep.java     | 110 +++
 .../mr/steps/UpdateCubeInfoAfterBuildStep.java  | 101 +++
 .../mr/steps/UpdateCubeInfoAfterMergeStep.java  | 136 ++++
 .../kylin/engine/mr/steps/CubeReducerTest.java  | 123 ++++
 .../kylin/engine/mr/steps/CubeSamplingTest.java | 134 ++++
 .../steps/FactDistinctColumnsReducerTest.java   |  37 +
 .../HiveToBaseCuboidMapperPerformanceTest.java  |  65 ++
 .../engine/mr/steps/MergeCuboidJobTest.java     |  88 +++
 .../engine/mr/steps/MergeCuboidMapperTest.java  | 194 ++++++
 .../kylin/engine/mr/steps/MockupMapContext.java | 313 +++++++++
 .../kylin/engine/mr/steps/NDCuboidJobTest.java  |  88 +++
 .../engine/mr/steps/NDCuboidMapperTest.java     | 119 ++++
 job/pom.xml                                     |   7 +
 .../apache/kylin/engine/BuildEngineFactory.java |  53 --
 .../apache/kylin/engine/IBatchCubingEngine.java |  35 -
 .../kylin/engine/IStreamingCubingEngine.java    |   8 -
 .../kylin/engine/mr/BatchCubingJobBuilder.java  | 128 ----
 .../kylin/engine/mr/BatchCubingJobBuilder2.java |  94 ---
 .../kylin/engine/mr/BatchMergeJobBuilder.java   |  88 ---
 .../kylin/engine/mr/BatchMergeJobBuilder2.java  |  98 ---
 .../kylin/engine/mr/ByteArrayWritable.java      | 166 -----
 .../org/apache/kylin/engine/mr/CubingJob.java   | 188 -----
 .../org/apache/kylin/engine/mr/IMRInput.java    |  69 --
 .../org/apache/kylin/engine/mr/IMROutput.java   |  78 ---
 .../org/apache/kylin/engine/mr/IMROutput2.java  |  88 ---
 .../kylin/engine/mr/JobBuilderSupport.java      | 168 -----
 .../kylin/engine/mr/MRBatchCubingEngine.java    |  47 --
 .../kylin/engine/mr/MRBatchCubingEngine2.java   |  47 --
 .../java/org/apache/kylin/engine/mr/MRUtil.java |  55 --
 .../kylin/engine/mr/steps/InMemCuboidJob.java   | 103 ---
 .../engine/mr/steps/InMemCuboidMapper.java      | 122 ----
 .../engine/mr/steps/InMemCuboidReducer.java     |  95 ---
 .../mr/steps/MapContextGTRecordWriter.java      |  96 ---
 .../mr/steps/MergeCuboidFromStorageJob.java     |  95 ---
 .../mr/steps/MergeCuboidFromStorageMapper.java  | 197 ------
 .../engine/mr/steps/MergeDictionaryStep.java    | 197 ------
 .../engine/mr/steps/MergeStatisticsStep.java    | 189 ------
 .../engine/mr/steps/SaveStatisticsStep.java     | 110 ---
 .../mr/steps/UpdateCubeInfoAfterBuildStep.java  | 101 ---
 .../mr/steps/UpdateCubeInfoAfterMergeStep.java  | 136 ----
 .../apache/kylin/job/CubeMetadataUpgrade.java   |  22 +-
 .../java/org/apache/kylin/job/JobInstance.java  | 504 --------------
 .../org/apache/kylin/job/JoinedFlatTable.java   | 236 -------
 .../java/org/apache/kylin/job/Scheduler.java    |  36 -
 .../apache/kylin/job/cmd/BaseCommandOutput.java |  29 -
 .../apache/kylin/job/cmd/ICommandOutput.java    |  44 --
 .../org/apache/kylin/job/cmd/IJobCommand.java   |  32 -
 .../java/org/apache/kylin/job/cmd/ShellCmd.java | 104 ---
 .../apache/kylin/job/cmd/ShellCmdOutput.java    |  84 ---
 .../kylin/job/common/HadoopCmdOutput.java       | 105 ---
 .../kylin/job/common/HadoopShellExecutable.java |  97 ---
 .../apache/kylin/job/common/HqlExecutable.java  | 106 ---
 .../kylin/job/common/MapReduceExecutable.java   | 248 -------
 .../kylin/job/common/ShellExecutable.java       | 143 ----
 .../kylin/job/constant/BatchConstants.java      |  58 --
 .../kylin/job/constant/ExecutableConstants.java |  80 ---
 .../kylin/job/constant/JobStatusEnum.java       |  49 --
 .../kylin/job/constant/JobStepCmdTypeEnum.java  |  27 -
 .../kylin/job/constant/JobStepStatusEnum.java   |  51 --
 .../org/apache/kylin/job/dao/ExecutableDao.java | 221 ------
 .../kylin/job/dao/ExecutableOutputPO.java       |  65 --
 .../org/apache/kylin/job/dao/ExecutablePO.java  |  78 ---
 .../job/deployment/HbaseConfigPrinterCLI.java   |   3 +-
 .../kylin/job/engine/JobEngineConfig.java       | 184 -----
 .../kylin/job/exception/ExecuteException.java   |  41 --
 .../exception/IllegalStateTranferException.java |  45 --
 .../kylin/job/exception/JobException.java       |  58 --
 .../kylin/job/exception/LockException.java      |  44 --
 .../job/exception/PersistentException.java      |  44 --
 .../kylin/job/exception/SchedulerException.java |  44 --
 .../kylin/job/execution/AbstractExecutable.java | 302 ---------
 .../kylin/job/execution/ChainedExecutable.java  |  31 -
 .../job/execution/DefaultChainedExecutable.java | 122 ----
 .../kylin/job/execution/DefaultOutput.java      |  97 ---
 .../apache/kylin/job/execution/Executable.java  |  42 --
 .../kylin/job/execution/ExecutableContext.java  |  30 -
 .../kylin/job/execution/ExecutableState.java    |  82 ---
 .../kylin/job/execution/ExecuteResult.java      |  54 --
 .../apache/kylin/job/execution/Idempotent.java  |  28 -
 .../org/apache/kylin/job/execution/Output.java  |  34 -
 .../kylin/job/hadoop/AbstractHadoopJob.java     | 401 -----------
 .../cardinality/ColumnCardinalityMapper.java    |   4 +-
 .../cardinality/HiveColumnCardinalityJob.java   |   4 +-
 .../HiveColumnCardinalityUpdateJob.java         |   3 +-
 .../kylin/job/hadoop/cube/BaseCuboidJob.java    |  39 --
 .../job/hadoop/cube/BaseCuboidMapperBase.java   | 205 ------
 .../kylin/job/hadoop/cube/CubeHFileJob.java     | 106 ---
 .../kylin/job/hadoop/cube/CubeHFileMapper.java  |  99 ---
 .../apache/kylin/job/hadoop/cube/CuboidJob.java | 200 ------
 .../kylin/job/hadoop/cube/CuboidReducer.java    | 101 ---
 .../cube/FactDistinctColumnsCombiner.java       |  63 --
 .../job/hadoop/cube/FactDistinctColumnsJob.java | 132 ----
 .../cube/FactDistinctColumnsMapperBase.java     |  89 ---
 .../hadoop/cube/FactDistinctColumnsReducer.java | 224 ------
 .../cube/FactDistinctHiveColumnsMapper.java     | 176 -----
 .../job/hadoop/cube/HiveToBaseCuboidMapper.java |  69 --
 .../job/hadoop/cube/KeyDistributionJob.java     | 157 -----
 .../job/hadoop/cube/KeyDistributionMapper.java  | 124 ----
 .../job/hadoop/cube/KeyDistributionReducer.java | 113 ---
 .../kylin/job/hadoop/cube/KeyValueCreator.java  | 104 ---
 .../kylin/job/hadoop/cube/MergeCuboidJob.java   | 101 ---
 .../job/hadoop/cube/MergeCuboidMapper.java      | 192 ------
 .../job/hadoop/cube/MetadataCleanupJob.java     | 160 -----
 .../kylin/job/hadoop/cube/NDCuboidJob.java      |  39 --
 .../kylin/job/hadoop/cube/NDCuboidMapper.java   | 142 ----
 .../job/hadoop/cube/NewBaseCuboidMapper.java    |   4 +-
 .../job/hadoop/cube/OrphanHBaseCleanJob.java    |   2 +-
 .../hadoop/cube/RangeKeyDistributionJob.java    | 110 ---
 .../hadoop/cube/RangeKeyDistributionMapper.java |  71 --
 .../cube/RangeKeyDistributionReducer.java       | 106 ---
 .../cube/RowKeyDistributionCheckerJob.java      |  98 ---
 .../cube/RowKeyDistributionCheckerMapper.java   | 107 ---
 .../cube/RowKeyDistributionCheckerReducer.java  |  51 --
 .../job/hadoop/cube/StorageCleanupJob.java      |   2 +-
 .../job/hadoop/dict/CreateDictionaryJob.java    |  75 --
 .../dict/CreateInvertedIndexDictionaryJob.java  |   2 +-
 .../kylin/job/hadoop/hbase/BulkLoadJob.java     | 101 ---
 .../kylin/job/hadoop/hbase/CreateHTableJob.java | 314 ---------
 .../kylin/job/hadoop/hbase/CubeHTableUtil.java  |  84 ---
 .../job/hadoop/hive/SqlHiveDataTypeMapping.java |  33 -
 .../job/hadoop/invertedindex/IIBulkLoadJob.java |   2 +-
 .../hadoop/invertedindex/IICreateHFileJob.java  |   2 +-
 .../hadoop/invertedindex/IICreateHTableJob.java |   6 +-
 .../invertedindex/IIDistinctColumnsJob.java     |   4 +-
 .../invertedindex/IIDistinctColumnsReducer.java |   2 +-
 .../hadoop/invertedindex/InvertedIndexJob.java  |  27 +-
 .../invertedindex/InvertedIndexMapper.java      |   4 +-
 .../invertedindex/InvertedIndexPartitioner.java |   4 +-
 .../invertedindex/InvertedIndexReducer.java     |   4 +-
 .../invertedindex/RandomKeyDistributionJob.java | 116 ----
 .../RandomKeyDistributionMapper.java            |  69 --
 .../RandomKeyDistributionReducer.java           |  67 --
 .../job/impl/threadpool/DefaultContext.java     |  61 --
 .../job/impl/threadpool/DefaultScheduler.java   | 209 ------
 .../inmemcubing/AbstractInMemCubeBuilder.java   |  92 ---
 .../job/inmemcubing/ConcurrentDiskStore.java    | 342 ----------
 .../job/inmemcubing/DoggedCubeBuilder.java      | 427 ------------
 .../kylin/job/inmemcubing/ICuboidWriter.java    |  28 -
 .../kylin/job/inmemcubing/InMemCubeBuilder.java | 661 ------------------
 .../kylin/job/inmemcubing/MemDiskStore.java     | 679 -------------------
 .../kylin/job/invertedindex/IIJobBuilder.java   |   4 +-
 .../kylin/job/manager/ExecutableManager.java    | 318 ---------
 .../kylin/job/streaming/CubeStreamConsumer.java |  16 +-
 .../apache/kylin/job/tools/CleanHtableCLI.java  |   3 +-
 .../tools/DefaultSslProtocolSocketFactory.java  | 150 ----
 .../job/tools/DefaultX509TrustManager.java      | 114 ----
 .../kylin/job/tools/DeployCoprocessorCLI.java   | 317 ---------
 .../kylin/job/tools/HadoopStatusChecker.java    |  99 ---
 .../kylin/job/tools/HadoopStatusGetter.java     | 116 ----
 .../kylin/job/tools/HtableAlterMetadataCLI.java |   2 +-
 .../kylin/job/tools/LZOSupportnessChecker.java  |  46 --
 .../apache/kylin/job/tools/OptionsHelper.java   |  81 ---
 .../apache/kylin/source/hive/HiveMRInput.java   | 174 -----
 .../source/hive/HiveSourceTableLoader.java      | 155 -----
 .../org/apache/kylin/source/hive/HiveTable.java |  99 ---
 .../kylin/source/hive/HiveTableReader.java      | 176 -----
 .../kylin/source/hive/HiveTableSource.java      |  43 --
 .../apache/kylin/storage/StorageFactory2.java   |  34 -
 .../kylin/storage/hbase/HBaseMROutput.java      |  60 --
 .../kylin/storage/hbase/HBaseMROutput2.java     | 298 --------
 .../kylin/storage/hbase/HBaseMRSteps.java       | 138 ----
 .../kylin/storage/hbase/HBaseStorage.java       |  46 --
 .../storage/hbase/InMemKeyValueCreator.java     |  73 --
 .../apache/kylin/storage/hbase/MergeGCStep.java | 121 ----
 .../apache/kylin/job/BaseTestExecutable.java    |  32 -
 .../apache/kylin/job/BasicLocalMetaTest.java    |  24 -
 .../kylin/job/BuildCubeWithEngineTest.java      |   2 +-
 .../apache/kylin/job/BuildIIWithEngineTest.java |   2 +-
 .../java/org/apache/kylin/job/DeployUtil.java   |  22 +-
 .../apache/kylin/job/ErrorTestExecutable.java   |  41 --
 .../apache/kylin/job/FailedTestExecutable.java  |  41 --
 .../apache/kylin/job/SelfStopExecutable.java    |  46 --
 .../apache/kylin/job/SucceedTestExecutable.java |  41 --
 .../apache/kylin/job/hadoop/cube/CopySeq.java   |  81 ---
 .../job/hadoop/cube/CubeHFileMapper2Test.java   |  94 ---
 .../job/hadoop/cube/CubeHFileMapperTest.java    |  81 ---
 .../kylin/job/hadoop/cube/CubeReducerTest.java  | 122 ----
 .../HiveToBaseCuboidMapperPerformanceTest.java  |  65 --
 .../job/hadoop/cube/KeyDistributionJobTest.java |  82 ---
 .../hadoop/cube/KeyDistributionMapperTest.java  | 171 -----
 .../job/hadoop/cube/MergeCuboidJobTest.java     |  88 ---
 .../job/hadoop/cube/MergeCuboidMapperTest.java  | 193 ------
 .../kylin/job/hadoop/cube/MockupMapContext.java | 314 ---------
 .../kylin/job/hadoop/cube/NDCuboidJobTest.java  |  88 ---
 .../job/hadoop/cube/NDCuboidMapperTest.java     | 115 ----
 .../cube/RandomKeyDistributionMapperTest.java   |  72 --
 .../cube/RandomKeyDistributionReducerTest.java  |  70 --
 .../cube/RangeKeyDistributionJobTest.java       |  71 --
 .../cube/RangeKeyDistributionMapperTest.java    | 111 ---
 .../cube/RangeKeyDistributionReducerTest.java   |  49 --
 .../job/hadoop/cubev2/CubeSamplingTest.java     | 134 ----
 .../cubev2/FactDistinctColumnsReducerTest.java  |  37 -
 .../job/hadoop/hbase/CreateHTableTest.java      |  66 --
 .../kylin/job/hadoop/hbase/TestHbaseClient.java | 101 ---
 .../kylin/job/hadoop/hdfs/ITHdfsOpsTest.java    |  66 --
 .../job/hadoop/hive/JoinedFlatTableTest.java    |  88 ---
 .../job/impl/threadpool/BaseSchedulerTest.java  |   2 +-
 .../inmemcubing/ConcurrentDiskStoreTest.java    |  93 ---
 .../DoggedCubeBuilderStressTest.java            |  95 ---
 .../job/inmemcubing/DoggedCubeBuilderTest.java  | 154 -----
 .../job/inmemcubing/InMemCubeBuilderTest.java   | 207 ------
 .../kylin/job/inmemcubing/MemDiskStoreTest.java |  97 ---
 .../inmemcubing/MemoryBudgetControllerTest.java |  98 ---
 .../job/manager/ExecutableManagerTest.java      | 143 ----
 .../apache/kylin/rest/service/CubeService.java  |   4 +-
 .../apache/kylin/rest/service/JobService.java   |   4 +-
 source-hive/pom.xml                             |   5 +
 .../apache/kylin/source/hive/HiveMRInput.java   | 174 +++++
 .../source/hive/HiveSourceTableLoader.java      | 155 +++++
 .../org/apache/kylin/source/hive/HiveTable.java |  99 +++
 .../kylin/source/hive/HiveTableReader.java      | 176 +++++
 .../kylin/source/hive/HiveTableSource.java      |  43 ++
 .../apache/kylin/source/hive/HqlExecutable.java | 105 +++
 .../job/spark/AbstractSparkApplication.java     |   2 +-
 .../apache/kylin/job/spark/SparkCountDemo.java  |   3 +-
 .../apache/kylin/job/spark/SparkHelloWorld.java |   2 +-
 .../apache/kylin/job/spark/SparkHiveDemo.java   |   2 +-
 storage-hbase/pom.xml                           |  11 +
 .../kylin/storage/hbase/HBaseMROutput.java      |  61 ++
 .../kylin/storage/hbase/HBaseMROutput2.java     | 297 ++++++++
 .../hbase/HBaseRegionSizeCalculator.java        | 128 ----
 .../kylin/storage/hbase/HBaseStorage.java       |  46 ++
 .../kylin/storage/hbase/RowValueDecoder.java    | 131 ----
 .../kylin/storage/hbase/ZookeeperJobLock.java   |  83 ---
 .../kylin/storage/hbase/steps/BulkLoadJob.java  |  99 +++
 .../storage/hbase/steps/CreateHTableJob.java    | 314 +++++++++
 .../kylin/storage/hbase/steps/CubeHFileJob.java | 106 +++
 .../storage/hbase/steps/CubeHFileMapper.java    |  99 +++
 .../storage/hbase/steps/CubeHTableUtil.java     |  85 +++
 .../kylin/storage/hbase/steps/HBaseMRSteps.java | 135 ++++
 .../hbase/steps/InMemKeyValueCreator.java       |  73 ++
 .../storage/hbase/steps/KeyValueCreator.java    | 104 +++
 .../kylin/storage/hbase/steps/MergeGCStep.java  | 121 ++++
 .../storage/hbase/steps/RowValueDecoder.java    | 131 ++++
 .../hbase/util/DeployCoprocessorCLI.java        | 317 +++++++++
 .../hbase/util/HBaseRegionSizeCalculator.java   | 128 ++++
 .../hbase/util/LZOSupportnessChecker.java       |  46 ++
 .../storage/hbase/util/ZookeeperJobLock.java    |  84 +++
 .../kylin/storage/hbase/CreateHTableTest.java   |  66 ++
 .../kylin/storage/hbase/ITHdfsOpsTest.java      |  65 ++
 .../storage/hbase/RowValueDecoderTest.java      |   1 +
 .../kylin/storage/hbase/TestHbaseClient.java    | 101 +++
 .../hbase/steps/CubeHFileMapper2Test.java       |  93 +++
 .../hbase/steps/CubeHFileMapperTest.java        |  81 +++
 .../storage/hbase/steps/MockupMapContext.java   | 313 +++++++++
 .../steps/RangeKeyDistributionJobTest.java      |  71 ++
 .../steps/RangeKeyDistributionMapperTest.java   | 112 +++
 .../steps/RangeKeyDistributionReducerTest.java  |  50 ++
 storage/pom.xml                                 |   5 -
 .../kylin/storage/ICachableStorageQuery.java    |  33 -
 .../java/org/apache/kylin/storage/IStorage.java |  28 -
 .../org/apache/kylin/storage/IStorageQuery.java |  36 -
 .../apache/kylin/storage/StorageContext.java    | 170 -----
 .../kylin/storage/cube/CubeCodeSystem.java      | 287 --------
 .../kylin/storage/cube/CubeGridTable.java       |  70 --
 .../storage/cube/CubeHBaseReadonlyStore.java    |  12 +-
 .../apache/kylin/storage/cube/CubeScanner.java  |  18 +-
 .../kylin/storage/cube/CubeTupleConverter.java  |   2 +-
 .../storage/cube/CuboidToGridTableMapping.java  | 172 -----
 .../cube/SequentialCubeTupleIterator.java       |   2 +-
 .../storage/gridtable/GTAggregateScanner.java   | 268 --------
 .../kylin/storage/gridtable/GTBuilder.java      |  74 --
 .../kylin/storage/gridtable/GTComboStore.java   | 115 ----
 .../storage/gridtable/GTFilterScanner.java      | 100 ---
 .../apache/kylin/storage/gridtable/GTInfo.java  | 246 -------
 .../storage/gridtable/GTInvertedIndex.java      | 205 ------
 .../gridtable/GTInvertedIndexOfColumn.java      | 115 ----
 .../kylin/storage/gridtable/GTRawScanner.java   | 111 ---
 .../kylin/storage/gridtable/GTRecord.java       | 285 --------
 .../kylin/storage/gridtable/GTRowBlock.java     | 259 -------
 .../storage/gridtable/GTSampleCodeSystem.java   | 101 ---
 .../kylin/storage/gridtable/GTScanRange.java    |  83 ---
 .../storage/gridtable/GTScanRangePlanner.java   | 486 -------------
 .../kylin/storage/gridtable/GTScanRequest.java  | 155 -----
 .../apache/kylin/storage/gridtable/GTUtil.java  | 221 ------
 .../kylin/storage/gridtable/GridTable.java      |  61 --
 .../kylin/storage/gridtable/IGTCodeSystem.java  |  43 --
 .../kylin/storage/gridtable/IGTComparator.java  |  15 -
 .../kylin/storage/gridtable/IGTScanner.java     |  13 -
 .../kylin/storage/gridtable/IGTStore.java       |  26 -
 .../storage/gridtable/UnitTestSupport.java      | 101 ---
 .../storage/gridtable/diskstore/FileSystem.java |  23 -
 .../gridtable/diskstore/GTDiskStore.java        | 179 -----
 .../gridtable/diskstore/HadoopFileSystem.java   |  97 ---
 .../gridtable/diskstore/LocalFileSystem.java    |  67 --
 .../gridtable/memstore/GTSimpleMemStore.java    | 112 ---
 .../storage/hbase/CubeSegmentTupleIterator.java |   1 +
 .../kylin/storage/hbase/CubeStorageQuery.java   |   1 +
 .../kylin/storage/hbase/CubeTupleConverter.java |   1 +
 .../hbase/SerializedHBaseTupleIterator.java     |   1 +
 .../observer/ObserverAggregators.java           |   2 +-
 .../coprocessor/observer/ObserverEnabler.java   |   2 +-
 .../org/apache/kylin/storage/tuple/Tuple.java   | 198 ------
 .../apache/kylin/storage/tuple/TupleInfo.java   | 114 ----
 .../gridtable/AggregationCacheMemSizeTest.java  | 213 ------
 .../storage/gridtable/DictGridTableTest.java    | 381 -----------
 .../storage/gridtable/SimpleGridTableTest.java  | 188 -----
 .../gridtable/SimpleInvertedIndexTest.java      | 183 -----
 453 files changed, 24835 insertions(+), 25665 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-common/pom.xml
----------------------------------------------------------------------
diff --git a/core-common/pom.xml b/core-common/pom.xml
index adb189e..06db315 100644
--- a/core-common/pom.xml
+++ b/core-common/pom.xml
@@ -56,6 +56,10 @@
             <artifactId>commons-configuration</artifactId>
         </dependency>
         <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-email</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/pom.xml
----------------------------------------------------------------------
diff --git a/core-cube/pom.xml b/core-cube/pom.xml
index 2b7177a..b74b69b 100644
--- a/core-cube/pom.xml
+++ b/core-cube/pom.xml
@@ -44,6 +44,11 @@
             <version>${project.parent.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>com.n3twork.druid</groupId>
+            <artifactId>extendedset</artifactId>
+        </dependency>
+
         <!-- Env & Test -->
         <dependency>
             <groupId>org.apache.kylin</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeCodeSystem.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeCodeSystem.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeCodeSystem.java
new file mode 100644
index 0000000..7a8c364
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeCodeSystem.java
@@ -0,0 +1,287 @@
+package org.apache.kylin.cube.gridtable;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.IGTCodeSystem;
+import org.apache.kylin.gridtable.IGTComparator;
+import org.apache.kylin.metadata.measure.MeasureAggregator;
+import org.apache.kylin.metadata.measure.serializer.DataTypeSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Created by shaoshi on 3/23/15.
+ * This implementation uses Dictionary to encode and decode the table; If a column doesn't have dictionary, will check
+ * its data type to serialize/deserialize it;
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class CubeCodeSystem implements IGTCodeSystem {
+    private static final Logger logger = LoggerFactory.getLogger(CubeCodeSystem.class);
+
+    // ============================================================================
+
+    private GTInfo info;
+    private Map<Integer, Dictionary> dictionaryMap; // column index ==> dictionary of column
+    private Map<Integer, Integer> fixLenMap; // column index ==> fixed length of column
+    private Map<Integer, Integer> dependentMetricsMap;
+    private IGTComparator comparator;
+    private DataTypeSerializer[] serializers;
+
+    public CubeCodeSystem(Map<Integer, Dictionary> dictionaryMap) {
+        this(dictionaryMap, Collections.<Integer, Integer>emptyMap(), Collections.<Integer, Integer>emptyMap());
+    }
+            
+    public CubeCodeSystem(Map<Integer, Dictionary> dictionaryMap, Map<Integer, Integer> fixLenMap, Map<Integer, Integer> dependentMetricsMap) {
+        this.dictionaryMap = dictionaryMap;
+        this.fixLenMap = fixLenMap;
+        this.dependentMetricsMap = dependentMetricsMap;
+    }
+
+    @Override
+    public void init(GTInfo info) {
+        this.info = info;
+
+        serializers = new DataTypeSerializer[info.getColumnCount()];
+        for (int i = 0; i < info.getColumnCount(); i++) {
+            // dimension with dictionary
+            if (dictionaryMap.get(i) != null) {
+                serializers[i] = new DictionarySerializer(dictionaryMap.get(i));
+            }
+            // dimension of fixed length
+            else if (fixLenMap.get(i) != null) {
+                serializers[i] = new FixLenSerializer(fixLenMap.get(i));
+            }
+            // metrics
+            else {
+                serializers[i] = DataTypeSerializer.create(info.getColumnType(i));
+            }
+        }
+
+        this.comparator = new IGTComparator() {
+            @Override
+            public boolean isNull(ByteArray code) {
+                // all 0xff is null
+                byte[] array = code.array();
+                for (int i = 0, j = code.offset(), n = code.length(); i < n; i++, j++) {
+                    if (array[j] != Dictionary.NULL)
+                        return false;
+                }
+                return true;
+            }
+
+            @Override
+            public int compare(ByteArray code1, ByteArray code2) {
+                return code1.compareTo(code2);
+            }
+        };
+    }
+
+    @Override
+    public IGTComparator getComparator() {
+        return comparator;
+    }
+
+    @Override
+    public int codeLength(int col, ByteBuffer buf) {
+        return serializers[col].peekLength(buf);
+    }
+
+    @Override
+    public int maxCodeLength(int col) {
+        return serializers[col].maxLength();
+    }
+
+    @Override
+    public void encodeColumnValue(int col, Object value, ByteBuffer buf) {
+        encodeColumnValue(col, value, 0, buf);
+    }
+
+    @Override
+    public void encodeColumnValue(int col, Object value, int roundingFlag, ByteBuffer buf) {
+        // this is a bit too complicated, but encoding only happens once at build time, so it is OK
+        DataTypeSerializer serializer = serializers[col];
+        try {
+            if (serializer instanceof DictionarySerializer) {
+                ((DictionarySerializer) serializer).serializeWithRounding(value, roundingFlag, buf);
+            } else {
+                serializer.serialize(value, buf);
+            }
+        } catch (ClassCastException ex) {
+            // try convert string into a correct object type
+            try {
+                if (value instanceof String) {
+                    Object converted = serializer.valueOf((String) value);
+                    if ((converted instanceof String) == false) {
+                        encodeColumnValue(col, converted, roundingFlag, buf);
+                        return;
+                    }
+                }
+            } catch (Throwable e) {
+                logger.error("Fail to encode value '" + value + "'", e);
+            }
+            throw ex;
+        }
+    }
+
+    @Override
+    public Object decodeColumnValue(int col, ByteBuffer buf) {
+       return serializers[col].deserialize(buf);
+    }
+
+    @Override
+    public MeasureAggregator<?>[] newMetricsAggregators(ImmutableBitSet columns, String[] aggrFunctions) {
+        assert columns.trueBitCount() == aggrFunctions.length;
+        
+        MeasureAggregator<?>[] result = new MeasureAggregator[aggrFunctions.length];
+        for (int i = 0; i < result.length; i++) {
+            int col = columns.trueBitAt(i);
+            result[i] = MeasureAggregator.create(aggrFunctions[i], info.getColumnType(col).toString());
+        }
+        
+        // deal with holistic distinct count
+        if (dependentMetricsMap != null) {
+            for (Integer child : dependentMetricsMap.keySet()) {
+                if (columns.get(child)) {
+                    Integer parent = dependentMetricsMap.get(child);
+                    if (columns.get(parent) == false)
+                        throw new IllegalStateException();
+                    
+                    int childIdx = columns.trueBitIndexOf(child);
+                    int parentIdx = columns.trueBitIndexOf(parent);
+                    result[childIdx].setDependentAggregator(result[parentIdx]);
+                }
+            }
+        }
+        
+        return result;
+    }
+
+    static class DictionarySerializer extends DataTypeSerializer {
+        private Dictionary dictionary;
+
+        DictionarySerializer(Dictionary dictionary) {
+            this.dictionary = dictionary;
+        }
+
+        public void serializeWithRounding(Object value, int roundingFlag, ByteBuffer buf) {
+            int id = dictionary.getIdFromValue(value, roundingFlag);
+            BytesUtil.writeUnsigned(id, dictionary.getSizeOfId(), buf);
+        }
+
+        @Override
+        public void serialize(Object value, ByteBuffer buf) {
+            int id = dictionary.getIdFromValue(value);
+            BytesUtil.writeUnsigned(id, dictionary.getSizeOfId(), buf);
+        }
+
+        @Override
+        public Object deserialize(ByteBuffer in) {
+            int id = BytesUtil.readUnsigned(in, dictionary.getSizeOfId());
+            return dictionary.getValueFromId(id);
+        }
+
+        @Override
+        public int peekLength(ByteBuffer in) {
+            return dictionary.getSizeOfId();
+        }
+
+        @Override
+        public int maxLength() {
+            return dictionary.getSizeOfId();
+        }
+
+        @Override
+        public Object valueOf(byte[] value) {
+            throw new UnsupportedOperationException();
+        }
+    }
+
+    static class FixLenSerializer extends DataTypeSerializer {
+
+        // be thread-safe and avoid repeated obj creation
+        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
+
+        private int fixLen;
+
+        FixLenSerializer(int fixLen) {
+            this.fixLen = fixLen;
+        }
+        
+        private byte[] currentBuf() {
+            byte[] buf = current.get();
+            if (buf == null) {
+                buf = new byte[fixLen];
+                current.set(buf);
+            }
+            return buf;
+        }
+
+        @Override
+        public void serialize(Object value, ByteBuffer out) {
+            byte[] buf = currentBuf();
+            if (value == null) {
+                Arrays.fill(buf, Dictionary.NULL);
+                out.put(buf);
+            } else {
+                byte[] bytes = Bytes.toBytes(value.toString());
+                if (bytes.length > fixLen) {
+                    throw new IllegalStateException("Expect at most " + fixLen + " bytes, but got " + bytes.length + ", value string: " + value.toString());
+                }
+                out.put(bytes);
+                for (int i = bytes.length; i < fixLen; i++) {
+                    out.put(RowConstants.ROWKEY_PLACE_HOLDER_BYTE);
+                }
+            }
+        }
+
+        @Override
+        public Object deserialize(ByteBuffer in) {
+            byte[] buf = currentBuf();
+            in.get(buf);
+
+            int tail = fixLen;
+            while (tail > 0 && (buf[tail - 1] == RowConstants.ROWKEY_PLACE_HOLDER_BYTE || buf[tail - 1] == Dictionary.NULL)) {
+                tail--;
+            }
+            
+            if (tail == 0) {
+                return buf[0] == Dictionary.NULL ? null : "";
+            }
+            
+            return Bytes.toString(buf, 0, tail);
+        }
+
+        @Override
+        public int peekLength(ByteBuffer in) {
+            return fixLen;
+        }
+
+        @Override
+        public int maxLength() {
+            return fixLen;
+        }
+
+        @Override
+        public Object valueOf(byte[] value) {
+            try {
+                return new String(value, "UTF-8");
+            } catch (UnsupportedEncodingException e) {
+                // does not happen
+                throw new RuntimeException(e);
+            }
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeGridTable.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeGridTable.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeGridTable.java
new file mode 100644
index 0000000..66466a3
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeGridTable.java
@@ -0,0 +1,70 @@
+package org.apache.kylin.cube.gridtable;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.Maps;
+
+@SuppressWarnings("rawtypes")
+public class CubeGridTable {
+
+    public static Map<TblColRef, Dictionary<?>> getDimensionToDictionaryMap(CubeSegment cubeSeg, long cuboidId) {
+        CubeDesc cubeDesc = cubeSeg.getCubeDesc();
+        CubeManager cubeMgr = CubeManager.getInstance(cubeSeg.getCubeInstance().getConfig());
+
+        // build a dictionary map
+        Map<TblColRef, Dictionary<?>> dictionaryMap = Maps.newHashMap();
+        List<TblColRef> dimCols = Cuboid.findById(cubeDesc, cuboidId).getColumns();
+        for (TblColRef col : dimCols) {
+            Dictionary<?> dictionary = cubeMgr.getDictionary(cubeSeg, col);
+            if (dictionary != null) {
+                dictionaryMap.put(col, dictionary);
+            }
+        }
+        return dictionaryMap;
+    }
+
+    public static GTInfo newGTInfo(CubeSegment cubeSeg, long cuboidId) {
+        Map<TblColRef, Dictionary<?>> dictionaryMap = getDimensionToDictionaryMap(cubeSeg, cuboidId);
+        return newGTInfo(cubeSeg.getCubeDesc(), cuboidId, dictionaryMap);
+    }
+    
+    public static GTInfo newGTInfo(CubeDesc cubeDesc, long cuboidId, Map<TblColRef, Dictionary<?>> dictionaryMap) {
+        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidId);
+        CuboidToGridTableMapping mapping = new CuboidToGridTableMapping(cuboid);
+        
+        Map<Integer, Dictionary> dictionaryByColIdx = Maps.newHashMap();
+        Map<Integer, Integer> fixLenByColIdx = Maps.newHashMap();
+
+        for (TblColRef dim : cuboid.getColumns()) {
+            int colIndex = mapping.getIndexOf(dim);
+            if (cubeDesc.getRowkey().isUseDictionary(dim)) {
+                Dictionary dict = dictionaryMap.get(dim);
+                dictionaryByColIdx.put(colIndex, dict);
+            } else {
+                int len = cubeDesc.getRowkey().getColumnLength(dim);
+                if (len == 0)
+                    throw new IllegalStateException();
+                
+                fixLenByColIdx.put(colIndex,  len);
+            }
+        }
+
+        GTInfo.Builder builder = GTInfo.builder();
+        builder.setTableName("Cuboid " + cuboidId);
+        builder.setCodeSystem(new CubeCodeSystem(dictionaryByColIdx, fixLenByColIdx, mapping.getDependentMetricsMap()));
+        builder.setColumns(mapping.getDataTypes());
+        builder.setPrimaryKey(mapping.getPrimaryKey());
+        builder.enableColumnBlock(mapping.getColumnBlocks());
+        return builder.build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CuboidToGridTableMapping.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CuboidToGridTableMapping.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CuboidToGridTableMapping.java
new file mode 100644
index 0000000..cf3a119
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CuboidToGridTableMapping.java
@@ -0,0 +1,172 @@
+package org.apache.kylin.cube.gridtable;
+
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class CuboidToGridTableMapping {
+
+    final private Cuboid cuboid;
+
+    private List<DataType> gtDataTypes;
+    private List<ImmutableBitSet> gtColBlocks;
+
+    private int nDimensions;
+    private Map<TblColRef, Integer> dim2gt;
+    private ImmutableBitSet gtPrimaryKey;
+
+    private int nMetrics;
+    private ListMultimap<FunctionDesc, Integer> metrics2gt; // because count distinct may have a holistic version
+
+    public CuboidToGridTableMapping(Cuboid cuboid) {
+        this.cuboid = cuboid;
+        init();
+    }
+    
+    private void init() {
+        int gtColIdx = 0;
+        gtDataTypes = Lists.newArrayList();
+        gtColBlocks = Lists.newArrayList();
+
+        // dimensions
+        dim2gt = Maps.newHashMap();
+        BitSet pk = new BitSet();
+        for (TblColRef dimension : cuboid.getColumns()) {
+            gtDataTypes.add(dimension.getType());
+            dim2gt.put(dimension, gtColIdx);
+            pk.set(gtColIdx);
+            gtColIdx++;
+        }
+        gtPrimaryKey = new ImmutableBitSet(pk);
+        gtColBlocks.add(gtPrimaryKey);
+
+        nDimensions = gtColIdx;
+        assert nDimensions == cuboid.getColumns().size();
+
+        // metrics
+        metrics2gt = LinkedListMultimap.create();
+        for (HBaseColumnFamilyDesc familyDesc : cuboid.getCube().getHbaseMapping().getColumnFamily()) {
+            for (HBaseColumnDesc hbaseColDesc : familyDesc.getColumns()) {
+                BitSet colBlock = new BitSet();
+                for (MeasureDesc measure : hbaseColDesc.getMeasures()) {
+                    // Count distinct & holistic count distinct are equals() but different.
+                    // Ensure the holistic version if exists is always the first.
+                    FunctionDesc func = measure.getFunction();
+                    if (func.isHolisticCountDistinct()) {
+                        List<Integer> existing = metrics2gt.removeAll(func);
+                        metrics2gt.put(func, gtColIdx);
+                        metrics2gt.putAll(func, existing);
+                    } else {
+                        metrics2gt.put(func, gtColIdx);
+                    }
+                    gtDataTypes.add(func.getReturnDataType());
+                    colBlock.set(gtColIdx);
+                    gtColIdx++;
+                }
+                gtColBlocks.add(new ImmutableBitSet(colBlock));
+            }
+        }
+        nMetrics = gtColIdx - nDimensions;
+        assert nMetrics == cuboid.getCube().getMeasures().size();
+    }
+
+    public int getColumnCount() {
+        return nDimensions + nMetrics;
+    }
+
+    public int getDimensionCount() {
+        return nDimensions;
+    }
+
+    public int getMetricsCount() {
+        return nMetrics;
+    }
+
+    public DataType[] getDataTypes() {
+        return (DataType[]) gtDataTypes.toArray(new DataType[gtDataTypes.size()]);
+    }
+
+    public ImmutableBitSet getPrimaryKey() {
+        return gtPrimaryKey;
+    }
+
+    public ImmutableBitSet[] getColumnBlocks() {
+        return (ImmutableBitSet[]) gtColBlocks.toArray(new ImmutableBitSet[gtColBlocks.size()]);
+    }
+
+    public int getIndexOf(TblColRef dimension) {
+        Integer i = dim2gt.get(dimension);
+        return i == null ? -1 : i.intValue();
+    }
+
+    public int getIndexOf(FunctionDesc metric) {
+        List<Integer> list = metrics2gt.get(metric);
+        // normal case
+        if (list.size() == 1) {
+            return list.get(0);
+        }
+        // count distinct & its holistic version
+        else if (list.size() == 2) {
+            assert metric.isCountDistinct();
+            return metric.isHolisticCountDistinct() ? list.get(0) : list.get(1);
+        }
+        // unexpected
+        else
+            return -1;
+    }
+    
+    public List<TblColRef> getCuboidDimensionsInGTOrder() {
+        return cuboid.getColumns();
+    }
+    
+    public Map<Integer, Integer> getDependentMetricsMap() {
+        Map<Integer, Integer> result = Maps.newHashMap();
+        List<MeasureDesc> measures = cuboid.getCube().getMeasures();
+        for (MeasureDesc child : measures) {
+            if (child.getDependentMeasureRef() != null) {
+                boolean ok = false;
+                for (MeasureDesc parent : measures) {
+                    if (parent.getName().equals(child.getDependentMeasureRef())) {
+                        int childIndex = getIndexOf(child.getFunction());
+                        int parentIndex = getIndexOf(parent.getFunction());
+                        result.put(childIndex, parentIndex);
+                        ok = true;
+                        break;
+                    }
+                }
+                if (!ok)
+                    throw new IllegalStateException("Cannot find dependent measure: " + child.getDependentMeasureRef());
+            }
+        }
+        return result.isEmpty() ? Collections.<Integer, Integer>emptyMap() : result;
+    }
+
+    public static MeasureDesc[] getMeasureSequenceOnGridTable(CubeDesc cube) {
+        MeasureDesc[] result = new MeasureDesc[cube.getMeasures().size()];
+        int i = 0;
+        for (HBaseColumnFamilyDesc familyDesc : cube.getHbaseMapping().getColumnFamily()) {
+            for (HBaseColumnDesc hbaseColDesc : familyDesc.getColumns()) {
+                for (MeasureDesc m : hbaseColDesc.getMeasures()) {
+                    result[i++] = m;
+                }
+            }
+        }
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
new file mode 100644
index 0000000..518a8d2
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
@@ -0,0 +1,92 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.GridTable;
+import org.apache.kylin.gridtable.IGTScanner;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An interface alike abstract class. Hold common tunable parameters and nothing more.
+ */
+abstract public class AbstractInMemCubeBuilder {
+
+    private static Logger logger = LoggerFactory.getLogger(AbstractInMemCubeBuilder.class);
+
+    final protected CubeDesc cubeDesc;
+    final protected Map<TblColRef, Dictionary<?>> dictionaryMap;
+    
+    protected int taskThreadCount = 4;
+    protected int reserveMemoryMB = 100;
+
+    public AbstractInMemCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<?>> dictionaryMap) {
+        if(cubeDesc == null)
+            throw new NullPointerException();
+        if (dictionaryMap == null)
+            throw new IllegalArgumentException("dictionary cannot be null");
+        
+        this.cubeDesc = cubeDesc;
+        this.dictionaryMap = dictionaryMap;
+    }
+    
+    public void setConcurrentThreads(int n) {
+        this.taskThreadCount = n;
+    }
+
+    public void setReserveMemoryMB(int mb) {
+        this.reserveMemoryMB = mb;
+    }
+
+    public Runnable buildAsRunnable(final BlockingQueue<List<String>> input, final ICuboidWriter output) {
+        return new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    build(input, output);
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        };
+    }
+    
+    abstract public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException;
+
+    protected void outputCuboid(long cuboidId, GridTable gridTable, ICuboidWriter output) throws IOException {
+        long startTime = System.currentTimeMillis();
+        GTScanRequest req = new GTScanRequest(gridTable.getInfo(), null, null, null);
+        IGTScanner scanner = gridTable.scan(req);
+        for (GTRecord record : scanner) {
+            output.write(cuboidId, record);
+        }
+        scanner.close();
+        logger.info("Cuboid " + cuboidId + " output takes " + (System.currentTimeMillis() - startTime) + "ms");
+    }
+    
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStore.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStore.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStore.java
new file mode 100644
index 0000000..9598a08
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStore.java
@@ -0,0 +1,342 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.StandardOpenOption;
+import java.util.HashSet;
+import java.util.NoSuchElementException;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTRowBlock;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.IGTStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A disk store that allows concurrent read and exclusive write.
+ */
+public class ConcurrentDiskStore implements IGTStore, Closeable {
+
+    private static final Logger logger = LoggerFactory.getLogger(MemDiskStore.class);
+    private static final boolean debug = true;
+
+    private static final int STREAM_BUFFER_SIZE = 8192;
+
+    final private GTInfo info;
+    final private Object lock;
+
+    final private File diskFile;
+    final private boolean delOnClose;
+
+    private Writer activeWriter;
+    private HashSet<Reader> activeReaders = new HashSet<Reader>();
+    private FileChannel writeChannel;
+    private FileChannel readChannel; // sharable across multi-threads
+
+    public ConcurrentDiskStore(GTInfo info) throws IOException {
+        this(info, File.createTempFile("ConcurrentDiskStore", ""), true);
+    }
+
+    public ConcurrentDiskStore(GTInfo info, File diskFile) throws IOException {
+        this(info, diskFile, false);
+    }
+
+    private ConcurrentDiskStore(GTInfo info, File diskFile, boolean delOnClose) throws IOException {
+        this.info = info;
+        this.lock = this;
+        this.diskFile = diskFile;
+        this.delOnClose = delOnClose;
+
+        // in case user forget to call close()
+        if (delOnClose)
+            diskFile.deleteOnExit();
+
+        if (debug)
+            logger.debug(this + " disk file " + diskFile.getAbsolutePath());
+    }
+
+    @Override
+    public GTInfo getInfo() {
+        return info;
+    }
+
+    @Override
+    public IGTStoreWriter rebuild(int shard) throws IOException {
+        return newWriter(0);
+    }
+
+    @Override
+    public IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) throws IOException {
+        throw new IllegalStateException("does not support append yet");
+        //return newWriter(diskFile.length());
+    }
+
+    private IGTStoreWriter newWriter(long startOffset) throws IOException {
+        synchronized (lock) {
+            if (activeWriter != null || !activeReaders.isEmpty())
+                throw new IllegalStateException();
+
+            openWriteChannel(startOffset);
+            activeWriter = new Writer(startOffset);
+            return activeWriter;
+        }
+    }
+
+    private void closeWriter(Writer w) {
+        synchronized (lock) {
+            if (activeWriter != w)
+                throw new IllegalStateException();
+
+            activeWriter = null;
+            closeWriteChannel();
+        }
+    }
+
+    @Override
+    public IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) throws IOException {
+        return newReader();
+    }
+
+    private IGTStoreScanner newReader() throws IOException {
+        synchronized (lock) {
+            if (activeWriter != null)
+                throw new IllegalStateException();
+
+            openReadChannel();
+            Reader r = new Reader(0);
+            activeReaders.add(r);
+            return r;
+        }
+    }
+
+    private void closeReader(Reader r) throws IOException {
+        synchronized (lock) {
+            if (activeReaders.contains(r) == false)
+                throw new IllegalStateException();
+
+            activeReaders.remove(r);
+            if (activeReaders.isEmpty())
+                closeReadChannel();
+        }
+    }
+
+    private class Reader implements IGTStoreScanner {
+        final DataInputStream din;
+        long fileLen;
+        long readOffset;
+
+        GTRowBlock block = GTRowBlock.allocate(info);
+        GTRowBlock next = null;
+
+        Reader(long startOffset) throws IOException {
+            this.fileLen = diskFile.length();
+            this.readOffset = startOffset;
+
+            if (debug)
+                logger.debug(ConcurrentDiskStore.this + " read start @ " + readOffset);
+
+            InputStream in = new InputStream() {
+                byte[] tmp = new byte[1];
+
+                @Override
+                public int read() throws IOException {
+                    int n = read(tmp, 0, 1);
+                    if (n <= 0)
+                        return -1;
+                    else
+                        return (int) tmp[0];
+                }
+
+                @Override
+                public int read(byte[] b, int off, int len) throws IOException {
+                    if (available() <= 0)
+                        return -1;
+
+                    int lenToGo = Math.min(available(), len);
+                    int nRead = 0;
+                    while (lenToGo > 0) {
+                        int n = readChannel.read(ByteBuffer.wrap(b, off, lenToGo), readOffset);
+
+                        lenToGo -= n;
+                        nRead += n;
+                        off += n;
+                        readOffset += n;
+                    }
+                    return nRead;
+                }
+
+                @Override
+                public int available() throws IOException {
+                    return (int) (fileLen - readOffset);
+                }
+            };
+            din = new DataInputStream(new BufferedInputStream(in, STREAM_BUFFER_SIZE));
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (next != null)
+                return true;
+
+            try {
+                if (din.available() > 0) {
+                    block.importFrom(din);
+                    next = block;
+                }
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+
+            return next != null;
+        }
+
+        @Override
+        public GTRowBlock next() {
+            if (next == null) {
+                hasNext();
+                if (next == null)
+                    throw new NoSuchElementException();
+            }
+            GTRowBlock r = next;
+            next = null;
+            return r;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void close() throws IOException {
+            din.close();
+            closeReader(this);
+
+            if (debug)
+                logger.debug(ConcurrentDiskStore.this + " read end @ " + readOffset);
+        }
+    }
+    
+    private class Writer implements IGTStoreWriter {
+        final DataOutputStream dout;
+        long writeOffset;
+
+        Writer(long startOffset) {
+            this.writeOffset = startOffset;
+            
+            if (debug)
+                logger.debug(ConcurrentDiskStore.this + " write start @ " + writeOffset);
+
+            OutputStream out = new OutputStream() {
+                byte[] tmp = new byte[1];
+
+                @Override
+                public void write(int b) throws IOException {
+                    tmp[0] = (byte) b;
+                    write(tmp, 0, 1);
+                }
+
+                @Override
+                public void write(byte[] bytes, int offset, int length) throws IOException {
+                    while (length > 0) {
+                        int n = writeChannel.write(ByteBuffer.wrap(bytes, offset, length), writeOffset);
+                        offset += n;
+                        length -= n;
+                        writeOffset += n;
+                    }
+                }
+            };
+            dout = new DataOutputStream(new BufferedOutputStream(out, STREAM_BUFFER_SIZE));
+        }
+        
+        @Override
+        public void write(GTRowBlock block) throws IOException {
+            block.export(dout);
+        }
+        
+        @Override
+        public void close() throws IOException {
+            dout.close();
+            closeWriter(this);
+
+            if (debug)
+                logger.debug(ConcurrentDiskStore.this + " write end @ " + writeOffset);
+        }
+    }
+
+    private void openWriteChannel(long startOffset) throws IOException {
+        if (startOffset > 0) { // TODO does not support append yet
+            writeChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.CREATE, StandardOpenOption.APPEND, StandardOpenOption.WRITE);
+        } else {
+            diskFile.delete();
+            writeChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+        }
+    }
+
+    private void closeWriteChannel() {
+        IOUtils.closeQuietly(writeChannel);
+        writeChannel = null;
+    }
+
+    private void openReadChannel() throws IOException {
+        if (readChannel == null) {
+            readChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.READ);
+        }
+    }
+
+    private void closeReadChannel() throws IOException {
+        IOUtils.closeQuietly(readChannel);
+        readChannel = null;
+    }
+
+    @Override
+    public void close() throws IOException {
+        synchronized (lock) {
+            if (activeWriter != null || !activeReaders.isEmpty())
+                throw new IllegalStateException();
+
+            if (delOnClose) {
+                diskFile.delete();
+            }
+
+            if (debug)
+                logger.debug(this + " closed");
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "ConcurrentDiskStore@" + (info.getTableName() == null ? this.hashCode() : info.getTableName());
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
new file mode 100644
index 0000000..3d60321
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
@@ -0,0 +1,434 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.common.util.MemoryBudgetController;
+import org.apache.kylin.cube.gridtable.CuboidToGridTableMapping;
+import org.apache.kylin.cube.inmemcubing.InMemCubeBuilder.CuboidResult;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.IGTScanner;
+import org.apache.kylin.metadata.measure.MeasureAggregators;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * When base cuboid does not fit in memory, cut the input into multiple splits and merge the split outputs at last.
+ */
+public class DoggedCubeBuilder extends AbstractInMemCubeBuilder {
+
+    private static Logger logger = LoggerFactory.getLogger(DoggedCubeBuilder.class);
+
+    private int splitRowThreshold = Integer.MAX_VALUE;
+    private int unitRows = 1000;
+
+    public DoggedCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<?>> dictionaryMap) {
+        super(cubeDesc, dictionaryMap);
+    }
+
+    public void setSplitRowThreshold(int rowThreshold) {
+        this.splitRowThreshold = rowThreshold;
+        this.unitRows = Math.min(unitRows, rowThreshold);
+    }
+
+    @Override
+    public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
+        new BuildOnce().build(input, output);
+    }
+
+    private class BuildOnce {
+
+        public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
+            final List<SplitThread> splits = new ArrayList<SplitThread>();
+            final Merger merger = new Merger();
+
+            long start = System.currentTimeMillis();
+            logger.info("Dogged Cube Build start");
+
+            try {
+                SplitThread last = null;
+                boolean eof = false;
+
+                while (!eof) {
+
+                    if (last != null && shouldCutSplit(splits)) {
+                        cutSplit(last);
+                        last = null;
+                    }
+
+                    checkException(splits);
+
+                    if (last == null) {
+                        last = new SplitThread();
+                        splits.add(last);
+                        last.start();
+                        logger.info("Split #" + splits.size() + " kickoff");
+                    }
+
+                    eof = feedSomeInput(input, last, unitRows);
+                }
+
+                for (SplitThread split : splits) {
+                    split.join();
+                }
+                checkException(splits);
+                logger.info("Dogged Cube Build splits complete, took " + (System.currentTimeMillis() - start) + " ms");
+
+                merger.mergeAndOutput(splits, output);
+
+            } catch (Throwable e) {
+                logger.error("Dogged Cube Build error", e);
+                if (e instanceof Error)
+                    throw (Error) e;
+                else if (e instanceof RuntimeException)
+                    throw (RuntimeException) e;
+                else
+                    throw new IOException(e);
+            } finally {
+                closeGirdTables(splits);
+                logger.info("Dogged Cube Build end, totally took " + (System.currentTimeMillis() - start) + " ms");
+                ensureExit(splits);
+                logger.info("Dogged Cube Build return");
+            }
+        }
+
+        private void closeGirdTables(List<SplitThread> splits) {
+            for (SplitThread split : splits) {
+                if (split.buildResult != null) {
+                    for (CuboidResult r : split.buildResult.values()) {
+                        try {
+                            r.table.close();
+                        } catch (Throwable e) {
+                            logger.error("Error closing grid table " + r.table, e);
+                        }
+                    }
+                }
+            }
+        }
+
+        private void ensureExit(List<SplitThread> splits) throws IOException {
+            try {
+                for (int i = 0; i < splits.size(); i++) {
+                    SplitThread split = splits.get(i);
+                    if (split.isAlive()) {
+                        abort(splits);
+                    }
+                }
+            } catch (Throwable e) {
+                logger.error("Dogged Cube Build error", e);
+            }
+        }
+
+        private void checkException(List<SplitThread> splits) throws IOException {
+            for (int i = 0; i < splits.size(); i++) {
+                SplitThread split = splits.get(i);
+                if (split.exception != null)
+                    abort(splits);
+            }
+        }
+
+        private void abort(List<SplitThread> splits) throws IOException {
+            for (SplitThread split : splits) {
+                split.builder.abort();
+            }
+
+            ArrayList<Throwable> errors = new ArrayList<Throwable>();
+            for (SplitThread split : splits) {
+                try {
+                    split.join();
+                } catch (InterruptedException e) {
+                    errors.add(e);
+                }
+                if (split.exception != null)
+                    errors.add(split.exception);
+            }
+
+            if (errors.isEmpty()) {
+                return;
+            } else if (errors.size() == 1) {
+                Throwable t = errors.get(0);
+                if (t instanceof IOException)
+                    throw (IOException) t;
+                else
+                    throw new IOException(t);
+            } else {
+                for (Throwable t : errors)
+                    logger.error("Exception during in-mem cube build", t);
+                throw new IOException(errors.size() + " exceptions during in-mem cube build, cause set to the first, check log for more", errors.get(0));
+            }
+        }
+
+        private boolean feedSomeInput(BlockingQueue<List<String>> input, SplitThread split, int n) {
+            try {
+                int i = 0;
+                while (i < n) {
+                    List<String> record = input.take();
+                    i++;
+
+                    while (split.inputQueue.offer(record, 1, TimeUnit.SECONDS) == false) {
+                        if (split.exception != null)
+                            return true; // got some error
+                    }
+                    split.inputRowCount++;
+
+                    if (record == null || record.isEmpty()) {
+                        return true;
+                    }
+                }
+                return false;
+
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        private void cutSplit(SplitThread last) {
+            try {
+                // signal the end of input
+                while (last.isAlive()) {
+                    if (last.inputQueue.offer(Collections.<String> emptyList())) {
+                        break;
+                    }
+                    Thread.sleep(1000);
+                }
+
+                // wait cuboid build done
+                while (last.isAlive()) {
+                    if (last.builder.isAllCuboidDone()) {
+                        break;
+                    }
+                    Thread.sleep(1000);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        private boolean shouldCutSplit(List<SplitThread> splits) {
+            int systemAvailMB = MemoryBudgetController.getSystemAvailMB();
+            int nSplit = splits.size();
+            long splitRowCount = nSplit == 0 ? 0 : splits.get(nSplit - 1).inputRowCount;
+
+            logger.debug(splitRowCount + " records went into split #" + nSplit + "; " + systemAvailMB + " MB left, " + reserveMemoryMB + " MB threshold");
+
+            return splitRowCount >= splitRowThreshold || systemAvailMB <= reserveMemoryMB;
+        }
+    }
+
+    private class SplitThread extends Thread {
+        final BlockingQueue<List<String>> inputQueue = new ArrayBlockingQueue<List<String>>(16);
+        final InMemCubeBuilder builder;
+
+        ConcurrentNavigableMap<Long, CuboidResult> buildResult;
+        long inputRowCount = 0;
+        RuntimeException exception;
+
+        public SplitThread() {
+            this.builder = new InMemCubeBuilder(cubeDesc, dictionaryMap);
+            this.builder.setConcurrentThreads(taskThreadCount);
+            this.builder.setReserveMemoryMB(reserveMemoryMB);
+        }
+
+        @Override
+        public void run() {
+            try {
+                buildResult = builder.build(inputQueue);
+            } catch (Exception e) {
+                if (e instanceof RuntimeException)
+                    this.exception = (RuntimeException) e;
+                else
+                    this.exception = new RuntimeException(e);
+            }
+        }
+    }
+
+    private class Merger {
+
+        MeasureAggregators reuseAggrs;
+        Object[] reuseMetricsArray;
+        ByteArray reuseMetricsSpace;
+
+        long lastCuboidColumnCount;
+        ImmutableBitSet lastMetricsColumns;
+
+        Merger() {
+            MeasureDesc[] measures = CuboidToGridTableMapping.getMeasureSequenceOnGridTable(cubeDesc);
+            reuseAggrs = new MeasureAggregators(measures);
+            reuseMetricsArray = new Object[measures.length];
+        }
+
+        public void mergeAndOutput(List<SplitThread> splits, ICuboidWriter output) throws IOException {
+            if (splits.size() == 1) {
+                for (CuboidResult cuboidResult : splits.get(0).buildResult.values()) {
+                    outputCuboid(cuboidResult.cuboidId, cuboidResult.table, output);
+                    cuboidResult.table.close();
+                }
+                return;
+            }
+
+            LinkedList<MergeSlot> open = Lists.newLinkedList();
+            for (SplitThread split : splits) {
+                open.add(new MergeSlot(split));
+            }
+
+            PriorityQueue<MergeSlot> heap = new PriorityQueue<MergeSlot>();
+
+            while (true) {
+                // ready records in open slots and add to heap
+                while (!open.isEmpty()) {
+                    MergeSlot slot = open.removeFirst();
+                    if (slot.fetchNext()) {
+                        heap.add(slot);
+                    }
+                }
+
+                // find the smallest on heap
+                MergeSlot smallest = heap.poll();
+                if (smallest == null)
+                    break;
+                open.add(smallest);
+
+                // merge with slots having the same key
+                if (smallest.isSameKey(heap.peek())) {
+                    Object[] metrics = getMetricsValues(smallest.currentRecord);
+                    reuseAggrs.reset();
+                    reuseAggrs.aggregate(metrics);
+                    do {
+                        MergeSlot slot = heap.poll();
+                        open.add(slot);
+                        metrics = getMetricsValues(slot.currentRecord);
+                        reuseAggrs.aggregate(metrics);
+                    } while (smallest.isSameKey(heap.peek()));
+
+                    reuseAggrs.collectStates(metrics);
+                    setMetricsValues(smallest.currentRecord, metrics);
+                }
+
+                output.write(smallest.currentCuboidId, smallest.currentRecord);
+            }
+        }
+
+        private void setMetricsValues(GTRecord record, Object[] metricsValues) {
+            ImmutableBitSet metrics = getMetricsColumns(record);
+
+            if (reuseMetricsSpace == null) {
+                reuseMetricsSpace = new ByteArray(record.getInfo().getMaxColumnLength(metrics));
+            }
+
+            record.setValues(metrics, reuseMetricsSpace, metricsValues);
+        }
+
+        private Object[] getMetricsValues(GTRecord record) {
+            ImmutableBitSet metrics = getMetricsColumns(record);
+            return record.getValues(metrics, reuseMetricsArray);
+        }
+
+        private ImmutableBitSet getMetricsColumns(GTRecord record) {
+            // metrics columns always come after dimension columns
+            if (lastCuboidColumnCount == record.getInfo().getColumnCount())
+                return lastMetricsColumns;
+
+            int to = record.getInfo().getColumnCount();
+            int from = to - reuseMetricsArray.length;
+            lastCuboidColumnCount = record.getInfo().getColumnCount();
+            lastMetricsColumns = new ImmutableBitSet(from, to);
+            return lastMetricsColumns;
+        }
+    }
+
+    private static class MergeSlot implements Comparable<MergeSlot> {
+
+        final Iterator<CuboidResult> cuboidIterator;
+        IGTScanner scanner;
+        Iterator<GTRecord> recordIterator;
+
+        long currentCuboidId;
+        GTRecord currentRecord;
+
+        public MergeSlot(SplitThread split) {
+            cuboidIterator = split.buildResult.values().iterator();
+        }
+
+        public boolean fetchNext() throws IOException {
+            if (recordIterator == null) {
+                if (cuboidIterator.hasNext()) {
+                    CuboidResult cuboid = cuboidIterator.next();
+                    currentCuboidId = cuboid.cuboidId;
+                    scanner = cuboid.table.scan(new GTScanRequest(cuboid.table.getInfo(), null, null, null));
+                    recordIterator = scanner.iterator();
+                } else {
+                    return false;
+                }
+            }
+
+            if (recordIterator.hasNext()) {
+                currentRecord = recordIterator.next();
+                return true;
+            } else {
+                scanner.close();
+                recordIterator = null;
+                return fetchNext();
+            }
+        }
+
+        @Override
+        public int compareTo(MergeSlot o) {
+            long cuboidComp = this.currentCuboidId - o.currentCuboidId;
+            if (cuboidComp != 0)
+                return cuboidComp < 0 ? -1 : 1;
+
+            // note GTRecord.equals() don't work because the two GTRecord comes from different GridTable
+            ImmutableBitSet pk = this.currentRecord.getInfo().getPrimaryKey();
+            for (int i = 0; i < pk.trueBitCount(); i++) {
+                int c = pk.trueBitAt(i);
+                int comp = this.currentRecord.get(c).compareTo(o.currentRecord.get(c));
+                if (comp != 0)
+                    return comp;
+            }
+            return 0;
+        }
+
+        public boolean isSameKey(MergeSlot o) {
+            if (o == null)
+                return false;
+            else
+                return this.compareTo(o) == 0;
+        }
+
+    };
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ICuboidWriter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ICuboidWriter.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ICuboidWriter.java
new file mode 100644
index 0000000..1afb665
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ICuboidWriter.java
@@ -0,0 +1,28 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import org.apache.kylin.gridtable.GTRecord;
+
+import java.io.IOException;
+
+/**
+ */
+public interface ICuboidWriter {
+    void write(long cuboidId, GTRecord record) throws IOException;
+}


[14/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java b/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java
index b00e10e..f8d6e1a 100644
--- a/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java
+++ b/job/src/main/java/org/apache/kylin/job/CubeMetadataUpgrade.java
@@ -45,23 +45,20 @@ import org.apache.kylin.cube.model.v1.CubeSegment;
 import org.apache.kylin.cube.model.v1.CubeSegmentStatusEnum;
 import org.apache.kylin.cube.model.v1.CubeStatusEnum;
 import org.apache.kylin.engine.mr.CubingJob;
-import org.apache.kylin.job.common.HadoopShellExecutable;
-import org.apache.kylin.job.common.MapReduceExecutable;
+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.CreateDictionaryJob;
+import org.apache.kylin.engine.mr.steps.FactDistinctColumnsJob;
+import org.apache.kylin.engine.mr.steps.MergeCuboidJob;
+import org.apache.kylin.engine.mr.steps.NDCuboidJob;
+import org.apache.kylin.engine.mr.steps.RangeKeyDistributionJob;
 import org.apache.kylin.job.common.ShellExecutable;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.constant.JobStatusEnum;
 import org.apache.kylin.job.constant.JobStepStatusEnum;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
-import org.apache.kylin.job.hadoop.cube.BaseCuboidJob;
-import org.apache.kylin.job.hadoop.cube.CubeHFileJob;
-import org.apache.kylin.job.hadoop.cube.FactDistinctColumnsJob;
-import org.apache.kylin.job.hadoop.cube.MergeCuboidJob;
-import org.apache.kylin.job.hadoop.cube.NDCuboidJob;
-import org.apache.kylin.job.hadoop.cube.RangeKeyDistributionJob;
-import org.apache.kylin.job.hadoop.dict.CreateDictionaryJob;
-import org.apache.kylin.job.hadoop.hbase.BulkLoadJob;
-import org.apache.kylin.job.hadoop.hbase.CreateHTableJob;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
@@ -72,6 +69,9 @@ import org.apache.kylin.metadata.project.ProjectManager;
 import org.apache.kylin.metadata.project.RealizationEntry;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.metadata.realization.RealizationType;
+import org.apache.kylin.storage.hbase.steps.BulkLoadJob;
+import org.apache.kylin.storage.hbase.steps.CreateHTableJob;
+import org.apache.kylin.storage.hbase.steps.CubeHFileJob;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/JobInstance.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/JobInstance.java b/job/src/main/java/org/apache/kylin/job/JobInstance.java
deleted file mode 100644
index e7f5772..0000000
--- a/job/src/main/java/org/apache/kylin/job/JobInstance.java
+++ /dev/null
@@ -1,504 +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;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
-import com.fasterxml.jackson.annotation.JsonBackReference;
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-import com.fasterxml.jackson.annotation.JsonManagedReference;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.Lists;
-import org.apache.kylin.common.persistence.RootPersistentEntity;
-import org.apache.kylin.cube.model.CubeBuildTypeEnum;
-import org.apache.kylin.job.constant.JobStatusEnum;
-import org.apache.kylin.job.constant.JobStepCmdTypeEnum;
-import org.apache.kylin.job.constant.JobStepStatusEnum;
-import org.apache.kylin.job.engine.JobEngineConfig;
-
-@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class JobInstance extends RootPersistentEntity implements Comparable<JobInstance> {
-
-    public static final String JOB_WORKING_DIR_PREFIX = "kylin-";
-
-    public static final String YARN_APP_ID = "yarn_application_id";
-    public static final String YARN_APP_URL = "yarn_application_tracking_url";
-    public static final String MR_JOB_ID = "mr_job_id";
-    public static final String HDFS_BYTES_WRITTEN = "hdfs_bytes_written";
-    public static final String SOURCE_RECORDS_COUNT = "source_records_count";
-    public static final String SOURCE_RECORDS_SIZE = "source_records_size";
-
-    public static String getStepIdentity(JobInstance jobInstance, JobStep jobStep) {
-        return jobInstance.getRelatedCube() + "." + jobInstance.getUuid() + "." + jobStep.getSequenceID();
-    }
-
-    public static String getJobIdentity(JobInstance jobInstance) {
-        return jobInstance.getRelatedCube() + "." + jobInstance.getUuid();
-    }
-
-    public static String getJobWorkingDir(JobInstance jobInstance, JobEngineConfig engineConfig) {
-        return getJobWorkingDir(jobInstance.getUuid(), engineConfig.getHdfsWorkingDirectory());
-    }
-
-    public static String getJobWorkingDir(String jobUuid, String hdfsWorkdingDir) {
-        if (jobUuid == null || jobUuid.equals("")) {
-            throw new IllegalArgumentException("jobUuid can't be null or empty");
-        }
-        return hdfsWorkdingDir + "/" + JOB_WORKING_DIR_PREFIX + jobUuid;
-    }
-
-    @JsonProperty("name")
-    private String name;
-
-    @JsonProperty("type")
-    private CubeBuildTypeEnum type; // java implementation
-    @JsonProperty("duration")
-    private long duration;
-    @JsonProperty("related_cube")
-    private String relatedCube;
-    @JsonProperty("related_segment")
-    private String relatedSegment;
-    @JsonProperty("exec_start_time")
-    private long execStartTime;
-    @JsonProperty("exec_end_time")
-    private long execEndTime;
-    @JsonProperty("mr_waiting")
-    private long mrWaiting = 0;
-    @JsonManagedReference
-    @JsonProperty("steps")
-    private List<JobStep> steps;
-    @JsonProperty("submitter")
-    private String submitter;
-    @JsonProperty("job_status")
-    private JobStatusEnum status;
-
-    public JobStep getRunningStep() {
-        for (JobStep step : this.getSteps()) {
-            if (step.getStatus().equals(JobStepStatusEnum.RUNNING) || step.getStatus().equals(JobStepStatusEnum.WAITING)) {
-                return step;
-            }
-        }
-
-        return null;
-    }
-
-    @JsonProperty("progress")
-    public double getProgress() {
-        int completedStepCount = 0;
-        for (JobStep step : this.getSteps()) {
-            if (step.getStatus().equals(JobStepStatusEnum.FINISHED)) {
-                completedStepCount++;
-            }
-        }
-
-        return 100.0 * completedStepCount / steps.size();
-    }
-
-    public JobStatusEnum getStatus() {
-        return this.status;
-    }
-
-    public void setStatus(JobStatusEnum status) {
-        this.status = status;
-    }
-
-//    @JsonProperty("job_status")
-//    public JobStatusEnum getStatus() {
-//
-//        // JobStatusEnum finalJobStatus;
-//        int compositResult = 0;
-//
-//        // if steps status are all NEW, then job status is NEW
-//        // if steps status are all FINISHED, then job status is FINISHED
-//        // if steps status are all PENDING, then job status is PENDING
-//        // if steps status are FINISHED and PENDING, the job status is PENDING
-//        // if one of steps status is RUNNING, then job status is RUNNING
-//        // if one of steps status is ERROR, then job status is ERROR
-//        // if one of steps status is KILLED, then job status is KILLED
-//        // default status is RUNNING
-//
-//        System.out.println(this.getName());
-//
-//        for (JobStep step : this.getSteps()) {
-//            //System.out.println("step: " + step.getSequenceID() + "'s status:" + step.getStatus());
-//            compositResult = compositResult | step.getStatus().getCode();
-//        }
-//
-//        System.out.println();
-//
-//        if (compositResult == JobStatusEnum.FINISHED.getCode()) {
-//            return JobStatusEnum.FINISHED;
-//        } else if (compositResult == JobStatusEnum.NEW.getCode()) {
-//            return JobStatusEnum.NEW;
-//        } else if (compositResult == JobStatusEnum.PENDING.getCode()) {
-//            return JobStatusEnum.PENDING;
-//        } else if (compositResult == (JobStatusEnum.FINISHED.getCode() | JobStatusEnum.PENDING.getCode())) {
-//            return JobStatusEnum.PENDING;
-//        } else if ((compositResult & JobStatusEnum.ERROR.getCode()) == JobStatusEnum.ERROR.getCode()) {
-//            return JobStatusEnum.ERROR;
-//        } else if ((compositResult & JobStatusEnum.DISCARDED.getCode()) == JobStatusEnum.DISCARDED.getCode()) {
-//            return JobStatusEnum.DISCARDED;
-//        } else if ((compositResult & JobStatusEnum.RUNNING.getCode()) == JobStatusEnum.RUNNING.getCode()) {
-//            return JobStatusEnum.RUNNING;
-//        }
-//
-//        return JobStatusEnum.RUNNING;
-//    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public CubeBuildTypeEnum getType() {
-        return type;
-    }
-
-    public void setType(CubeBuildTypeEnum type) {
-        this.type = type;
-    }
-
-    public long getDuration() {
-        return duration;
-    }
-
-    public void setDuration(long duration) {
-        this.duration = duration;
-    }
-
-    public String getRelatedCube() {
-        return relatedCube;
-    }
-
-    public void setRelatedCube(String relatedCube) {
-        this.relatedCube = relatedCube;
-    }
-
-    public String getRelatedSegment() {
-        return relatedSegment;
-    }
-
-    public void setRelatedSegment(String relatedSegment) {
-        this.relatedSegment = relatedSegment;
-    }
-
-    /**
-     * @return the execStartTime
-     */
-    public long getExecStartTime() {
-        return execStartTime;
-    }
-
-    /**
-     * @param execStartTime the execStartTime to set
-     */
-    public void setExecStartTime(long execStartTime) {
-        this.execStartTime = execStartTime;
-    }
-
-    /**
-     * @return the execEndTime
-     */
-    public long getExecEndTime() {
-        return execEndTime;
-    }
-
-    /**
-     * @param execEndTime the execEndTime to set
-     */
-    public void setExecEndTime(long execEndTime) {
-        this.execEndTime = execEndTime;
-    }
-
-    public long getMrWaiting() {
-        return this.mrWaiting;
-    }
-
-    public void setMrWaiting(long mrWaiting) {
-        this.mrWaiting = mrWaiting;
-    }
-
-    public List<JobStep> getSteps() {
-        if (steps == null) {
-            steps = Lists.newArrayList();
-        }
-        return steps;
-    }
-
-    public void clearSteps() {
-        getSteps().clear();
-    }
-
-    public void addSteps(Collection<JobStep> steps) {
-        this.getSteps().addAll(steps);
-    }
-
-    public void addStep(JobStep step) {
-        getSteps().add(step);
-    }
-
-    public void addStep(int index, JobStep step) {
-        getSteps().add(index, step);
-    }
-
-    public JobStep findStep(String stepName) {
-        for (JobStep step : getSteps()) {
-            if (stepName.equals(step.getName())) {
-                return step;
-            }
-        }
-        return null;
-    }
-
-        
-    public String getSubmitter() {
-        return submitter;
-    }
-
-    public void setSubmitter(String submitter) {
-        this.submitter = submitter;
-    }
-
-
-
-
-    @JsonIgnoreProperties(ignoreUnknown = true)
-    public static class JobStep implements Comparable<JobStep> {
-
-        @JsonBackReference
-        private JobInstance jobInstance;
-        
-        @JsonProperty("id")
-        private String id;
-
-        @JsonProperty("name")
-        private String name;
-
-        @JsonProperty("sequence_id")
-        private int sequenceID;
-
-        @JsonProperty("exec_cmd")
-        private String execCmd;
-
-        @JsonProperty("interrupt_cmd")
-        private String InterruptCmd;
-
-        @JsonProperty("exec_start_time")
-        private long execStartTime;
-        @JsonProperty("exec_end_time")
-        private long execEndTime;
-        @JsonProperty("exec_wait_time")
-        private long execWaitTime;
-
-        @JsonProperty("step_status")
-        private JobStepStatusEnum status;
-
-        @JsonProperty("cmd_type")
-        private JobStepCmdTypeEnum cmdType = JobStepCmdTypeEnum.SHELL_CMD_HADOOP;
-
-        @JsonProperty("info")
-        private ConcurrentHashMap<String, String> info = new ConcurrentHashMap<String, String>();
-
-        @JsonProperty("run_async")
-        private boolean runAsync = false;
-
-        private ConcurrentHashMap<String, String> getInfo() {
-            return info;
-        }
-
-        public void putInfo(String key, String value) {
-            getInfo().put(key, value);
-        }
-
-        public String getInfo(String key) {
-            return getInfo().get(key);
-        }
-
-        public void clearInfo() {
-            getInfo().clear();
-        }
-
-        public String getName() {
-            return name;
-        }
-
-        public void setName(String name) {
-            this.name = name;
-        }
-
-        public int getSequenceID() {
-            return sequenceID;
-        }
-
-        public void setSequenceID(int sequenceID) {
-            this.sequenceID = sequenceID;
-        }
-
-        public String getExecCmd() {
-            return execCmd;
-        }
-
-        public void setExecCmd(String execCmd) {
-            this.execCmd = execCmd;
-        }
-
-        public JobStepStatusEnum getStatus() {
-            return status;
-        }
-
-        public void setStatus(JobStepStatusEnum status) {
-            this.status = status;
-        }
-        
-        
-
-        public String getId() {
-            return id;
-        }
-
-        public void setId(String id) {
-            this.id = id;
-        }
-
-        /**
-         * @return the execStartTime
-         */
-        public long getExecStartTime() {
-            return execStartTime;
-        }
-
-        /**
-         * @param execStartTime the execStartTime to set
-         */
-        public void setExecStartTime(long execStartTime) {
-            this.execStartTime = execStartTime;
-        }
-
-        /**
-         * @return the execEndTime
-         */
-        public long getExecEndTime() {
-            return execEndTime;
-        }
-
-        /**
-         * @param execEndTime the execEndTime to set
-         */
-        public void setExecEndTime(long execEndTime) {
-            this.execEndTime = execEndTime;
-        }
-
-        public long getExecWaitTime() {
-            return execWaitTime;
-        }
-
-        public void setExecWaitTime(long execWaitTime) {
-            this.execWaitTime = execWaitTime;
-        }
-
-        public String getInterruptCmd() {
-            return InterruptCmd;
-        }
-
-        public void setInterruptCmd(String interruptCmd) {
-            InterruptCmd = interruptCmd;
-        }
-
-        public JobStepCmdTypeEnum getCmdType() {
-            return cmdType;
-        }
-
-        public void setCmdType(JobStepCmdTypeEnum cmdType) {
-            this.cmdType = cmdType;
-        }
-
-        /**
-         * @return the runAsync
-         */
-        public boolean isRunAsync() {
-            return runAsync;
-        }
-
-        /**
-         * @param runAsync the runAsync to set
-         */
-        public void setRunAsync(boolean runAsync) {
-            this.runAsync = runAsync;
-        }
-
-        /**
-         * @return the jobInstance
-         */
-        public JobInstance getJobInstance() {
-            return jobInstance;
-        }
-
-        @Override
-        public int hashCode() {
-            final int prime = 31;
-            int result = 1;
-            result = prime * result + ((name == null) ? 0 : name.hashCode());
-            result = prime * result + sequenceID;
-            return result;
-        }
-
-        @Override
-        public boolean equals(Object obj) {
-            if (this == obj)
-                return true;
-            if (obj == null)
-                return false;
-            if (getClass() != obj.getClass())
-                return false;
-            JobStep other = (JobStep) obj;
-            if (name == null) {
-                if (other.name != null)
-                    return false;
-            } else if (!name.equals(other.name))
-                return false;
-            if (sequenceID != other.sequenceID)
-                return false;
-            return true;
-        }
-
-        @Override
-        public int compareTo(JobStep o) {
-            if (this.sequenceID < o.sequenceID) {
-                return -1;
-            } else if (this.sequenceID > o.sequenceID) {
-                return 1;
-            } else {
-                return 0;
-            }
-        }
-    }
-
-    @Override
-    public int compareTo(JobInstance o) {
-        return o.lastModified<this.lastModified?-1:o.lastModified>this.lastModified?1:0;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java b/job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
deleted file mode 100644
index eba6bd4..0000000
--- a/job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
+++ /dev/null
@@ -1,236 +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;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.hadoop.hive.SqlHiveDataTypeMapping;
-import org.apache.kylin.metadata.model.DataModelDesc;
-import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
-import org.apache.kylin.metadata.model.IntermediateColumnDesc;
-import org.apache.kylin.metadata.model.JoinDesc;
-import org.apache.kylin.metadata.model.LookupDesc;
-import org.apache.kylin.metadata.model.PartitionDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.w3c.dom.Document;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-
-public class JoinedFlatTable {
-
-    public static final String FACT_TABLE_ALIAS = "FACT_TABLE";
-
-    public static final String LOOKUP_TABLE_ALAIS_PREFIX = "LOOKUP_";
-
-    public static String getTableDir(IJoinedFlatTableDesc intermediateTableDesc, String storageDfsDir) {
-        return storageDfsDir + "/" + intermediateTableDesc.getTableName();
-    }
-
-    public static String generateCreateTableStatement(IJoinedFlatTableDesc intermediateTableDesc, String storageDfsDir) {
-        StringBuilder ddl = new StringBuilder();
-
-        ddl.append("CREATE EXTERNAL TABLE IF NOT EXISTS " + intermediateTableDesc.getTableName() + "\n");
-
-        ddl.append("(" + "\n");
-        for (int i = 0; i < intermediateTableDesc.getColumnList().size(); i++) {
-            IntermediateColumnDesc col = intermediateTableDesc.getColumnList().get(i);
-            if (i > 0) {
-                ddl.append(",");
-            }
-            ddl.append(colName(col.getCanonicalName()) + " " + SqlHiveDataTypeMapping.getHiveDataType(col.getDataType()) + "\n");
-        }
-        ddl.append(")" + "\n");
-
-        ddl.append("ROW FORMAT DELIMITED FIELDS TERMINATED BY '\\177'" + "\n");
-        ddl.append("STORED AS SEQUENCEFILE" + "\n");
-        ddl.append("LOCATION '" + storageDfsDir + "/" + intermediateTableDesc.getTableName() + "';").append("\n");
-        // ddl.append("TBLPROPERTIES ('serialization.null.format'='\\\\N')" +
-        // ";\n");
-        return ddl.toString();
-    }
-
-    public static String generateDropTableStatement(IJoinedFlatTableDesc intermediateTableDesc) {
-        StringBuilder ddl = new StringBuilder();
-        ddl.append("DROP TABLE IF EXISTS " + intermediateTableDesc.getTableName() + ";").append("\n");
-        return ddl.toString();
-    }
-
-    public static String generateInsertDataStatement(IJoinedFlatTableDesc intermediateTableDesc, JobEngineConfig engineConfig) throws IOException {
-        StringBuilder sql = new StringBuilder();
-
-        File hadoopPropertiesFile = new File(engineConfig.getHiveConfFilePath());
-
-        if (hadoopPropertiesFile.exists()) {
-            DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
-            DocumentBuilder builder;
-            Document doc;
-            try {
-                builder = factory.newDocumentBuilder();
-                doc = builder.parse(hadoopPropertiesFile);
-                NodeList nl = doc.getElementsByTagName("property");
-                for (int i = 0; i < nl.getLength(); i++) {
-                    String name = doc.getElementsByTagName("name").item(i).getFirstChild().getNodeValue();
-                    String value = doc.getElementsByTagName("value").item(i).getFirstChild().getNodeValue();
-                    if (name.equals("tmpjars") == false) {
-                        sql.append("SET " + name + "=" + value + ";").append("\n");
-                    }
-                }
-
-            } catch (ParserConfigurationException e) {
-                throw new IOException(e);
-            } catch (SAXException e) {
-                throw new IOException(e);
-            }
-        }
-
-        sql.append("INSERT OVERWRITE TABLE " + intermediateTableDesc.getTableName() + " " + generateSelectDataStatement(intermediateTableDesc) + ";").append("\n");
-
-        return sql.toString();
-    }
-
-    public static String generateSelectDataStatement(IJoinedFlatTableDesc intermediateTableDesc) {
-        StringBuilder sql = new StringBuilder();
-        sql.append("SELECT" + "\n");
-        String tableAlias;
-        Map<String, String> tableAliasMap = buildTableAliasMap(intermediateTableDesc.getDataModel());
-        for (int i = 0; i < intermediateTableDesc.getColumnList().size(); i++) {
-            IntermediateColumnDesc col = intermediateTableDesc.getColumnList().get(i);
-            if (i > 0) {
-                sql.append(",");
-            }
-            tableAlias = tableAliasMap.get(col.getTableName());
-            sql.append(tableAlias + "." + col.getColumnName() + "\n");
-        }
-        appendJoinStatement(intermediateTableDesc, sql, tableAliasMap);
-        appendWhereStatement(intermediateTableDesc, sql, tableAliasMap);
-        return sql.toString();
-    }
-
-    private static Map<String, String> buildTableAliasMap(DataModelDesc dataModelDesc) {
-        Map<String, String> tableAliasMap = new HashMap<String, String>();
-
-        tableAliasMap.put(dataModelDesc.getFactTable().toUpperCase(), FACT_TABLE_ALIAS);
-
-        int i = 1;
-        for (LookupDesc lookupDesc: dataModelDesc.getLookups()) {
-            JoinDesc join = lookupDesc.getJoin();
-            if (join != null) {
-                tableAliasMap.put(lookupDesc.getTable().toUpperCase(), LOOKUP_TABLE_ALAIS_PREFIX + i);
-                i++;
-            }
-
-        }
-        return tableAliasMap;
-    }
-
-    private static void appendJoinStatement(IJoinedFlatTableDesc intermediateTableDesc, StringBuilder sql, Map<String, String> tableAliasMap) {
-        Set<String> dimTableCache = new HashSet<String>();
-
-        DataModelDesc dataModelDesc = intermediateTableDesc.getDataModel();
-        String factTableName = dataModelDesc.getFactTable();
-        String factTableAlias = tableAliasMap.get(factTableName);
-        sql.append("FROM " + factTableName + " as " + factTableAlias + " \n");
-
-        for (LookupDesc lookupDesc : dataModelDesc.getLookups()) {
-            JoinDesc join = lookupDesc.getJoin();
-            if (join != null && join.getType().equals("") == false) {
-                String joinType = join.getType().toUpperCase();
-                String dimTableName = lookupDesc.getTable();
-                if (!dimTableCache.contains(dimTableName)) {
-                    TblColRef[] pk = join.getPrimaryKeyColumns();
-                    TblColRef[] fk = join.getForeignKeyColumns();
-                    if (pk.length != fk.length) {
-                        throw new RuntimeException("Invalid join condition of lookup table:" + lookupDesc);
-                    }
-                    sql.append(joinType + " JOIN " + dimTableName + " as " + tableAliasMap.get(dimTableName) + "\n");
-                    sql.append("ON ");
-                    for (int i = 0; i < pk.length; i++) {
-                        if (i > 0) {
-                            sql.append(" AND ");
-                        }
-                        sql.append(factTableAlias + "." + fk[i].getName() + " = " + tableAliasMap.get(dimTableName) + "." + pk[i].getName());
-                    }
-                    sql.append("\n");
-
-                    dimTableCache.add(dimTableName);
-                }
-            }
-        }
-    }
-
-    private static void appendWhereStatement(IJoinedFlatTableDesc intermediateTableDesc, StringBuilder sql, Map<String, String> tableAliasMap) {
-        if (!(intermediateTableDesc instanceof CubeJoinedFlatTableDesc)) {
-            return;//TODO: for now only cube segments support filter and partition
-        }
-        CubeJoinedFlatTableDesc desc = (CubeJoinedFlatTableDesc) intermediateTableDesc;
-
-        boolean hasCondition = false;
-        StringBuilder whereBuilder = new StringBuilder();
-        whereBuilder.append("WHERE");
-
-        CubeDesc cubeDesc = desc.getCubeDesc();
-        DataModelDesc model = cubeDesc.getModel();
-        
-        if (model.getFilterCondition() != null && model.getFilterCondition().equals("") == false) {
-            whereBuilder.append(" (").append(model.getFilterCondition()).append(") ");
-            hasCondition = true;
-        }
-
-        CubeSegment cubeSegment = desc.getCubeSegment();
-
-        if (null != cubeSegment) {
-            PartitionDesc partDesc = model.getPartitionDesc();
-            long dateStart = cubeSegment.getDateRangeStart();
-            long dateEnd = cubeSegment.getDateRangeEnd();
-
-            if (!(dateStart == 0 && dateEnd == Long.MAX_VALUE)) {
-                whereBuilder.append(hasCondition ? " AND (" : " (");
-                whereBuilder.append(partDesc.getPartitionConditionBuilder().buildDateRangeCondition(partDesc, dateStart, dateEnd, tableAliasMap));
-                whereBuilder.append(")\n");
-                hasCondition = true;
-            }
-        }
-
-        if (hasCondition) {
-            sql.append(whereBuilder.toString());
-        }
-    }
-
-    private static String colName(String canonicalColName) {
-        return canonicalColName.replace(".", "_");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/Scheduler.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/Scheduler.java b/job/src/main/java/org/apache/kylin/job/Scheduler.java
deleted file mode 100644
index 2ed2fc2..0000000
--- a/job/src/main/java/org/apache/kylin/job/Scheduler.java
+++ /dev/null
@@ -1,36 +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;
-
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.exception.SchedulerException;
-import org.apache.kylin.job.execution.Executable;
-import org.apache.kylin.job.lock.JobLock;
-
-/**
- */
-public interface Scheduler<T extends Executable> {
-
-    void init(JobEngineConfig jobEngineConfig, JobLock jobLock) throws SchedulerException;
-
-    void shutdown() throws SchedulerException;
-
-    boolean stop(T executable) throws SchedulerException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/cmd/BaseCommandOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cmd/BaseCommandOutput.java b/job/src/main/java/org/apache/kylin/job/cmd/BaseCommandOutput.java
deleted file mode 100644
index 29b5324..0000000
--- a/job/src/main/java/org/apache/kylin/job/cmd/BaseCommandOutput.java
+++ /dev/null
@@ -1,29 +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.cmd;
-
-/**
- */
-public abstract class BaseCommandOutput implements ICommandOutput {
-
-    @Override
-    public void log(String message) {
-        this.appendOutput(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/cmd/ICommandOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cmd/ICommandOutput.java b/job/src/main/java/org/apache/kylin/job/cmd/ICommandOutput.java
deleted file mode 100644
index 6cab6a3..0000000
--- a/job/src/main/java/org/apache/kylin/job/cmd/ICommandOutput.java
+++ /dev/null
@@ -1,44 +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.cmd;
-
-import org.apache.kylin.common.util.Logger;
-import org.apache.kylin.job.constant.JobStepStatusEnum;
-
-/**
- * @author xjiang
- * 
- */
-public interface ICommandOutput extends Logger {
-
-    public void setStatus(JobStepStatusEnum status);
-
-    public JobStepStatusEnum getStatus();
-
-    public void appendOutput(String message);
-
-    public String getOutput();
-
-    public void setExitCode(int exitCode);
-
-    public int getExitCode();
-
-    public void reset();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/cmd/IJobCommand.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cmd/IJobCommand.java b/job/src/main/java/org/apache/kylin/job/cmd/IJobCommand.java
deleted file mode 100644
index 5a47173..0000000
--- a/job/src/main/java/org/apache/kylin/job/cmd/IJobCommand.java
+++ /dev/null
@@ -1,32 +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.cmd;
-
-import org.apache.kylin.job.exception.JobException;
-
-/**
- * @author xjiang
- * 
- */
-public interface IJobCommand {
-
-    public ICommandOutput execute() throws JobException;
-
-    public void cancel() throws JobException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java b/job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
deleted file mode 100644
index 6a718fc..0000000
--- a/job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.cmd;
-
-import org.apache.kylin.common.util.CliCommandExecutor;
-import org.apache.kylin.job.constant.JobStepStatusEnum;
-import org.apache.kylin.job.exception.JobException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.*;
-
-/**
- * @author xjiang
- * 
- */
-public class ShellCmd implements IJobCommand {
-
-    private static Logger log = LoggerFactory.getLogger(ShellCmd.class);
-
-    private final String executeCommand;
-    private final ICommandOutput output;
-    private final boolean isAsync;
-    private final CliCommandExecutor cliCommandExecutor;
-
-    private FutureTask<Integer> future;
-
-    private ShellCmd(String executeCmd, ICommandOutput out, String host, int port, String user, String password, boolean async) {
-        this.executeCommand = executeCmd;
-        this.output = out;
-        this.cliCommandExecutor = new CliCommandExecutor();
-        this.cliCommandExecutor.setRunAtRemote(host, port, user, password);
-        this.isAsync = async;
-    }
-
-    public ShellCmd(String executeCmd, String host, int port, String user, String password, boolean async) {
-        this(executeCmd, new ShellCmdOutput(), host, port, user, password, async);
-    }
-
-    @Override
-    public ICommandOutput execute() throws JobException {
-
-        final ExecutorService executor = Executors.newSingleThreadExecutor();
-        future = new FutureTask<Integer>(new Callable<Integer>() {
-            public Integer call() throws JobException, IOException {
-                executor.shutdown();
-                return executeCommand(executeCommand);
-            }
-        });
-        executor.execute(future);
-
-        int exitCode = -1;
-        if (!isAsync) {
-            try {
-                exitCode = future.get();
-                log.info("finish executing");
-            } catch (CancellationException e) {
-                log.debug("Command is cancelled");
-                exitCode = -2;
-            } catch (Exception e) {
-                throw new JobException("Error when execute job " + executeCommand, e);
-            } finally {
-                if (exitCode == 0) {
-                    output.setStatus(JobStepStatusEnum.FINISHED);
-                } else if (exitCode == -2) {
-                    output.setStatus(JobStepStatusEnum.DISCARDED);
-                } else {
-                    output.setStatus(JobStepStatusEnum.ERROR);
-                }
-                output.setExitCode(exitCode);
-            }
-        }
-        return output;
-    }
-
-    protected int executeCommand(String command) throws JobException, IOException {
-        output.reset();
-        output.setStatus(JobStepStatusEnum.RUNNING);
-        return cliCommandExecutor.execute(command, output).getFirst();
-    }
-
-    @Override
-    public void cancel() throws JobException {
-        future.cancel(true);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java b/job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
deleted file mode 100644
index ebcad47..0000000
--- a/job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.cmd;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.kylin.job.constant.JobStepStatusEnum;
-
-/**
- * @author xjiang
- * 
- */
-public class ShellCmdOutput extends BaseCommandOutput implements ICommandOutput {
-
-    protected static final Logger log = LoggerFactory.getLogger(ShellCmdOutput.class);
-
-    protected StringBuilder output;
-    protected int exitCode;
-    protected JobStepStatusEnum status;
-
-    public ShellCmdOutput() {
-        init();
-    }
-
-    private void init() {
-        output = new StringBuilder();
-        exitCode = -1;
-        status = JobStepStatusEnum.NEW;
-    }
-
-    @Override
-    public JobStepStatusEnum getStatus() {
-        return status;
-    }
-
-    @Override
-    public void setStatus(JobStepStatusEnum s) {
-        this.status = s;
-    }
-
-    @Override
-    public String getOutput() {
-        return output.toString();
-    }
-
-    @Override
-    public void appendOutput(String message) {
-        output.append(message).append(System.getProperty("line.separator"));
-        log.debug(message);
-    }
-
-    @Override
-    public int getExitCode() {
-        return exitCode;
-    }
-
-    @Override
-    public void setExitCode(int code) {
-        exitCode = code;
-    }
-
-    @Override
-    public void reset() {
-        init();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/common/HadoopCmdOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/common/HadoopCmdOutput.java b/job/src/main/java/org/apache/kylin/job/common/HadoopCmdOutput.java
deleted file mode 100644
index 288fd31..0000000
--- a/job/src/main/java/org/apache/kylin/job/common/HadoopCmdOutput.java
+++ /dev/null
@@ -1,105 +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.common;
-
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.TaskCounter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * @author xduo
- * 
- */
-public class HadoopCmdOutput {
-
-    protected static final Logger log = LoggerFactory.getLogger(HadoopCmdOutput.class);
-
-    private final StringBuilder output;
-    private final Job job;
-
-    public HadoopCmdOutput(Job job, StringBuilder output) {
-        super();
-        this.job = job;
-        this.output = output;
-    }
-
-    public String getMrJobId() {
-        return getInfo().get(ExecutableConstants.MR_JOB_ID);
-    }
-
-    public Map<String, String> getInfo() {
-        if (job != null) {
-            Map<String, String> status = new HashMap<String, String>();
-            if (null != job.getJobID()) {
-                status.put(ExecutableConstants.MR_JOB_ID, job.getJobID().toString());
-            }
-            if (null != job.getTrackingURL()) {
-                status.put(ExecutableConstants.YARN_APP_URL, job.getTrackingURL().toString());
-            }
-            return status;
-        } else {
-            return Collections.emptyMap();
-        }
-    }
-
-    private String mapInputRecords;
-    private String hdfsBytesWritten;
-    private String hdfsBytesRead;
-
-    public String getMapInputRecords() {
-        return mapInputRecords;
-    }
-
-    public String getHdfsBytesWritten() {
-        return hdfsBytesWritten;
-    }
-
-    public String getHdfsBytesRead() {
-        return hdfsBytesRead;
-    }
-    
-    public void updateJobCounter() {
-        try {
-            Counters counters = job.getCounters();
-            if (counters == null) {
-                String errorMsg = "no counters for job " + getMrJobId();
-                log.warn(errorMsg);
-                output.append(errorMsg);
-                return;
-            }
-            this.output.append(counters.toString()).append("\n");
-            log.debug(counters.toString());
-
-            mapInputRecords = String.valueOf(counters.findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue());
-            hdfsBytesWritten = String.valueOf(counters.findCounter("FileSystemCounters", "HDFS_BYTES_WRITTEN").getValue());
-            hdfsBytesRead = String.valueOf(counters.findCounter("FileSystemCounters", "HDFS_BYTES_READ").getValue());
-        } catch (Exception e) {
-            log.error(e.getLocalizedMessage(), e);
-            output.append(e.getLocalizedMessage());
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/common/HadoopShellExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/common/HadoopShellExecutable.java b/job/src/main/java/org/apache/kylin/job/common/HadoopShellExecutable.java
deleted file mode 100644
index 2da5b2a..0000000
--- a/job/src/main/java/org/apache/kylin/job/common/HadoopShellExecutable.java
+++ /dev/null
@@ -1,97 +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.common;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.reflect.Constructor;
-
-import org.apache.hadoop.util.ToolRunner;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.job.execution.AbstractExecutable;
-
-/**
- */
-public class HadoopShellExecutable extends AbstractExecutable {
-
-    private static final String KEY_MR_JOB = "HADOOP_SHELL_JOB_CLASS";
-    private static final String KEY_PARAMS = "HADOOP_SHELL_JOB_PARAMS";
-
-    public HadoopShellExecutable() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        final String mapReduceJobClass = getJobClass();
-        String params = getJobParams();
-        Preconditions.checkNotNull(mapReduceJobClass);
-        Preconditions.checkNotNull(params);
-        try {
-            final Constructor<? extends AbstractHadoopJob> constructor = ClassUtil.forName(mapReduceJobClass, AbstractHadoopJob.class).getConstructor();
-            final AbstractHadoopJob job = constructor.newInstance();
-            String[] args = params.trim().split("\\s+");
-            logger.info("parameters of the HadoopShellExecutable:");
-            logger.info(params);
-            int result;
-            StringBuilder log = new StringBuilder();
-            try {
-                result = ToolRunner.run(job, args);
-            } catch (Exception ex) {
-                logger.error("error execute " + this.toString(), ex);
-                StringWriter stringWriter = new StringWriter();
-                ex.printStackTrace(new PrintWriter(stringWriter));
-                log.append(stringWriter.toString()).append("\n");
-                result = 2;
-            }
-            log.append("result code:").append(result);
-            return result == 0 ? new ExecuteResult(ExecuteResult.State.SUCCEED, log.toString()):new ExecuteResult(ExecuteResult.State.FAILED, log.toString());
-        } catch (ReflectiveOperationException e) {
-            logger.error("error getMapReduceJobClass, class name:" + getParam(KEY_MR_JOB), e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        } catch (Exception e) {
-            logger.error("error execute " + this.toString(), e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-
-    public void setJobClass(Class<? extends AbstractHadoopJob> clazzName) {
-        setParam(KEY_MR_JOB, clazzName.getName());
-    }
-
-    public String getJobClass() throws ExecuteException {
-        return getParam(KEY_MR_JOB);
-    }
-
-    public void setJobParams(String param) {
-        setParam(KEY_PARAMS, param);
-    }
-
-    public String getJobParams() {
-        return getParam(KEY_PARAMS);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/common/HqlExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/common/HqlExecutable.java b/job/src/main/java/org/apache/kylin/job/common/HqlExecutable.java
deleted file mode 100644
index ffe45ed..0000000
--- a/job/src/main/java/org/apache/kylin/job/common/HqlExecutable.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.common;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-import org.apache.kylin.source.hive.HiveClient;
-import org.datanucleus.store.types.backed.HashMap;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.google.common.collect.Lists;
-
-import org.apache.kylin.common.util.JsonUtil;
-
-/**
- */
-public class HqlExecutable extends AbstractExecutable {
-
-    private static final String HQL = "hql";
-    private static final String HIVE_CONFIG = "hive-config";
-
-    public HqlExecutable() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        try {
-            Map<String, String> configMap = getConfiguration();
-            HiveClient hiveClient = new HiveClient(configMap);
-            
-            for (String hql: getHqls()) {
-                hiveClient.executeHQL(hql);
-            }
-            return new ExecuteResult(ExecuteResult.State.SUCCEED);
-        } catch (Exception e) {
-            logger.error("error run hive query:" + getHqls(), e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-    
-    public void setConfiguration(Map<String, String> configMap) {
-        if(configMap != null) {
-            String configStr = "";
-            try {
-                configStr = JsonUtil.writeValueAsString(configMap);
-            } catch (JsonProcessingException e) {
-                e.printStackTrace();
-            }
-            setParam(HIVE_CONFIG, configStr);
-        }
-    }
-
-
-    @SuppressWarnings("unchecked")
-    private Map<String, String> getConfiguration() {
-        String configStr = getParam(HIVE_CONFIG);
-        Map<String, String> result = null;
-        if(configStr != null) {
-            try {
-                result = JsonUtil.readValue(configStr, HashMap.class);
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-        }
-        
-        return result;
-    }
-    
-    public void setHqls(List<String> hqls) {
-        setParam(HQL, StringUtils.join(hqls, ";"));
-    }
-
-    private List<String> getHqls() {
-        final String hqls = getParam(HQL);
-        if (hqls != null) {
-            return Lists.newArrayList(StringUtils.split(hqls, ";"));
-        } else {
-            return Collections.emptyList();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/common/MapReduceExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/common/MapReduceExecutable.java b/job/src/main/java/org/apache/kylin/job/common/MapReduceExecutable.java
deleted file mode 100644
index f8eab6c..0000000
--- a/job/src/main/java/org/apache/kylin/job/common/MapReduceExecutable.java
+++ /dev/null
@@ -1,248 +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.common;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.reflect.Constructor;
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Cluster;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.JobStatus;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.kylin.job.constant.JobStepStatusEnum;
-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.ExecutableState;
-import org.apache.kylin.job.execution.ExecuteResult;
-import org.apache.kylin.job.execution.Output;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.job.tools.HadoopStatusChecker;
-
-import com.google.common.base.Preconditions;
-
-/**
- */
-public class MapReduceExecutable extends AbstractExecutable {
-
-    private static final String KEY_MR_JOB = "MR_JOB_CLASS";
-    private static final String KEY_PARAMS = "MR_JOB_PARAMS";
-    private static final String KEY_COUNTER_SAVEAS = "MR_COUNTER_SAVEAS";
-    
-    public static final String MAP_REDUCE_WAIT_TIME = "mapReduceWaitTime";
-
-    public MapReduceExecutable() {
-        super();
-    }
-
-    @Override
-    protected void onExecuteStart(ExecutableContext executableContext) {
-        final Output output = executableManager.getOutput(getId());
-        if (output.getExtra().containsKey(START_TIME)) {
-            final String mrJobId = output.getExtra().get(ExecutableConstants.MR_JOB_ID);
-            if (mrJobId == null) {
-                executableManager.updateJobOutput(getId(), ExecutableState.RUNNING, null, null);
-                return;
-            }
-            try {
-                Job job = new Cluster(new Configuration()).getJob(JobID.forName(mrJobId));
-                if (job.getJobState() == JobStatus.State.FAILED) {
-                    //remove previous mr job info
-                    super.onExecuteStart(executableContext);
-                } else {
-                    executableManager.updateJobOutput(getId(), ExecutableState.RUNNING, null, null);
-                }
-            } catch (IOException e) {
-                logger.warn("error get hadoop status");
-                super.onExecuteStart(executableContext);
-            } catch (InterruptedException e) {
-                logger.warn("error get hadoop status");
-                super.onExecuteStart(executableContext);
-            }
-        } else {
-            super.onExecuteStart(executableContext);
-        }
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        final String mapReduceJobClass = getMapReduceJobClass();
-        String params = getMapReduceParams();
-        Preconditions.checkNotNull(mapReduceJobClass);
-        Preconditions.checkNotNull(params);
-        try {
-            Job job;
-            final Map<String, String> extra = executableManager.getOutput(getId()).getExtra();
-            if (extra.containsKey(ExecutableConstants.MR_JOB_ID)) {
-                job = new Cluster(new Configuration()).getJob(JobID.forName(extra.get(ExecutableConstants.MR_JOB_ID)));
-                logger.info("mr_job_id:" + extra.get(ExecutableConstants.MR_JOB_ID + " resumed"));
-            } else {
-                final Constructor<? extends AbstractHadoopJob> constructor = ClassUtil.forName(mapReduceJobClass, AbstractHadoopJob.class).getConstructor();
-                final AbstractHadoopJob hadoopJob = constructor.newInstance();
-                hadoopJob.setAsync(true); // so the ToolRunner.run() returns right away
-                logger.info("parameters of the MapReduceExecutable:");
-                logger.info(params);
-                String[] args = params.trim().split("\\s+");
-                try {
-                    //for async mr job, ToolRunner just return 0;
-                    ToolRunner.run(hadoopJob, args);
-                } catch (Exception ex) {
-                    StringBuilder log = new StringBuilder();
-                    logger.error("error execute " + this.toString(), ex);
-                    StringWriter stringWriter = new StringWriter();
-                    ex.printStackTrace(new PrintWriter(stringWriter));
-                    log.append(stringWriter.toString()).append("\n");
-                    log.append("result code:").append(2);
-                    return new ExecuteResult(ExecuteResult.State.ERROR, log.toString());
-                }
-                job = hadoopJob.getJob();
-            }
-            final StringBuilder output = new StringBuilder();
-            final HadoopCmdOutput hadoopCmdOutput = new HadoopCmdOutput(job, output);
-
-            final String restStatusCheckUrl = getRestStatusCheckUrl(job, context.getConfig());
-            if (restStatusCheckUrl == null) {
-                logger.error("restStatusCheckUrl is null");
-                return new ExecuteResult(ExecuteResult.State.ERROR, "restStatusCheckUrl is null");
-            }
-            String mrJobId = hadoopCmdOutput.getMrJobId();
-            HadoopStatusChecker statusChecker = new HadoopStatusChecker(restStatusCheckUrl, mrJobId, output);
-            JobStepStatusEnum status = JobStepStatusEnum.NEW;
-            while (!isDiscarded()) {
-                JobStepStatusEnum newStatus = statusChecker.checkStatus();
-                if (status == JobStepStatusEnum.KILLED) {
-                    executableManager.updateJobOutput(getId(), ExecutableState.ERROR, Collections.<String, String>emptyMap(), "killed by admin");
-                    return new ExecuteResult(ExecuteResult.State.FAILED, "killed by admin");
-                }
-                if (status == JobStepStatusEnum.WAITING && (newStatus == JobStepStatusEnum.FINISHED || newStatus == JobStepStatusEnum.ERROR || newStatus == JobStepStatusEnum.RUNNING)) {
-                    final long waitTime = System.currentTimeMillis() - getStartTime();
-                    setMapReduceWaitTime(waitTime);
-                }
-                status = newStatus;
-                executableManager.addJobInfo(getId(), hadoopCmdOutput.getInfo());
-                if (status.isComplete()) {
-                    final Map<String, String> info = hadoopCmdOutput.getInfo();
-                    readCounters(hadoopCmdOutput, info);
-                    executableManager.addJobInfo(getId(), info);
-
-                    if (status == JobStepStatusEnum.FINISHED) {
-                        return new ExecuteResult(ExecuteResult.State.SUCCEED, output.toString());
-                    } else {
-                        return new ExecuteResult(ExecuteResult.State.FAILED, output.toString());
-                    }
-                }
-                Thread.sleep(context.getConfig().getYarnStatusCheckIntervalSeconds() * 1000);
-            }
-            //TODO kill discarded mr job using "hadoop job -kill " + mrJobId
-
-            return new ExecuteResult(ExecuteResult.State.DISCARDED, output.toString());
-
-        } catch (ReflectiveOperationException e) {
-            logger.error("error getMapReduceJobClass, class name:" + getParam(KEY_MR_JOB), e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        } catch (Exception e) {
-            logger.error("error execute " + this.toString(), e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-
-    private void readCounters(final HadoopCmdOutput hadoopCmdOutput, final Map<String, String> info) {
-        hadoopCmdOutput.updateJobCounter();
-        info.put(ExecutableConstants.SOURCE_RECORDS_COUNT, hadoopCmdOutput.getMapInputRecords());
-        info.put(ExecutableConstants.SOURCE_RECORDS_SIZE, hadoopCmdOutput.getHdfsBytesRead());
-        info.put(ExecutableConstants.HDFS_BYTES_WRITTEN, hadoopCmdOutput.getHdfsBytesWritten());
-        
-        String saveAs = getParam(KEY_COUNTER_SAVEAS);
-        if (saveAs != null) {
-            String[] saveAsNames = saveAs.split(",");
-            saveCounterAs(hadoopCmdOutput.getMapInputRecords(), saveAsNames, 0, info);
-            saveCounterAs(hadoopCmdOutput.getHdfsBytesRead(), saveAsNames, 1, info);
-            saveCounterAs(hadoopCmdOutput.getHdfsBytesWritten(), saveAsNames, 2, info);
-        }
-    }
-
-    private void saveCounterAs(String counter, String[] saveAsNames, int i, Map<String, String> info) {
-        if (saveAsNames.length > i && StringUtils.isBlank(saveAsNames[i]) == false) {
-            info.put(saveAsNames[i].trim(), counter);
-        }
-    }
-
-    private String getRestStatusCheckUrl(Job job, KylinConfig config) {
-        final String yarnStatusCheckUrl = config.getYarnStatusCheckUrl();
-        if (yarnStatusCheckUrl != null) {
-            return yarnStatusCheckUrl;
-        } else {
-            logger.info(KylinConfig.KYLIN_JOB_YARN_APP_REST_CHECK_URL + " is not set, read from job configuration");
-        }
-        String rmWebHost = job.getConfiguration().get("yarn.resourcemanager.webapp.address");
-        if (StringUtils.isEmpty(rmWebHost)) {
-            return null;
-        }
-        if (rmWebHost.startsWith("http://") || rmWebHost.startsWith("https://")) {
-            //do nothing
-        } else {
-            rmWebHost = "http://" + rmWebHost;
-        }
-        logger.info("yarn.resourcemanager.webapp.address:" + rmWebHost);
-        return rmWebHost + "/ws/v1/cluster/apps/${job_id}?anonymous=true";
-    }
-
-    public long getMapReduceWaitTime() {
-        return getExtraInfoAsLong(MAP_REDUCE_WAIT_TIME, 0L);
-    }
-
-    public void setMapReduceWaitTime(long t) {
-        addExtraInfo(MAP_REDUCE_WAIT_TIME, t + "");
-    }
-
-    public void setMapReduceJobClass(Class<? extends AbstractHadoopJob> clazzName) {
-        setParam(KEY_MR_JOB, clazzName.getName());
-    }
-
-    public String getMapReduceJobClass() throws ExecuteException {
-        return getParam(KEY_MR_JOB);
-    }
-
-    public void setMapReduceParams(String param) {
-        setParam(KEY_PARAMS, param);
-    }
-
-    public String getMapReduceParams() {
-        return getParam(KEY_PARAMS);
-    }
-    
-    public String getCounterSaveAs() {
-        return getParam(KEY_COUNTER_SAVEAS);
-    }
-    
-    public void setCounterSaveAs(String value) {
-        setParam(KEY_COUNTER_SAVEAS, value);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/common/ShellExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/common/ShellExecutable.java b/job/src/main/java/org/apache/kylin/job/common/ShellExecutable.java
deleted file mode 100644
index 786698e..0000000
--- a/job/src/main/java/org/apache/kylin/job/common/ShellExecutable.java
+++ /dev/null
@@ -1,143 +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.common;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.kylin.common.util.Pair;
-
-import com.google.common.collect.Maps;
-import org.apache.kylin.common.util.Logger;
-import org.apache.kylin.job.constant.ExecutableConstants;
-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;
-
-/**
- */
-public class ShellExecutable extends AbstractExecutable {
-
-    private static final String CMD = "cmd";
-
-    public ShellExecutable() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        try {
-            logger.info("executing:" + getCmd());
-            final ShellExecutableLogger logger = new ShellExecutableLogger();
-            final Pair<Integer, String> result = context.getConfig().getCliCommandExecutor().execute(getCmd(), logger);
-            executableManager.addJobInfo(getId(), logger.getInfo());
-            return new ExecuteResult(result.getFirst() == 0? ExecuteResult.State.SUCCEED: ExecuteResult.State.FAILED, result.getSecond());
-        } catch (IOException e) {
-            logger.error("job:" + getId() + " execute finished with exception", e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-
-    public void setCmd(String cmd) {
-        setParam(CMD, cmd);
-    }
-
-    public String getCmd() {
-        return getParam(CMD);
-    }
-
-    private static class ShellExecutableLogger implements Logger {
-
-        private final Map<String, String> info = Maps.newHashMap();
-
-        private static final Pattern PATTERN_APP_ID = Pattern.compile("Submitted application (.*?) to ResourceManager");
-        private static final Pattern PATTERN_APP_URL = Pattern.compile("The url to track the job: (.*)");
-        private static final Pattern PATTERN_JOB_ID = Pattern.compile("Running job: (.*)");
-        private static final Pattern PATTERN_HDFS_BYTES_WRITTEN = Pattern.compile("(?:HD|MAPR)FS: Number of bytes written=(\\d+)");
-        private static final Pattern PATTERN_SOURCE_RECORDS_COUNT = Pattern.compile("Map input records=(\\d+)");
-        private static final Pattern PATTERN_SOURCE_RECORDS_SIZE = Pattern.compile("(?:HD|MAPR)FS Read: (\\d+) HDFS Write");
-
-        // hive
-        private static final Pattern PATTERN_HIVE_APP_ID_URL = Pattern.compile("Starting Job = (.*?), Tracking URL = (.*)");
-        private static final Pattern PATTERN_HIVE_BYTES_WRITTEN = Pattern.compile("(?:HD|MAPR)FS Read: (\\d+) HDFS Write: (\\d+) SUCCESS");
-
-        @Override
-        public void log(String message) {
-            Matcher matcher = PATTERN_APP_ID.matcher(message);
-            if (matcher.find()) {
-                String appId = matcher.group(1);
-                info.put(ExecutableConstants.YARN_APP_ID, appId);
-            }
-
-            matcher = PATTERN_APP_URL.matcher(message);
-            if (matcher.find()) {
-                String appTrackingUrl = matcher.group(1);
-                info.put(ExecutableConstants.YARN_APP_URL, appTrackingUrl);
-            }
-
-            matcher = PATTERN_JOB_ID.matcher(message);
-            if (matcher.find()) {
-                String mrJobID = matcher.group(1);
-                info.put(ExecutableConstants.MR_JOB_ID, mrJobID);
-            }
-
-            matcher = PATTERN_HDFS_BYTES_WRITTEN.matcher(message);
-            if (matcher.find()) {
-                String hdfsWritten = matcher.group(1);
-                info.put(ExecutableConstants.HDFS_BYTES_WRITTEN, hdfsWritten);
-            }
-
-            matcher = PATTERN_SOURCE_RECORDS_COUNT.matcher(message);
-            if (matcher.find()) {
-                String sourceCount = matcher.group(1);
-                info.put(ExecutableConstants.SOURCE_RECORDS_COUNT, sourceCount);
-            }
-
-            matcher = PATTERN_SOURCE_RECORDS_SIZE.matcher(message);
-            if (matcher.find()) {
-                String sourceSize = matcher.group(1);
-                info.put(ExecutableConstants.SOURCE_RECORDS_SIZE, sourceSize);
-            }
-
-            // hive
-            matcher = PATTERN_HIVE_APP_ID_URL.matcher(message);
-            if (matcher.find()) {
-                String jobId = matcher.group(1);
-                String trackingUrl = matcher.group(2);
-                info.put(ExecutableConstants.MR_JOB_ID, jobId);
-                info.put(ExecutableConstants.YARN_APP_URL, trackingUrl);
-            }
-
-            matcher = PATTERN_HIVE_BYTES_WRITTEN.matcher(message);
-            if (matcher.find()) {
-                // String hdfsRead = matcher.group(1);
-                String hdfsWritten = matcher.group(2);
-                info.put(ExecutableConstants.HDFS_BYTES_WRITTEN, hdfsWritten);
-            }
-        }
-
-        Map<String, String> getInfo() {
-            return info;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java b/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
deleted file mode 100644
index 3a64d02..0000000
--- a/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.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.constant;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-public interface BatchConstants {
-
-    public static final char INTERMEDIATE_TABLE_ROW_DELIMITER = 127;
-
-    public static final String CFG_CUBE_NAME = "cube.name";
-    public static final String CFG_CUBE_SEGMENT_NAME = "cube.segment.name";
-
-    public static final String CFG_II_NAME = "ii.name";
-    public static final String CFG_II_SEGMENT_NAME = "ii.segment.name";
-
-    public static final String INPUT_DELIM = "input.delim";
-    public static final String OUTPUT_PATH = "output.path";
-
-    public static final String TABLE_NAME = "table.name";
-    public static final String TABLE_COLUMNS = "table.columns";
-
-    public static final String CFG_IS_MERGE = "is.merge";
-    public static final String CFG_CUBE_INTERMEDIATE_TABLE_ROW_DELIMITER = "cube.intermediate.table.row.delimiter";
-
-    public static final String MAPREDUCE_COUTNER_GROUP_NAME = "Cube Builder";
-
-    public static final String MAPPER_SAMPLE_NUMBER = "mapper.sample.number";
-    public static final String REGION_NUMBER = "region.number";
-    public static final String CUBE_CAPACITY = "cube.capacity";
-
-    public static final String CFG_STATISTICS_ENABLED = "statistics.enabled";
-    public static final String CFG_STATISTICS_OUTPUT = "statistics.ouput";
-    public static final String CFG_STATISTICS_SAMPLING_PERCENT = "statistics.sampling.percent";
-    public static final String CFG_STATISTICS_CUBE_ESTIMATION = "cube_statistics.txt";
-    public static final String CFG_STATISTICS_CUBOID_ESTIMATION = "cuboid_statistics.seq";
-
-    public static final int COUNTER_MAX = 100000;
-    public static final int ERROR_RECORD_THRESHOLD = 100;
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java b/job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
deleted file mode 100644
index fdcfdbe..0000000
--- a/job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
+++ /dev/null
@@ -1,80 +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.constant;
-
-/**
- */
-public final class ExecutableConstants {
-
-    private ExecutableConstants(){}
-
-    public static final String YARN_APP_ID = "yarn_application_id";
-
-    public static final String YARN_APP_URL = "yarn_application_tracking_url";
-    public static final String MR_JOB_ID = "mr_job_id";
-    public static final String HDFS_BYTES_WRITTEN = "hdfs_bytes_written";
-    public static final String SOURCE_RECORDS_COUNT = "source_records_count";
-    public static final String SOURCE_RECORDS_SIZE = "source_records_size";
-    public static final String GLOBAL_LISTENER_NAME = "ChainListener";
-
-
-
-
-    public static final int DEFAULT_SCHEDULER_INTERVAL_SECONDS = 60;
-
-    public static final String CUBE_JOB_GROUP_NAME = "cube_job_group";
-
-    public static final String DAEMON_JOB_GROUP_NAME = "daemon_job_group";
-    public static final String STEP_NAME_BUILD_DICTIONARY = "Build Dimension Dictionary";
-
-    public static final String STEP_NAME_CREATE_FLAT_HIVE_TABLE = "Create Intermediate Flat Hive Table";
-    public static final String STEP_NAME_FACT_DISTINCT_COLUMNS = "Extract Fact Table Distinct Columns";
-    public static final String STEP_NAME_BUILD_BASE_CUBOID = "Build Base Cuboid Data";
-    public static final String STEP_NAME_BUILD_IN_MEM_CUBE = "Build Cube";
-    public static final String STEP_NAME_BUILD_N_D_CUBOID = "Build N-Dimension Cuboid Data";
-    public static final String STEP_NAME_GET_CUBOID_KEY_DISTRIBUTION = "Calculate HTable Region Splits";
-    public static final String STEP_NAME_CREATE_HBASE_TABLE = "Create HTable";
-    public static final String STEP_NAME_CONVERT_CUBOID_TO_HFILE = "Convert Cuboid Data to HFile";
-    public static final String STEP_NAME_BULK_LOAD_HFILE = "Load HFile to HBase Table";
-    public static final String STEP_NAME_MERGE_DICTIONARY = "Merge Cuboid Dictionary";
-    public static final String STEP_NAME_MERGE_STATISTICS = "Merge Cuboid Statistics";
-    public static final String STEP_NAME_SAVE_STATISTICS = "Save Cuboid Statistics";
-    public static final String STEP_NAME_MERGE_CUBOID = "Merge Cuboid Data";
-    public static final String STEP_NAME_UPDATE_CUBE_INFO = "Update Cube Info";
-    public static final String STEP_NAME_GARBAGE_COLLECTION = "Garbage Collection";
-    
-    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 PROP_ENGINE_CONTEXT = "jobengineConfig";
-    public static final String PROP_JOB_FLOW = "jobFlow";
-    public static final String PROP_JOBINSTANCE_UUID = "jobInstanceUuid";
-    public static final String PROP_JOBSTEP_SEQ_ID = "jobStepSequenceID";
-    public static final String PROP_COMMAND = "command";
-    // public static final String PROP_STORAGE_LOCATION =
-    // "storageLocationIdentifier";
-    public static final String PROP_JOB_ASYNC = "jobAsync";
-    public static final String PROP_JOB_CMD_EXECUTOR = "jobCmdExecutor";
-    public static final String PROP_JOB_CMD_OUTPUT = "jobCmdOutput";
-    public static final String PROP_JOB_KILLED = "jobKilled";
-    public static final String PROP_JOB_RUNTIME_FLOWS = "jobFlows";
-
-    public static final String NOTIFY_EMAIL_TEMPLATE = "<div><b>Build Result of Job ${job_name}</b><pre><ul>" + "<li>Build Result: <b>${result}</b></li>" + "<li>Job Engine: ${job_engine}</li>" + "<li>Cube Name: ${cube_name}</li>" + "<li>Start Time: ${start_time}</li>" + "<li>Duration: ${duration}</li>" + "<li>MR Waiting: ${mr_waiting}</li>" + "<li>Last Update Time: ${last_update_time}</li>" + "<li>Submitter: ${submitter}</li>" + "<li>Error Log: ${error_log}</li>" + "</ul></pre><div/>";
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/constant/JobStatusEnum.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/constant/JobStatusEnum.java b/job/src/main/java/org/apache/kylin/job/constant/JobStatusEnum.java
deleted file mode 100644
index a4ef564..0000000
--- a/job/src/main/java/org/apache/kylin/job/constant/JobStatusEnum.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.constant;
-
-public enum JobStatusEnum {
-
-    NEW(0), PENDING(1), RUNNING(2), FINISHED(4), ERROR(8), DISCARDED(16);
-
-    private final int code;
-
-    private JobStatusEnum(int statusCode) {
-        this.code = statusCode;
-    }
-
-    public static JobStatusEnum getByCode(int statusCode) {
-        for (JobStatusEnum status : values()) {
-            if (status.getCode() == statusCode) {
-                return status;
-            }
-        }
-
-        return null;
-    }
-
-    public int getCode() {
-        return this.code;
-    }
-
-    public boolean isComplete() {
-        return code == JobStatusEnum.FINISHED.getCode() || code == JobStatusEnum.ERROR.getCode() || code == JobStatusEnum.DISCARDED.getCode();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/constant/JobStepCmdTypeEnum.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/constant/JobStepCmdTypeEnum.java b/job/src/main/java/org/apache/kylin/job/constant/JobStepCmdTypeEnum.java
deleted file mode 100644
index 02b40a3..0000000
--- a/job/src/main/java/org/apache/kylin/job/constant/JobStepCmdTypeEnum.java
+++ /dev/null
@@ -1,27 +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.constant;
-
-/**
- * @author xduo, ysong1
- * 
- */
-public enum JobStepCmdTypeEnum {
-    SHELL_CMD, SHELL_CMD_HADOOP, JAVA_CMD_HADOOP_FACTDISTINCT, JAVA_CMD_HADOOP_BASECUBOID, JAVA_CMD_HADOOP_NDCUBOID, JAVA_CMD_HADOOP_RANGEKEYDISTRIBUTION, JAVA_CMD_HADOOP_CONVERTHFILE, JAVA_CMD_HADOOP_MERGECUBOID, JAVA_CMD_HADOOP_NO_MR_DICTIONARY, JAVA_CMD_HADDOP_NO_MR_CREATEHTABLE, JAVA_CMD_HADOOP_NO_MR_BULKLOAD
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/constant/JobStepStatusEnum.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/constant/JobStepStatusEnum.java b/job/src/main/java/org/apache/kylin/job/constant/JobStepStatusEnum.java
deleted file mode 100644
index 08ee79a..0000000
--- a/job/src/main/java/org/apache/kylin/job/constant/JobStepStatusEnum.java
+++ /dev/null
@@ -1,51 +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.constant;
-
-public enum JobStepStatusEnum {
-    NEW(0), PENDING(1), RUNNING(2), FINISHED(4), ERROR(8), DISCARDED(16), WAITING(32), KILLED(64);
-
-    private final int code;
-
-    private JobStepStatusEnum(int statusCode) {
-        this.code = statusCode;
-    }
-
-    public static JobStepStatusEnum getByCode(int statusCode) {
-        for (JobStepStatusEnum status : values()) {
-            if (status.getCode() == statusCode) {
-                return status;
-            }
-        }
-
-        return null;
-    }
-
-    public int getCode() {
-        return this.code;
-    }
-
-    public boolean isComplete() {
-        return code == JobStepStatusEnum.FINISHED.getCode() || code == JobStepStatusEnum.ERROR.getCode() || code == JobStepStatusEnum.DISCARDED.getCode();
-    }
-
-    public boolean isRunable() {
-        return code == JobStepStatusEnum.PENDING.getCode() || code == JobStepStatusEnum.ERROR.getCode();
-    }
-}



[09/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilder.java b/job/src/main/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilder.java
deleted file mode 100644
index 2cde011..0000000
--- a/job/src/main/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilder.java
+++ /dev/null
@@ -1,661 +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.inmemcubing;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.common.util.MemoryBudgetController;
-import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.cuboid.CuboidScheduler;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.measure.DoubleMutable;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.cube.CubeGridTable;
-import org.apache.kylin.storage.gridtable.GTAggregateScanner;
-import org.apache.kylin.storage.gridtable.GTBuilder;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.GridTable;
-import org.apache.kylin.storage.gridtable.IGTScanner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-/**
- * Build a cube (many cuboids) in memory. Calculating multiple cuboids at the same time as long as memory permits.
- * Assumes base cuboid fits in memory or otherwise OOM exception will occur.
- */
-public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
-
-    private static Logger logger = LoggerFactory.getLogger(InMemCubeBuilder.class);
-    private static final LongMutable ONE = new LongMutable(1l);
-
-    private final CuboidScheduler cuboidScheduler;
-    private final long baseCuboidId;
-    private final int totalCuboidCount;
-    private final CubeJoinedFlatTableDesc intermediateTableDesc;
-    private final MeasureCodec measureCodec;
-    private final String[] metricsAggrFuncs;
-    private final int[] hbaseMeasureRefIndex;
-    private final MeasureDesc[] measureDescs;
-    private final int measureCount;
-
-    private MemoryBudgetController memBudget;
-    private Thread[] taskThreads;
-    private Throwable[] taskThreadExceptions;
-    private LinkedBlockingQueue<CuboidTask> taskPending;
-    private AtomicInteger taskCuboidCompleted = new AtomicInteger(0);
-
-    private CuboidResult baseResult;
-    private Object[] totalSumForSanityCheck;
-    private ICuboidCollector resultCollector;
-
-    public InMemCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<?>> dictionaryMap) {
-        super(cubeDesc, dictionaryMap);
-        this.cuboidScheduler = new CuboidScheduler(cubeDesc);
-        this.baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        this.totalCuboidCount = cuboidScheduler.getCuboidCount();
-        this.intermediateTableDesc = new CubeJoinedFlatTableDesc(cubeDesc, null);
-        this.measureCodec = new MeasureCodec(cubeDesc.getMeasures());
-
-        Map<String, Integer> measureIndexMap = Maps.newHashMap();
-        List<String> metricsAggrFuncsList = Lists.newArrayList();
-        measureCount = cubeDesc.getMeasures().size();
-
-        List<MeasureDesc> measureDescsList = Lists.newArrayList();
-        hbaseMeasureRefIndex = new int[measureCount];
-        int measureRef = 0;
-        for (HBaseColumnFamilyDesc familyDesc : cubeDesc.getHbaseMapping().getColumnFamily()) {
-            for (HBaseColumnDesc hbaseColDesc : familyDesc.getColumns()) {
-                for (MeasureDesc measure : hbaseColDesc.getMeasures()) {
-                    for (int j = 0; j < measureCount; j++) {
-                        if (cubeDesc.getMeasures().get(j).equals(measure)) {
-                            measureDescsList.add(measure);
-                            hbaseMeasureRefIndex[measureRef] = j;
-                            break;
-                        }
-                    }
-                    measureRef++;
-                }
-            }
-        }
-
-        for (int i = 0; i < measureCount; i++) {
-            MeasureDesc measureDesc = measureDescsList.get(i);
-            metricsAggrFuncsList.add(measureDesc.getFunction().getExpression());
-            measureIndexMap.put(measureDesc.getName(), i);
-        }
-        this.metricsAggrFuncs = metricsAggrFuncsList.toArray(new String[metricsAggrFuncsList.size()]);
-        this.measureDescs = cubeDesc.getMeasures().toArray(new MeasureDesc[measureCount]);
-    }
-
-    private GridTable newGridTableByCuboidID(long cuboidID) throws IOException {
-        GTInfo info = CubeGridTable.newGTInfo(cubeDesc, cuboidID, dictionaryMap);
-
-        // Below several store implementation are very similar in performance. The ConcurrentDiskStore is the simplest.
-        // MemDiskStore store = new MemDiskStore(info, memBudget == null ? MemoryBudgetController.ZERO_BUDGET : memBudget);
-        // MemDiskStore store = new MemDiskStore(info, MemoryBudgetController.ZERO_BUDGET);
-        ConcurrentDiskStore store = new ConcurrentDiskStore(info);
-
-        GridTable gridTable = new GridTable(info, store);
-        return gridTable;
-    }
-
-    private Pair<ImmutableBitSet, ImmutableBitSet> getDimensionAndMetricColumnBitSet(long cuboidId) {
-        BitSet bitSet = BitSet.valueOf(new long[] { cuboidId });
-        BitSet dimension = new BitSet();
-        dimension.set(0, bitSet.cardinality());
-        BitSet metrics = new BitSet();
-        metrics.set(bitSet.cardinality(), bitSet.cardinality() + this.measureCount);
-        return new Pair<ImmutableBitSet, ImmutableBitSet>(new ImmutableBitSet(dimension), new ImmutableBitSet(metrics));
-    }
-
-    @Override
-    public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
-        ConcurrentNavigableMap<Long, CuboidResult> result = build(input);
-        for (CuboidResult cuboidResult : result.values()) {
-            outputCuboid(cuboidResult.cuboidId, cuboidResult.table, output);
-            cuboidResult.table.close();
-        }
-    }
-
-    ConcurrentNavigableMap<Long, CuboidResult> build(BlockingQueue<List<String>> input) throws IOException {
-        final ConcurrentNavigableMap<Long, CuboidResult> result = new ConcurrentSkipListMap<Long, CuboidResult>();
-        build(input, new ICuboidCollector() {
-            @Override
-            public void collect(CuboidResult cuboidResult) {
-                result.put(cuboidResult.cuboidId, cuboidResult);
-            }
-        });
-        return result;
-    }
-
-    interface ICuboidCollector {
-        void collect(CuboidResult result);
-    }
-
-    static class CuboidResult {
-        public long cuboidId;
-        public GridTable table;
-        public int nRows;
-        public long timeSpent;
-        public int aggrCacheMB;
-
-        public CuboidResult(long cuboidId, GridTable table, int nRows, long timeSpent, int aggrCacheMB) {
-            this.cuboidId = cuboidId;
-            this.table = table;
-            this.nRows = nRows;
-            this.timeSpent = timeSpent;
-            this.aggrCacheMB = aggrCacheMB;
-        }
-    }
-
-    private void build(BlockingQueue<List<String>> input, ICuboidCollector collector) throws IOException {
-        long startTime = System.currentTimeMillis();
-        logger.info("In Mem Cube Build start, " + cubeDesc.getName());
-
-        // multiple threads to compute cuboid in parallel
-        taskPending = new LinkedBlockingQueue<>();
-        taskCuboidCompleted.set(0);
-        taskThreads = prepareTaskThreads();
-        taskThreadExceptions = new Throwable[taskThreadCount];
-
-        // build base cuboid
-        resultCollector = collector;
-        totalSumForSanityCheck = null;
-        baseResult = createBaseCuboid(input);
-        if (baseResult.nRows == 0)
-            return;
-
-        // plan memory budget
-        makeMemoryBudget();
-
-        // kick off N-D cuboid tasks and output
-        addChildTasks(baseResult);
-        start(taskThreads);
-
-        // wait complete
-        join(taskThreads);
-
-        long endTime = System.currentTimeMillis();
-        logger.info("In Mem Cube Build end, " + cubeDesc.getName() + ", takes " + (endTime - startTime) + " ms");
-
-        throwExceptionIfAny();
-    }
-
-    public void abort() {
-        interrupt(taskThreads);
-    }
-
-    private void start(Thread... threads) {
-        for (Thread t : threads)
-            t.start();
-    }
-
-    private void interrupt(Thread... threads) {
-        for (Thread t : threads)
-            t.interrupt();
-    }
-
-    private void join(Thread... threads) throws IOException {
-        try {
-            for (Thread t : threads)
-                t.join();
-        } catch (InterruptedException e) {
-            throw new IOException("interrupted while waiting task and output complete", e);
-        }
-    }
-
-    private void throwExceptionIfAny() throws IOException {
-        ArrayList<Throwable> errors = new ArrayList<Throwable>();
-        for (int i = 0; i < taskThreadCount; i++) {
-            Throwable t = taskThreadExceptions[i];
-            if (t != null)
-                errors.add(t);
-        }
-        if (errors.isEmpty()) {
-            return;
-        } else if (errors.size() == 1) {
-            Throwable t = errors.get(0);
-            if (t instanceof IOException)
-                throw (IOException) t;
-            else
-                throw new IOException(t);
-        } else {
-            for (Throwable t : errors)
-                logger.error("Exception during in-mem cube build", t);
-            throw new IOException(errors.size() + " exceptions during in-mem cube build, cause set to the first, check log for more", errors.get(0));
-        }
-    }
-
-    private Thread[] prepareTaskThreads() {
-        Thread[] result = new Thread[taskThreadCount];
-        for (int i = 0; i < taskThreadCount; i++) {
-            result[i] = new CuboidTaskThread(i);
-        }
-        return result;
-    }
-
-    public boolean isAllCuboidDone() {
-        return taskCuboidCompleted.get() == totalCuboidCount;
-    }
-
-    private class CuboidTaskThread extends Thread {
-        private int id;
-
-        CuboidTaskThread(int id) {
-            super("CuboidTask-" + id);
-            this.id = id;
-        }
-
-        @Override
-        public void run() {
-            try {
-                while (!isAllCuboidDone()) {
-                    CuboidTask task = null;
-                    while (task == null && taskHasNoException()) {
-                        task = taskPending.poll(15, TimeUnit.SECONDS);
-                    }
-                    // if task error occurs
-                    if (task == null)
-                        break;
-
-                    CuboidResult newCuboid = buildCuboid(task.parent, task.childCuboidId);
-                    addChildTasks(newCuboid);
-
-                    if (isAllCuboidDone()) {
-                        for (Thread t : taskThreads) {
-                            if (t != Thread.currentThread())
-                                t.interrupt();
-                        }
-                    }
-                }
-            } catch (Throwable ex) {
-                if (!isAllCuboidDone()) {
-                    logger.error("task thread exception", ex);
-                    taskThreadExceptions[id] = ex;
-                }
-            }
-        }
-    }
-
-    private boolean taskHasNoException() {
-        for (int i = 0; i < taskThreadExceptions.length; i++)
-            if (taskThreadExceptions[i] != null)
-                return false;
-        return true;
-    }
-
-    private void addChildTasks(CuboidResult parent) {
-        List<Long> children = cuboidScheduler.getSpanningCuboid(parent.cuboidId);
-        for (Long child : children) {
-            taskPending.add(new CuboidTask(parent, child));
-        }
-    }
-
-    private int getSystemAvailMB() {
-        Runtime.getRuntime().gc();
-        try {
-            Thread.sleep(500);
-        } catch (InterruptedException e) {
-            logger.error("", e);
-        }
-        return MemoryBudgetController.getSystemAvailMB();
-    }
-
-    private void makeMemoryBudget() {
-        int systemAvailMB = getSystemAvailMB();
-        logger.info("System avail " + systemAvailMB + " MB");
-        int reserve = Math.max(reserveMemoryMB, baseResult.aggrCacheMB / 3);
-        logger.info("Reserve " + reserve + " MB for system basics");
-
-        int budget = systemAvailMB - reserve;
-        if (budget < baseResult.aggrCacheMB) {
-            // make sure we have base aggr cache as minimal
-            budget = baseResult.aggrCacheMB;
-            logger.warn("!!! System avail memory (" + systemAvailMB + " MB) is less than base aggr cache (" + baseResult.aggrCacheMB + " MB) + minimal reservation (" + reserve + " MB), consider increase JVM heap -Xmx");
-        }
-
-        logger.info("Memory Budget is " + budget + " MB");
-        memBudget = new MemoryBudgetController(budget);
-    }
-
-    private CuboidResult createBaseCuboid(BlockingQueue<List<String>> input) throws IOException {
-        GridTable baseCuboid = newGridTableByCuboidID(baseCuboidId);
-        GTBuilder baseBuilder = baseCuboid.rebuild();
-        IGTScanner baseInput = new InputConverter(baseCuboid.getInfo(), input);
-
-        int mbBefore = getSystemAvailMB();
-        int mbAfter = 0;
-
-        Pair<ImmutableBitSet, ImmutableBitSet> dimensionMetricsBitSet = getDimensionAndMetricColumnBitSet(baseCuboidId);
-        GTScanRequest req = new GTScanRequest(baseCuboid.getInfo(), null, dimensionMetricsBitSet.getFirst(), dimensionMetricsBitSet.getSecond(), metricsAggrFuncs, null);
-        GTAggregateScanner aggregationScanner = new GTAggregateScanner(baseInput, req);
-
-        long startTime = System.currentTimeMillis();
-        logger.info("Calculating cuboid " + baseCuboidId);
-
-        int count = 0;
-        for (GTRecord r : aggregationScanner) {
-            if (mbAfter == 0) {
-                mbAfter = getSystemAvailMB();
-            }
-            baseBuilder.write(r);
-            count++;
-        }
-        aggregationScanner.close();
-        baseBuilder.close();
-
-        long timeSpent = System.currentTimeMillis() - startTime;
-        logger.info("Cuboid " + baseCuboidId + " has " + count + " rows, build takes " + timeSpent + "ms");
-
-        int mbBaseAggrCacheOnHeap = mbAfter == 0 ? 0 : mbBefore - mbAfter;
-        int mbEstimateBaseAggrCache = (int) (aggregationScanner.getEstimateSizeOfAggrCache() / MemoryBudgetController.ONE_MB);
-        int mbBaseAggrCache = Math.max((int) (mbBaseAggrCacheOnHeap * 1.1), mbEstimateBaseAggrCache);
-        mbBaseAggrCache = Math.max(mbBaseAggrCache, 10); // let it be 10 MB at least
-        logger.info("Base aggr cache is " + mbBaseAggrCache + " MB (heap " + mbBaseAggrCacheOnHeap + " MB, estimate " + mbEstimateBaseAggrCache + " MB)");
-
-        return updateCuboidResult(baseCuboidId, baseCuboid, count, timeSpent, mbBaseAggrCache);
-    }
-
-    private CuboidResult updateCuboidResult(long cuboidId, GridTable table, int nRows, long timeSpent, int aggrCacheMB) {
-        if (aggrCacheMB <= 0) {
-            aggrCacheMB = (int) Math.ceil(1.0 * nRows / baseResult.nRows * baseResult.aggrCacheMB);
-        }
-
-        CuboidResult result = new CuboidResult(cuboidId, table, nRows, timeSpent, aggrCacheMB);
-        taskCuboidCompleted.incrementAndGet();
-
-        resultCollector.collect(result);
-        return result;
-    }
-
-    private CuboidResult buildCuboid(CuboidResult parent, long cuboidId) throws IOException {
-        final String consumerName = "AggrCache@Cuboid " + cuboidId;
-        MemoryBudgetController.MemoryConsumer consumer = new MemoryBudgetController.MemoryConsumer() {
-            @Override
-            public int freeUp(int mb) {
-                return 0; // cannot free up on demand
-            }
-
-            @Override
-            public String toString() {
-                return consumerName;
-            }
-        };
-
-        // reserve memory for aggregation cache, can't be larger than the parent
-        memBudget.reserveInsist(consumer, parent.aggrCacheMB);
-        try {
-            return aggregateCuboid(parent, cuboidId);
-        } finally {
-            memBudget.reserve(consumer, 0);
-        }
-    }
-
-    private CuboidResult aggregateCuboid(CuboidResult parent, long cuboidId) throws IOException {
-        Pair<ImmutableBitSet, ImmutableBitSet> columnBitSets = getDimensionAndMetricColumnBitSet(parent.cuboidId);
-        ImmutableBitSet parentDimensions = columnBitSets.getFirst();
-        ImmutableBitSet measureColumns = columnBitSets.getSecond();
-        ImmutableBitSet childDimensions = parentDimensions;
-
-        long mask = Long.highestOneBit(parent.cuboidId);
-        long childCuboidId = cuboidId;
-        long parentCuboidIdActualLength = Long.SIZE - Long.numberOfLeadingZeros(parent.cuboidId);
-        int index = 0;
-        for (int i = 0; i < parentCuboidIdActualLength; i++) {
-            if ((mask & parent.cuboidId) > 0) {
-                if ((mask & childCuboidId) == 0) {
-                    // this dim will be aggregated
-                    childDimensions = childDimensions.set(index, false);
-                }
-                index++;
-            }
-            mask = mask >> 1;
-        }
-
-        return scanAndAggregateGridTable(parent.table, cuboidId, childDimensions, measureColumns);
-    }
-
-    private CuboidResult scanAndAggregateGridTable(GridTable gridTable, long cuboidId, ImmutableBitSet aggregationColumns, ImmutableBitSet measureColumns) throws IOException {
-        long startTime = System.currentTimeMillis();
-        logger.info("Calculating cuboid " + cuboidId);
-
-        GTScanRequest req = new GTScanRequest(gridTable.getInfo(), null, aggregationColumns, measureColumns, metricsAggrFuncs, null);
-        GTAggregateScanner scanner = (GTAggregateScanner) gridTable.scan(req);
-        GridTable newGridTable = newGridTableByCuboidID(cuboidId);
-        GTBuilder builder = newGridTable.rebuild();
-
-        ImmutableBitSet allNeededColumns = aggregationColumns.or(measureColumns);
-
-        GTRecord newRecord = new GTRecord(newGridTable.getInfo());
-        int count = 0;
-        try {
-            for (GTRecord record : scanner) {
-                count++;
-                for (int i = 0; i < allNeededColumns.trueBitCount(); i++) {
-                    int c = allNeededColumns.trueBitAt(i);
-                    newRecord.set(i, record.get(c));
-                }
-                builder.write(newRecord);
-            }
-
-            // disable sanity check for performance
-            sanityCheck(scanner.getTotalSumForSanityCheck());
-        } finally {
-            scanner.close();
-            builder.close();
-        }
-
-        long timeSpent = System.currentTimeMillis() - startTime;
-        logger.info("Cuboid " + cuboidId + " has " + count + " rows, build takes " + timeSpent + "ms");
-
-        return updateCuboidResult(cuboidId, newGridTable, count, timeSpent, 0);
-    }
-
-    //@SuppressWarnings("unused")
-    private void sanityCheck(Object[] totalSum) {
-        // double sum introduces error and causes result not exactly equal
-        for (int i = 0; i < totalSum.length; i++) {
-            if (totalSum[i] instanceof DoubleMutable) {
-                totalSum[i] = Math.round(((DoubleMutable) totalSum[i]).get());
-            }
-        }
-
-        if (totalSumForSanityCheck == null) {
-            totalSumForSanityCheck = totalSum;
-            return;
-        }
-        if (Arrays.equals(totalSumForSanityCheck, totalSum) == false) {
-            throw new IllegalStateException();
-        }
-    }
-
-    // ===========================================================================
-
-    private static class CuboidTask implements Comparable<CuboidTask> {
-        final CuboidResult parent;
-        final long childCuboidId;
-
-        CuboidTask(CuboidResult parent, long childCuboidId) {
-            this.parent = parent;
-            this.childCuboidId = childCuboidId;
-        }
-
-        @Override
-        public int compareTo(CuboidTask o) {
-            long comp = this.childCuboidId - o.childCuboidId;
-            return comp < 0 ? -1 : (comp > 0 ? 1 : 0);
-        }
-    }
-
-    // ============================================================================
-
-    private class InputConverter implements IGTScanner {
-        GTInfo info;
-        GTRecord record;
-        BlockingQueue<List<String>> input;
-
-        public InputConverter(GTInfo info, BlockingQueue<List<String>> input) {
-            this.info = info;
-            this.input = input;
-            this.record = new GTRecord(info);
-        }
-
-        @Override
-        public Iterator<GTRecord> iterator() {
-            return new Iterator<GTRecord>() {
-
-                List<String> currentObject = null;
-
-                @Override
-                public boolean hasNext() {
-                    try {
-                        currentObject = input.take();
-                    } catch (InterruptedException e) {
-                        throw new RuntimeException(e);
-                    }
-                    return currentObject != null && currentObject.size() > 0;
-                }
-
-                @Override
-                public GTRecord next() {
-                    if (currentObject.size() == 0)
-                        throw new IllegalStateException();
-
-                    buildGTRecord(currentObject, record);
-                    return record;
-                }
-
-                @Override
-                public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-            };
-        }
-
-        @Override
-        public void close() throws IOException {
-        }
-
-        @Override
-        public GTInfo getInfo() {
-            return info;
-        }
-
-        @Override
-        public int getScannedRowCount() {
-            return 0;
-        }
-
-        @Override
-        public int getScannedRowBlockCount() {
-            return 0;
-        }
-
-        private void buildGTRecord(List<String> row, GTRecord record) {
-            Object[] dimensions = buildKey(row);
-            Object[] metricsValues = buildValue(row);
-            Object[] recordValues = new Object[dimensions.length + metricsValues.length];
-            System.arraycopy(dimensions, 0, recordValues, 0, dimensions.length);
-            System.arraycopy(metricsValues, 0, recordValues, dimensions.length, metricsValues.length);
-            record.setValues(recordValues);
-        }
-
-        private Object[] buildKey(List<String> row) {
-            int keySize = intermediateTableDesc.getRowKeyColumnIndexes().length;
-            Object[] key = new Object[keySize];
-
-            for (int i = 0; i < keySize; i++) {
-                key[i] = row.get(intermediateTableDesc.getRowKeyColumnIndexes()[i]);
-            }
-
-            return key;
-        }
-
-        private Object[] buildValue(List<String> row) {
-
-            Object[] values = new Object[measureCount];
-            MeasureDesc measureDesc = null;
-
-            for (int position = 0; position < hbaseMeasureRefIndex.length; position++) {
-                int i = hbaseMeasureRefIndex[position];
-                measureDesc = measureDescs[i];
-
-                Object value = null;
-                int[] flatTableIdx = intermediateTableDesc.getMeasureColumnIndexes()[i];
-                FunctionDesc function = cubeDesc.getMeasures().get(i).getFunction();
-                if (function.isCount() || function.isHolisticCountDistinct()) {
-                    // note for holistic count distinct, this value will be ignored
-                    value = ONE;
-                } else if (flatTableIdx == null) {
-                    value = measureCodec.getSerializer(i).valueOf(measureDesc.getFunction().getParameter().getValue());
-                } else if (flatTableIdx.length == 1) {
-                    value = measureCodec.getSerializer(i).valueOf(toBytes(row.get(flatTableIdx[0])));
-                } else {
-
-                    byte[] result = null;
-                    for (int x = 0; x < flatTableIdx.length; x++) {
-                        byte[] split = toBytes(row.get(flatTableIdx[x]));
-                        if (result == null) {
-                            result = Arrays.copyOf(split, split.length);
-                        } else {
-                            byte[] newResult = new byte[result.length + split.length];
-                            System.arraycopy(result, 0, newResult, 0, result.length);
-                            System.arraycopy(split, 0, newResult, result.length, split.length);
-                            result = newResult;
-                        }
-                    }
-                    value = measureCodec.getSerializer(i).valueOf(result);
-                }
-                values[position] = value;
-            }
-            return values;
-        }
-
-        private byte[] toBytes(String v) {
-            return v == null ? null : Bytes.toBytes(v);
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/inmemcubing/MemDiskStore.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/inmemcubing/MemDiskStore.java b/job/src/main/java/org/apache/kylin/job/inmemcubing/MemDiskStore.java
deleted file mode 100644
index badb14f..0000000
--- a/job/src/main/java/org/apache/kylin/job/inmemcubing/MemDiskStore.java
+++ /dev/null
@@ -1,679 +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.inmemcubing;
-
-import static org.apache.kylin.common.util.MemoryBudgetController.*;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.Closeable;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.file.StandardOpenOption;
-import java.util.NoSuchElementException;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.common.util.MemoryBudgetController;
-import org.apache.kylin.common.util.MemoryBudgetController.MemoryConsumer;
-import org.apache.kylin.common.util.MemoryBudgetController.NotEnoughBudgetException;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTRowBlock;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.IGTStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MemDiskStore implements IGTStore, Closeable {
-
-    private static final Logger logger = LoggerFactory.getLogger(MemDiskStore.class);
-    private static final boolean debug = true;
-
-    private static final int STREAM_BUFFER_SIZE = 8192;
-    private static final int MEM_CHUNK_SIZE_MB = 5;
-
-    private final GTInfo info;
-    private final Object lock; // all public methods that read/write object states are synchronized on this lock
-    private final MemPart memPart;
-    private final DiskPart diskPart;
-    private final boolean delOnClose;
-
-    private Writer ongoingWriter;
-
-    public MemDiskStore(GTInfo info, MemoryBudgetController budgetCtrl) throws IOException {
-        this(info, budgetCtrl, File.createTempFile("MemDiskStore", ""), true);
-    }
-
-    public MemDiskStore(GTInfo info, MemoryBudgetController budgetCtrl, File diskFile) throws IOException {
-        this(info, budgetCtrl, diskFile, false);
-    }
-
-    private MemDiskStore(GTInfo info, MemoryBudgetController budgetCtrl, File diskFile, boolean delOnClose) throws IOException {
-        this.info = info;
-        this.lock = this;
-        this.memPart = new MemPart(budgetCtrl);
-        this.diskPart = new DiskPart(diskFile);
-        this.delOnClose = delOnClose;
-
-        // in case user forget to call close()
-        if (delOnClose)
-            diskFile.deleteOnExit();
-    }
-
-    @Override
-    public GTInfo getInfo() {
-        return info;
-    }
-
-    @Override
-    public IGTStoreWriter rebuild(int shard) throws IOException {
-        return newWriter(0);
-    }
-
-    @Override
-    public IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) throws IOException {
-        return newWriter(length());
-    }
-
-    private Writer newWriter(long startOffset) throws IOException {
-        synchronized (lock) {
-            if (ongoingWriter != null)
-                throw new IllegalStateException();
-
-            ongoingWriter = new Writer(startOffset);
-            return ongoingWriter;
-        }
-    }
-
-    @Override
-    public IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) throws IOException {
-        synchronized (lock) {
-            return new Reader();
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        // synchronized inside the parts close()
-        memPart.close();
-        diskPart.close();
-    }
-
-    public long length() {
-        synchronized (lock) {
-            return Math.max(memPart.tailOffset(), diskPart.tailOffset);
-        }
-    }
-
-    @Override
-    public String toString() {
-        return "MemDiskStore@" + (info.getTableName() == null ? this.hashCode() : info.getTableName());
-    }
-
-    private class Reader implements IGTStoreScanner {
-
-        final DataInputStream din;
-        long readOffset = 0;
-        long memRead = 0;
-        long diskRead = 0;
-        int nReadCalls = 0;
-
-        GTRowBlock block = GTRowBlock.allocate(info);
-        GTRowBlock next = null;
-
-        Reader() throws IOException {
-            diskPart.openRead();
-            if (debug)
-                logger.debug(MemDiskStore.this + " read start @ " + readOffset);
-
-            InputStream in = new InputStream() {
-                byte[] tmp = new byte[1];
-                MemChunk memChunk;
-
-                @Override
-                public int read() throws IOException {
-                    int n = read(tmp, 0, 1);
-                    if (n <= 0)
-                        return -1;
-                    else
-                        return (int) tmp[0];
-                }
-
-                @Override
-                public int read(byte[] b, int off, int len) throws IOException {
-                    synchronized (lock) {
-                        nReadCalls++;
-                        if (available() <= 0)
-                            return -1;
-
-                        if (memChunk == null && memPart.headOffset() <= readOffset && readOffset < memPart.tailOffset()) {
-                            memChunk = memPart.seekMemChunk(readOffset);
-                        }
-
-                        int lenToGo = Math.min(available(), len);
-
-                        int nRead = 0;
-                        while (lenToGo > 0) {
-                            int n;
-                            if (memChunk != null) {
-                                if (memChunk.headOffset() > readOffset) {
-                                    memChunk = null;
-                                    continue;
-                                }
-                                if (readOffset >= memChunk.tailOffset()) {
-                                    memChunk = memChunk.next;
-                                    continue;
-                                }
-                                int chunkOffset = (int) (readOffset - memChunk.headOffset());
-                                n = Math.min((int) (memChunk.tailOffset() - readOffset), lenToGo);
-                                System.arraycopy(memChunk.data, chunkOffset, b, off, n);
-                                memRead += n;
-                            } else {
-                                n = diskPart.read(readOffset, b, off, lenToGo);
-                                diskRead += n;
-                            }
-                            lenToGo -= n;
-                            nRead += n;
-                            off += n;
-                            readOffset += n;
-                        }
-                        return nRead;
-                    }
-                }
-
-                @Override
-                public int available() throws IOException {
-                    synchronized (lock) {
-                        return (int) (length() - readOffset);
-                    }
-                }
-            };
-
-            din = new DataInputStream(new BufferedInputStream(in, STREAM_BUFFER_SIZE));
-        }
-
-        @Override
-        public boolean hasNext() {
-            if (next != null)
-                return true;
-
-            try {
-                if (din.available() > 0) {
-                    block.importFrom(din);
-                    next = block;
-                }
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
-
-            return next != null;
-        }
-
-        @Override
-        public GTRowBlock next() {
-            if (next == null) {
-                hasNext();
-                if (next == null)
-                    throw new NoSuchElementException();
-            }
-            GTRowBlock r = next;
-            next = null;
-            return r;
-        }
-
-        @Override
-        public void remove() {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void close() throws IOException {
-            synchronized (lock) {
-                din.close();
-                diskPart.closeRead();
-                if (debug)
-                    logger.debug(MemDiskStore.this + " read end @ " + readOffset + ", " + (memRead) + " from mem, " + (diskRead) + " from disk, " + nReadCalls + " read() calls");
-            }
-        }
-
-    }
-
-    private class Writer implements IGTStoreWriter {
-
-        final DataOutputStream dout;
-        long writeOffset;
-        long memWrite = 0;
-        long diskWrite = 0;
-        int nWriteCalls;
-        boolean closed = false;
-
-        Writer(long startOffset) throws IOException {
-            writeOffset = 0; // TODO does not support append yet
-            memPart.clear();
-            diskPart.clear();
-            diskPart.openWrite(false);
-            if (debug)
-                logger.debug(MemDiskStore.this + " write start @ " + writeOffset);
-
-            memPart.activateMemWrite();
-
-            OutputStream out = new OutputStream() {
-                byte[] tmp = new byte[1];
-                boolean memPartActivated = true;
-
-                @Override
-                public void write(int b) throws IOException {
-                    tmp[0] = (byte) b;
-                    write(tmp, 0, 1);
-                }
-
-                @Override
-                public void write(byte[] bytes, int offset, int length) throws IOException {
-                    // lock inside memPart.write() and diskPartm.write()
-                    nWriteCalls++;
-                    while (length > 0) {
-                        int n;
-                        if (memPartActivated) {
-                            n = memPart.write(bytes, offset, length, writeOffset);
-                            memWrite += n;
-                            if (n == 0) {
-                                memPartActivated = false;
-                            }
-                        } else {
-                            n = diskPart.write(writeOffset, bytes, offset, length);
-                            diskWrite += n;
-                        }
-                        offset += n;
-                        length -= n;
-                        writeOffset += n;
-                    }
-                }
-            };
-            dout = new DataOutputStream(new BufferedOutputStream(out, STREAM_BUFFER_SIZE));
-        }
-
-        @Override
-        public void write(GTRowBlock block) throws IOException {
-            block.export(dout);
-        }
-
-        @Override
-        public void close() throws IOException {
-            synchronized (lock) {
-                if (!closed) {
-                    dout.close();
-                    memPart.deactivateMemWrite();
-                }
-
-                if (memPart.asyncFlusher == null) {
-                    assert writeOffset == diskPart.tailOffset;
-                    diskPart.closeWrite();
-                    ongoingWriter = null;
-                    if (debug)
-                        logger.debug(MemDiskStore.this + " write end @ " + writeOffset + ", " + (memWrite) + " to mem, " + (diskWrite) + " to disk, " + nWriteCalls + " write() calls");
-                } else {
-                    // the asyncFlusher will call this close() again later
-                }
-                closed = true;
-            }
-        }
-    }
-
-    private static class MemChunk {
-        long diskOffset;
-        int length;
-        byte[] data;
-        MemChunk next;
-
-        boolean isFull() {
-            return length == data.length;
-        }
-
-        long headOffset() {
-            return diskOffset;
-        }
-
-        long tailOffset() {
-            return diskOffset + length;
-        }
-
-        int freeSpace() {
-            return data.length - length;
-        }
-    }
-
-    private class MemPart implements Closeable, MemoryConsumer {
-
-        final MemoryBudgetController budgetCtrl;
-
-        // async flush thread checks this flag out of sync block
-        volatile boolean writeActivated;
-        MemChunk firstChunk;
-        MemChunk lastChunk;
-        int chunkCount;
-
-        Thread asyncFlusher;
-        MemChunk asyncFlushChunk;
-        long asyncFlushDiskOffset;
-        Throwable asyncFlushException;
-
-        MemPart(MemoryBudgetController budgetCtrl) {
-            this.budgetCtrl = budgetCtrl;
-        }
-
-        long headOffset() {
-            return firstChunk == null ? 0 : firstChunk.headOffset();
-        }
-
-        long tailOffset() {
-            return lastChunk == null ? 0 : lastChunk.tailOffset();
-        }
-
-        public MemChunk seekMemChunk(long diskOffset) {
-            MemChunk c = firstChunk;
-            while (c != null && c.headOffset() <= diskOffset) {
-                if (diskOffset < c.tailOffset())
-                    break;
-                c = c.next;
-            }
-            return c;
-        }
-
-        public int write(byte[] bytes, int offset, int length, long diskOffset) {
-            int needMoreMem = 0;
-
-            synchronized (lock) {
-                if (writeActivated == false)
-                    return 0;
-
-                // write is only expected at the tail
-                if (diskOffset != tailOffset())
-                    return 0;
-
-                if (chunkCount == 0 || lastChunk.isFull())
-                    needMoreMem = (chunkCount + 1) * MEM_CHUNK_SIZE_MB;
-            }
-
-            // call to budgetCtrl.reserve() must be out of synchronized block, or deadlock may happen between MemoryConsumers
-            if (needMoreMem > 0) {
-                try {
-                    budgetCtrl.reserve(this, needMoreMem);
-                } catch (NotEnoughBudgetException ex) {
-                    deactivateMemWrite();
-                    return 0;
-                }
-            }
-
-            synchronized (lock) {
-                if (needMoreMem > 0 && (chunkCount == 0 || lastChunk.isFull())) {
-                    MemChunk chunk = new MemChunk();
-                    chunk.diskOffset = diskOffset;
-                    chunk.data = new byte[ONE_MB * MEM_CHUNK_SIZE_MB - 48]; // -48 for MemChunk overhead
-                    if (chunkCount == 0) {
-                        firstChunk = lastChunk = chunk;
-                    } else {
-                        lastChunk.next = chunk;
-                        lastChunk = chunk;
-                    }
-                    chunkCount++;
-                }
-
-                int n = Math.min(lastChunk.freeSpace(), length);
-                System.arraycopy(bytes, offset, lastChunk.data, lastChunk.length, n);
-                lastChunk.length += n;
-
-                if (n > 0)
-                    asyncFlush(lastChunk, diskOffset, n);
-
-                return n;
-            }
-        }
-
-        private void asyncFlush(MemChunk lastChunk, long diskOffset, int n) {
-            if (asyncFlushChunk == null) {
-                asyncFlushChunk = lastChunk;
-                asyncFlushDiskOffset = diskOffset;
-            }
-
-            if (asyncFlusher == null) {
-                asyncFlusher = new Thread() {
-                    public void run() {
-                        asyncFlushException = null;
-                        if (debug)
-                            logger.debug(MemDiskStore.this + " async flush started @ " + asyncFlushDiskOffset);
-                        try {
-                            while (writeActivated) {
-                                flushToDisk();
-                                Thread.sleep(10);
-                            }
-                            flushToDisk();
-
-                            if (debug)
-                                logger.debug(MemDiskStore.this + " async flush ended @ " + asyncFlushDiskOffset);
-
-                            synchronized (lock) {
-                                asyncFlusher = null;
-                                asyncFlushChunk = null;
-                                if (ongoingWriter.closed) {
-                                    ongoingWriter.close(); // call writer.close() again to clean up
-                                }
-                            }
-                        } catch (Throwable ex) {
-                            asyncFlushException = ex;
-                        }
-                    }
-                };
-                asyncFlusher.start();
-            }
-        }
-
-        private void flushToDisk() throws IOException {
-            byte[] data;
-            int offset = 0;
-            int length = 0;
-            int flushedLen = 0;
-
-            while (true) {
-                data = null;
-                synchronized (lock) {
-                    asyncFlushDiskOffset += flushedLen; // bytes written in last loop
-                    //                    if (debug)
-                    //                        logger.debug(GTMemDiskStore.this + " async flush @ " + asyncFlushDiskOffset);
-                    if (asyncFlushChunk != null && asyncFlushChunk.tailOffset() == asyncFlushDiskOffset) {
-                        asyncFlushChunk = asyncFlushChunk.next;
-                    }
-                    if (asyncFlushChunk != null) {
-                        data = asyncFlushChunk.data;
-                        offset = (int) (asyncFlushDiskOffset - asyncFlushChunk.headOffset());
-                        length = asyncFlushChunk.length - offset;
-                    }
-                }
-
-                if (data == null)
-                    break;
-
-                flushedLen = diskPart.write(asyncFlushDiskOffset, data, offset, length);
-            }
-        }
-
-        @Override
-        public int freeUp(int mb) {
-            synchronized (lock) {
-                int mbReleased = 0;
-                while (chunkCount > 0 && mbReleased < mb) {
-                    if (firstChunk == asyncFlushChunk)
-                        break;
-
-                    mbReleased += MEM_CHUNK_SIZE_MB;
-                    chunkCount--;
-                    if (chunkCount == 0) {
-                        firstChunk = lastChunk = null;
-                    } else {
-                        MemChunk next = firstChunk.next;
-                        firstChunk.next = null;
-                        firstChunk = next;
-                    }
-                }
-                return mbReleased;
-            }
-        }
-
-        public void activateMemWrite() {
-            if (budgetCtrl.getTotalBudgetMB() > 0) {
-                writeActivated = true;
-                if (debug)
-                    logger.debug(MemDiskStore.this + " mem write activated");
-            }
-        }
-
-        public void deactivateMemWrite() {
-            writeActivated = false;
-            if (debug)
-                logger.debug(MemDiskStore.this + " mem write de-activated");
-        }
-
-        public void clear() {
-            chunkCount = 0;
-            firstChunk = lastChunk = null;
-            budgetCtrl.reserve(this, 0);
-        }
-
-        @Override
-        public void close() throws IOException {
-            synchronized (lock) {
-                if (asyncFlushException != null)
-                    throwAsyncException(asyncFlushException);
-            }
-            try {
-                asyncFlusher.join();
-            } catch (NullPointerException npe) {
-                // that's fine, async flusher may not present
-            } catch (InterruptedException e) {
-                logger.warn("async join interrupted", e);
-            }
-            synchronized (lock) {
-                if (asyncFlushException != null)
-                    throwAsyncException(asyncFlushException);
-
-                clear();
-            }
-        }
-
-        private void throwAsyncException(Throwable ex) throws IOException {
-            if (ex instanceof IOException)
-                throw (IOException) ex;
-            else
-                throw new IOException(ex);
-        }
-
-        @Override
-        public String toString() {
-            return MemDiskStore.this.toString();
-        }
-
-    }
-
-    private class DiskPart implements Closeable {
-        final File diskFile;
-        FileChannel writeChannel;
-        FileChannel readChannel;
-        int readerCount = 0; // allow parallel readers
-        long tailOffset;
-
-        DiskPart(File diskFile) throws IOException {
-            this.diskFile = diskFile;
-            this.tailOffset = diskFile.length();
-            if (debug)
-                logger.debug(MemDiskStore.this + " disk file " + diskFile.getAbsolutePath());
-        }
-
-        public void openRead() throws IOException {
-            if (readChannel == null) {
-                readChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.READ);
-            }
-            readerCount++;
-        }
-
-        public int read(long diskOffset, byte[] bytes, int offset, int length) throws IOException {
-            return readChannel.read(ByteBuffer.wrap(bytes, offset, length), diskOffset);
-        }
-
-        public void closeRead() throws IOException {
-            closeRead(false);
-        }
-
-        private void closeRead(boolean force) throws IOException {
-            readerCount--;
-            if (readerCount == 0 || force) {
-                if (readChannel != null) {
-                    readChannel.close();
-                    readChannel = null;
-                }
-            }
-        }
-
-        public void openWrite(boolean append) throws IOException {
-            if (append) {
-                writeChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.CREATE, StandardOpenOption.APPEND, StandardOpenOption.WRITE);
-                tailOffset = diskFile.length();
-            } else {
-                diskFile.delete();
-                writeChannel = FileChannel.open(diskFile.toPath(), StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
-                tailOffset = 0;
-            }
-        }
-
-        public int write(long diskOffset, byte[] bytes, int offset, int length) throws IOException {
-            synchronized (lock) {
-                int n = writeChannel.write(ByteBuffer.wrap(bytes, offset, length), diskOffset);
-                tailOffset = Math.max(diskOffset + n, tailOffset);
-                return n;
-            }
-        }
-
-        public void closeWrite() throws IOException {
-            if (writeChannel != null) {
-                writeChannel.close();
-                writeChannel = null;
-            }
-        }
-
-        public void clear() throws IOException {
-            diskFile.delete();
-            tailOffset = 0;
-        }
-
-        @Override
-        public void close() throws IOException {
-            synchronized (lock) {
-                closeWrite();
-                closeRead(true);
-                if (delOnClose) {
-                    diskFile.delete();
-                }
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java b/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
index cc68e1b..2ef3d94 100644
--- a/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
@@ -23,10 +23,10 @@ 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.common.HadoopShellExecutable;
-import org.apache.kylin.job.common.MapReduceExecutable;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.execution.AbstractExecutable;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java b/job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
deleted file mode 100644
index 05f8c8e..0000000
--- a/job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
+++ /dev/null
@@ -1,318 +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.manager;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.job.dao.ExecutableDao;
-import org.apache.kylin.job.dao.ExecutableOutputPO;
-import org.apache.kylin.job.dao.ExecutablePO;
-import org.apache.kylin.job.exception.IllegalStateTranferException;
-import org.apache.kylin.job.exception.PersistentException;
-import org.apache.kylin.job.execution.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-import java.lang.reflect.Constructor;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- */
-public class ExecutableManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(ExecutableManager.class);
-    private static final ConcurrentHashMap<KylinConfig, ExecutableManager> CACHE = new ConcurrentHashMap<KylinConfig, ExecutableManager>();
-    @SuppressWarnings("unused")
-    private final KylinConfig config;
-
-    private ExecutableDao executableDao;
-
-    public static ExecutableManager getInstance(KylinConfig config) {
-        ExecutableManager r = CACHE.get(config);
-        if (r == null) {
-            r = new ExecutableManager(config);
-            CACHE.put(config, r);
-            if (CACHE.size() > 1) {
-                logger.warn("More than one singleton exist");
-            }
-
-        }
-        return r;
-    }
-
-    private ExecutableManager(KylinConfig config) {
-        logger.info("Using metadata url: " + config);
-        this.config = config;
-        this.executableDao = ExecutableDao.getInstance(config);
-    }
-
-    public void addJob(AbstractExecutable executable) {
-        try {
-            executableDao.addJob(parse(executable));
-            addJobOutput(executable);
-        } catch (PersistentException e) {
-            logger.error("fail to submit job:" + executable.getId(), e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    private void addJobOutput(AbstractExecutable executable) throws PersistentException {
-        ExecutableOutputPO executableOutputPO = new ExecutableOutputPO();
-        executableOutputPO.setUuid(executable.getId());
-        executableDao.addJobOutput(executableOutputPO);
-        if (executable instanceof DefaultChainedExecutable) {
-            for (AbstractExecutable subTask: ((DefaultChainedExecutable) executable).getTasks()) {
-                addJobOutput(subTask);
-            }
-        }
-    }
-
-    //for ut
-    public void deleteJob(String jobId) {
-        try {
-            executableDao.deleteJob(jobId);
-        } catch (PersistentException e) {
-            logger.error("fail to delete job:" + jobId, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    public AbstractExecutable getJob(String uuid) {
-        try {
-            return parseTo(executableDao.getJob(uuid));
-        } catch (PersistentException e) {
-            logger.error("fail to get job:" + uuid, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    public Output getOutput(String uuid) {
-        try {
-            final ExecutableOutputPO jobOutput = executableDao.getJobOutput(uuid);
-            Preconditions.checkArgument(jobOutput != null, "there is no related output for job id:" + uuid);
-            return parseOutput(jobOutput);
-        } catch (PersistentException e) {
-            logger.error("fail to get job output:" + uuid, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    private DefaultOutput parseOutput(ExecutableOutputPO jobOutput) {
-        final DefaultOutput result = new DefaultOutput();
-        result.setExtra(jobOutput.getInfo());
-        result.setState(ExecutableState.valueOf(jobOutput.getStatus()));
-        result.setVerboseMsg(jobOutput.getContent());
-        result.setLastModified(jobOutput.getLastModified());
-        return result;
-    }
-
-    public Map<String, Output> getAllOutputs() {
-        try {
-            final List<ExecutableOutputPO> jobOutputs = executableDao.getJobOutputs();
-            HashMap<String, Output> result = Maps.newHashMap();
-            for (ExecutableOutputPO jobOutput : jobOutputs) {
-                result.put(jobOutput.getId(), parseOutput(jobOutput));
-            }
-            return result;
-        } catch (PersistentException e) {
-            logger.error("fail to get all job output:", e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    public List<AbstractExecutable> getAllExecutables() {
-        try {
-            return Lists.transform(executableDao.getJobs(), new Function<ExecutablePO, AbstractExecutable>() {
-                @Nullable
-                @Override
-                public AbstractExecutable apply(ExecutablePO input) {
-                        return parseTo(input);
-                }
-            });
-        } catch (PersistentException e) {
-            logger.error("error get All Jobs", e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    public List<String> getAllJobIds() {
-        try {
-            return executableDao.getJobIds();
-        } catch (PersistentException e) {
-            logger.error("error get All Job Ids", e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void updateAllRunningJobsToError() {
-        try {
-            final List<ExecutableOutputPO> jobOutputs = executableDao.getJobOutputs();
-            for (ExecutableOutputPO executableOutputPO : jobOutputs) {
-                if (executableOutputPO.getStatus().equalsIgnoreCase(ExecutableState.RUNNING.toString())) {
-                    executableOutputPO.setStatus(ExecutableState.ERROR.toString());
-                    executableDao.updateJobOutput(executableOutputPO);
-                }
-            }
-        } catch (PersistentException e) {
-            logger.error("error reset job status from RUNNING to ERROR", e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void resumeJob(String jobId) {
-        AbstractExecutable job = getJob(jobId);
-        if (job == null) {
-            return;
-        }
-        updateJobOutput(jobId, ExecutableState.READY, null, null);
-        if (job instanceof DefaultChainedExecutable) {
-            List<AbstractExecutable> tasks = ((DefaultChainedExecutable) job).getTasks();
-            for (AbstractExecutable task : tasks) {
-                if (task.getStatus() == ExecutableState.ERROR) {
-                    updateJobOutput(task.getId(), ExecutableState.READY, null, null);
-                    break;
-                }
-            }
-        }
-    }
-
-    public void discardJob(String jobId) {
-        AbstractExecutable job = getJob(jobId);
-        if (job instanceof DefaultChainedExecutable) {
-            List<AbstractExecutable> tasks = ((DefaultChainedExecutable) job).getTasks();
-            for (AbstractExecutable task : tasks) {
-                if (!task.getStatus().isFinalState()) {
-                    updateJobOutput(task.getId(), ExecutableState.DISCARDED, null, null);
-                }
-            }
-        }
-        updateJobOutput(jobId, ExecutableState.DISCARDED, null, null);
-    }
-
-    public void updateJobOutput(String jobId, ExecutableState newStatus, Map<String, String> info, String output) {
-        try {
-            final ExecutableOutputPO jobOutput = executableDao.getJobOutput(jobId);
-            Preconditions.checkArgument(jobOutput != null, "there is no related output for job id:" + jobId);
-            ExecutableState oldStatus = ExecutableState.valueOf(jobOutput.getStatus());
-            if (newStatus != null && oldStatus != newStatus) {
-                if (!ExecutableState.isValidStateTransfer(oldStatus, newStatus)) {
-                    throw new IllegalStateTranferException("there is no valid state transfer from:" + oldStatus + " to:" + newStatus);
-                }
-                jobOutput.setStatus(newStatus.toString());
-            }
-            if (info != null) {
-                jobOutput.setInfo(info);
-            }
-            if (output != null) {
-                jobOutput.setContent(output);
-            }
-            executableDao.updateJobOutput(jobOutput);
-            logger.info("job id:" + jobId + " from " + oldStatus + " to " + newStatus);
-        } catch (PersistentException e) {
-            logger.error("error change job:" + jobId + " to " + newStatus.toString());
-            throw new RuntimeException(e);
-        }
-    }
-
-    //for migration only
-    //TODO delete when migration finished
-    public void resetJobOutput(String jobId, ExecutableState state, String output) {
-        try {
-            final ExecutableOutputPO jobOutput = executableDao.getJobOutput(jobId);
-            jobOutput.setStatus(state.toString());
-            if (output != null) {
-                jobOutput.setContent(output);
-            }
-            executableDao.updateJobOutput(jobOutput);
-        } catch (PersistentException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void addJobInfo(String id, Map<String, String> info) {
-        if (info == null) {
-            return;
-        }
-        try {
-            ExecutableOutputPO output = executableDao.getJobOutput(id);
-            Preconditions.checkArgument(output != null, "there is no related output for job id:" + id);
-            output.getInfo().putAll(info);
-            executableDao.updateJobOutput(output);
-        } catch (PersistentException e) {
-            logger.error("error update job info, id:" + id + "  info:" + info.toString());
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void addJobInfo(String id, String key, String value) {
-        Map<String, String> info = Maps.newHashMap();
-        info.put(key, value);
-        addJobInfo(id, info);
-    }
-
-    private static ExecutablePO parse(AbstractExecutable executable) {
-        ExecutablePO result = new ExecutablePO();
-        result.setName(executable.getName());
-        result.setUuid(executable.getId());
-        result.setType(executable.getClass().getName());
-        result.setParams(executable.getParams());
-        if (executable instanceof ChainedExecutable) {
-            List<ExecutablePO> tasks = Lists.newArrayList();
-            for (AbstractExecutable task : ((ChainedExecutable) executable).getTasks()) {
-                tasks.add(parse(task));
-            }
-            result.setTasks(tasks);
-        }
-        return result;
-    }
-
-    private static AbstractExecutable parseTo(ExecutablePO executablePO) {
-        if (executablePO == null) {
-            return null;
-        }
-        String type = executablePO.getType();
-        try {
-            Class<? extends AbstractExecutable> clazz = ClassUtil.forName(type, AbstractExecutable.class);
-            Constructor<? extends AbstractExecutable> constructor = clazz.getConstructor();
-            AbstractExecutable result = constructor.newInstance();
-            result.setId(executablePO.getUuid());
-            result.setName(executablePO.getName());
-            result.setParams(executablePO.getParams());
-            List<ExecutablePO> tasks = executablePO.getTasks();
-            if (tasks != null && !tasks.isEmpty()) {
-                Preconditions.checkArgument(result instanceof ChainedExecutable);
-                for (ExecutablePO subTask: tasks) {
-                    ((ChainedExecutable) result).addTask(parseTo(subTask));
-                }
-            }
-            return result;
-        } catch (ReflectiveOperationException e) {
-            throw new IllegalArgumentException("cannot parse this job:" + executablePO.getId(), e);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java b/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java
index 5fc445c..36feb15 100644
--- a/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java
+++ b/job/src/main/java/org/apache/kylin/job/streaming/CubeStreamConsumer.java
@@ -33,6 +33,9 @@ import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.gridtable.CuboidToGridTableMapping;
+import org.apache.kylin.cube.inmemcubing.ICuboidWriter;
+import org.apache.kylin.cube.inmemcubing.InMemCubeBuilder;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
@@ -42,18 +45,15 @@ import org.apache.kylin.dict.DictionaryGenerator;
 import org.apache.kylin.dict.DictionaryInfo;
 import org.apache.kylin.dict.DictionaryManager;
 import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.cube.FactDistinctColumnsReducer;
-import org.apache.kylin.job.hadoop.hbase.CubeHTableUtil;
-import org.apache.kylin.job.inmemcubing.ICuboidWriter;
-import org.apache.kylin.job.inmemcubing.InMemCubeBuilder;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.steps.FactDistinctColumnsReducer;
+import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.source.ReadableTable.TableSignature;
-import org.apache.kylin.storage.cube.CuboidToGridTableMapping;
-import org.apache.kylin.storage.gridtable.GTRecord;
 import org.apache.kylin.storage.hbase.HBaseConnection;
-import org.apache.kylin.storage.hbase.InMemKeyValueCreator;
+import org.apache.kylin.storage.hbase.steps.CubeHTableUtil;
+import org.apache.kylin.storage.hbase.steps.InMemKeyValueCreator;
 import org.apache.kylin.streaming.MicroStreamBatch;
 import org.apache.kylin.streaming.MicroStreamBatchConsumer;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java b/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
index e1ebe20..47ed52e 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
@@ -28,8 +28,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/tools/DefaultSslProtocolSocketFactory.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/DefaultSslProtocolSocketFactory.java b/job/src/main/java/org/apache/kylin/job/tools/DefaultSslProtocolSocketFactory.java
deleted file mode 100644
index 89713c2..0000000
--- a/job/src/main/java/org/apache/kylin/job/tools/DefaultSslProtocolSocketFactory.java
+++ /dev/null
@@ -1,150 +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.net.InetAddress;
-import java.net.Socket;
-import java.net.UnknownHostException;
-
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManager;
-
-import org.apache.commons.httpclient.ConnectTimeoutException;
-import org.apache.commons.httpclient.HttpClientError;
-import org.apache.commons.httpclient.params.HttpConnectionParams;
-import org.apache.commons.httpclient.protocol.ControllerThreadSocketFactory;
-import org.apache.commons.httpclient.protocol.SecureProtocolSocketFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author xduo
- * 
- */
-public class DefaultSslProtocolSocketFactory implements SecureProtocolSocketFactory {
-    /** Log object for this class. */
-    private static Logger LOG = LoggerFactory.getLogger(DefaultSslProtocolSocketFactory.class);
-    private SSLContext sslcontext = null;
-
-    /**
-     * Constructor for DefaultSslProtocolSocketFactory.
-     */
-    public DefaultSslProtocolSocketFactory() {
-        super();
-    }
-
-    /**
-     * @see SecureProtocolSocketFactory#createSocket(java.lang.String,int,java.net.InetAddress,int)
-     */
-    public Socket createSocket(String host, int port, InetAddress clientHost, int clientPort) throws IOException, UnknownHostException {
-        return getSSLContext().getSocketFactory().createSocket(host, port, clientHost, clientPort);
-    }
-
-    /**
-     * Attempts to get a new socket connection to the given host within the
-     * given time limit.
-     * 
-     * <p>
-     * To circumvent the limitations of older JREs that do not support connect
-     * timeout a controller thread is executed. The controller thread attempts
-     * to create a new socket within the given limit of time. If socket
-     * constructor does not return until the timeout expires, the controller
-     * terminates and throws an {@link ConnectTimeoutException}
-     * </p>
-     * 
-     * @param host
-     *            the host name/IP
-     * @param port
-     *            the port on the host
-     * @param localAddress
-     *            the local host name/IP to bind the socket to
-     * @param localPort
-     *            the port on the local machine
-     * @param params
-     *            {@link HttpConnectionParams Http connection parameters}
-     * 
-     * @return Socket a new socket
-     * 
-     * @throws IOException
-     *             if an I/O error occurs while creating the socket
-     * @throws UnknownHostException
-     *             if the IP address of the host cannot be determined
-     * @throws ConnectTimeoutException
-     *             DOCUMENT ME!
-     * @throws IllegalArgumentException
-     *             DOCUMENT ME!
-     */
-    public Socket createSocket(final String host, final int port, final InetAddress localAddress, final int localPort, final HttpConnectionParams params) throws IOException, UnknownHostException, ConnectTimeoutException {
-        if (params == null) {
-            throw new IllegalArgumentException("Parameters may not be null");
-        }
-
-        int timeout = params.getConnectionTimeout();
-
-        if (timeout == 0) {
-            return createSocket(host, port, localAddress, localPort);
-        } else {
-            // To be eventually deprecated when migrated to Java 1.4 or above
-            return ControllerThreadSocketFactory.createSocket(this, host, port, localAddress, localPort, timeout);
-        }
-    }
-
-    /**
-     * @see SecureProtocolSocketFactory#createSocket(java.lang.String,int)
-     */
-    public Socket createSocket(String host, int port) throws IOException, UnknownHostException {
-        return getSSLContext().getSocketFactory().createSocket(host, port);
-    }
-
-    /**
-     * @see SecureProtocolSocketFactory#createSocket(java.net.Socket,java.lang.String,int,boolean)
-     */
-    public Socket createSocket(Socket socket, String host, int port, boolean autoClose) throws IOException, UnknownHostException {
-        return getSSLContext().getSocketFactory().createSocket(socket, host, port, autoClose);
-    }
-
-    public boolean equals(Object obj) {
-        return ((obj != null) && obj.getClass().equals(DefaultX509TrustManager.class));
-    }
-
-    public int hashCode() {
-        return DefaultX509TrustManager.class.hashCode();
-    }
-
-    private static SSLContext createEasySSLContext() {
-        try {
-            SSLContext context = SSLContext.getInstance("TLS");
-            context.init(null, new TrustManager[] { new DefaultX509TrustManager(null) }, null);
-
-            return context;
-        } catch (Exception e) {
-            LOG.error(e.getMessage(), e);
-            throw new HttpClientError(e.toString());
-        }
-    }
-
-    private SSLContext getSSLContext() {
-        if (this.sslcontext == null) {
-            this.sslcontext = createEasySSLContext();
-        }
-
-        return this.sslcontext;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/tools/DefaultX509TrustManager.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/DefaultX509TrustManager.java b/job/src/main/java/org/apache/kylin/job/tools/DefaultX509TrustManager.java
deleted file mode 100644
index 8fc2dcd..0000000
--- a/job/src/main/java/org/apache/kylin/job/tools/DefaultX509TrustManager.java
+++ /dev/null
@@ -1,114 +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.security.KeyStore;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.cert.CertificateException;
-import java.security.cert.X509Certificate;
-
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.TrustManager;
-import javax.net.ssl.TrustManagerFactory;
-import javax.net.ssl.X509TrustManager;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author xduo
- * 
- */
-public class DefaultX509TrustManager implements X509TrustManager {
-
-    /** Log object for this class. */
-    private static Logger LOG = LoggerFactory.getLogger(DefaultX509TrustManager.class);
-    private X509TrustManager standardTrustManager = null;
-
-    /**
-     * Constructor for DefaultX509TrustManager.
-     * 
-     */
-    public DefaultX509TrustManager(KeyStore keystore) throws NoSuchAlgorithmException, KeyStoreException {
-        super();
-
-        TrustManagerFactory factory = TrustManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-        factory.init(keystore);
-
-        TrustManager[] trustmanagers = factory.getTrustManagers();
-
-        if (trustmanagers.length == 0) {
-            throw new NoSuchAlgorithmException("SunX509 trust manager not supported");
-        }
-
-        this.standardTrustManager = (X509TrustManager) trustmanagers[0];
-    }
-
-    public X509Certificate[] getAcceptedIssuers() {
-        return this.standardTrustManager.getAcceptedIssuers();
-    }
-
-    public boolean isClientTrusted(X509Certificate[] certificates) {
-        return true;
-        // return this.standardTrustManager.isClientTrusted(certificates);
-    }
-
-    public boolean isServerTrusted(X509Certificate[] certificates) {
-        if ((certificates != null) && LOG.isDebugEnabled()) {
-            LOG.debug("Server certificate chain:");
-
-            for (int i = 0; i < certificates.length; i++) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("X509Certificate[" + i + "]=" + certificates[i]);
-                }
-            }
-        }
-
-        if ((certificates != null) && (certificates.length == 1)) {
-            X509Certificate certificate = certificates[0];
-
-            try {
-                certificate.checkValidity();
-            } catch (CertificateException e) {
-                LOG.error(e.toString());
-
-                return false;
-            }
-
-            return true;
-        } else {
-            return true;
-            // return this.standardTrustManager.isServerTrusted(certificates);
-        }
-    }
-
-    @Override
-    public void checkClientTrusted(X509Certificate[] chain, String authType) throws CertificateException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void checkServerTrusted(X509Certificate[] chain, String authType) throws CertificateException {
-        // TODO Auto-generated method stub
-
-    }
-
-}


[16/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeSamplingTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeSamplingTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeSamplingTest.java
new file mode 100644
index 0000000..e3397c0
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeSamplingTest.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import com.google.common.collect.Lists;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+/**
+ */
+public class CubeSamplingTest {
+
+    private static final int ROW_LENGTH = 10;
+
+    private final List<String> row = new ArrayList<String>(ROW_LENGTH);
+    private final ByteArray[] row_index = new ByteArray[ROW_LENGTH];
+
+    private Integer[][] allCuboidsBitSet;
+    private HashFunction hf = null;
+    private long baseCuboidId;
+    private HyperLogLogPlusCounter[] allCuboidsHLL = null;
+    private final byte[] seperator = Bytes.toBytes(",");
+
+    @Before
+    public void setup() {
+
+        baseCuboidId = (1l << ROW_LENGTH) - 1;
+        List<Long> allCuboids = Lists.newArrayList();
+        List<Integer[]> allCuboidsBitSetList = Lists.newArrayList();
+        for (long i = 1; i < baseCuboidId; i++) {
+            allCuboids.add(i);
+            addCuboidBitSet(i, allCuboidsBitSetList);
+        }
+
+        allCuboidsBitSet = allCuboidsBitSetList.toArray(new Integer[allCuboidsBitSetList.size()][]);
+        System.out.println("Totally have " + allCuboidsBitSet.length + " cuboids.");
+        allCuboidsHLL = new HyperLogLogPlusCounter[allCuboids.size()];
+        for (int i = 0; i < allCuboids.size(); i++) {
+            allCuboidsHLL[i] = new HyperLogLogPlusCounter(14);
+        }
+
+        //  hf = Hashing.goodFastHash(32);
+//        hf = Hashing.md5();
+        hf = Hashing.murmur3_32();
+
+        for (int i = 0; i < ROW_LENGTH; i++) {
+            row_index[i] = new ByteArray();
+        }
+    }
+
+    private void addCuboidBitSet(long cuboidId, List<Integer[]> allCuboidsBitSet) {
+        BitSet bitSet = BitSet.valueOf(new long[]{cuboidId});
+        Integer[] indice = new Integer[bitSet.cardinality()];
+
+        long mask = Long.highestOneBit(baseCuboidId);
+        int position = 0;
+        for (int i = 0; i < ROW_LENGTH; i++) {
+            if ((mask & cuboidId) > 0) {
+                indice[position] = i;
+                position++;
+            }
+            mask = mask >> 1;
+        }
+
+        allCuboidsBitSet.add(indice);
+
+    }
+
+    @Test
+    public void test() {
+
+        long start = System.currentTimeMillis();
+        List<String> row;
+        for (int i = 0; i < 10000; i++) {
+            row = getRandomRow();
+            putRowKeyToHLL(row);
+        }
+
+        long duration = System.currentTimeMillis() - start;
+        System.out.println("The test takes " + duration / 1000 + "seconds.");
+    }
+
+    private void putRowKeyToHLL(List<String> row) {
+        int x = 0;
+        for (String field : row) {
+            Hasher hc = hf.newHasher();
+            row_index[x++].set(hc.putString(field).hash().asBytes());
+        }
+
+        for (int i = 0, n = allCuboidsBitSet.length; i < n; i++) {
+            Hasher hc = hf.newHasher();
+            for (int position = 0; position < allCuboidsBitSet[i].length; position++) {
+                hc.putBytes(row_index[allCuboidsBitSet[i][position]].array());
+                hc.putBytes(seperator);
+            }
+            allCuboidsHLL[i].add(hc.hash().asBytes());
+        }
+    }
+
+    private List<String> getRandomRow() {
+        row.clear();
+        for (int i = 0; i < ROW_LENGTH; i++) {
+            row.add(RandomStringUtils.random(10));
+        }
+        return row;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducerTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducerTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducerTest.java
new file mode 100644
index 0000000..ae75c61
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducerTest.java
@@ -0,0 +1,37 @@
+package org.apache.kylin.engine.mr.steps;
+
+import com.google.common.collect.Maps;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.engine.mr.steps.FactDistinctColumnsReducer;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ */
+public class FactDistinctColumnsReducerTest {
+
+
+    @Test
+    public void testWriteCuboidStatistics() throws IOException {
+
+        final Configuration conf = HadoopUtil.getCurrentConfiguration();
+        final Path outputPath = new Path("file:///tmp/kylin/cuboidstatistics/" + UUID.randomUUID().toString());
+        if (!FileSystem.getLocal(conf).exists(outputPath)) {
+//            FileSystem.getLocal(conf).create(outputPath);
+        }
+
+        System.out.println(outputPath);
+        Map<Long, HyperLogLogPlusCounter> cuboidHLLMap = Maps.newHashMap();
+        FactDistinctColumnsReducer.writeCuboidStatistics(conf, outputPath, cuboidHLLMap, 100);
+
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapperPerformanceTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapperPerformanceTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapperPerformanceTest.java
new file mode 100644
index 0000000..fad521f
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapperPerformanceTest.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class HiveToBaseCuboidMapperPerformanceTest {
+
+    String metadataUrl = "hbase:yadesk00:2181:/hbase-unsecure";
+    String cubeName = "test_kylin_cube_with_slr";
+    Path srcPath = new Path("/download/test_kylin_cube_with_slr_intermediate_table_64mb.seq");
+
+    @Ignore("convenient trial tool for dev")
+    @Test
+    public void test() throws IOException, InterruptedException {
+        Configuration hconf = new Configuration();
+        HiveToBaseCuboidMapper mapper = new HiveToBaseCuboidMapper();
+        Context context = MockupMapContext.create(hconf, metadataUrl, cubeName, null);
+
+        mapper.setup(context);
+
+        Reader reader = new Reader(hconf, SequenceFile.Reader.file(srcPath));
+        Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), hconf);
+        Text value = new Text();
+
+        while (reader.next(key, value)) {
+            mapper.map(key, value, context);
+        }
+
+        reader.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MergeCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MergeCuboidJobTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MergeCuboidJobTest.java
new file mode 100644
index 0000000..45a2dcc
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MergeCuboidJobTest.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.engine.mr.steps.MergeCuboidJob;
+
+/**
+ * @author ysong1
+ */
+public class MergeCuboidJobTest extends LocalFileMetadataTestCase {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws Exception {
+        conf = new Configuration();
+        conf.set("fs.default.name", "file:///");
+        conf.set("mapred.job.tracker", "local");
+
+        // for local runner out-of-memory issue
+        conf.set("mapreduce.task.io.sort.mb", "10");
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void test() throws Exception {
+        // String input =
+        // "src/test/resources/data/base_cuboid,src/test/resources/data/6d_cuboid";
+        String output = "target/test-output/merged_cuboid";
+        String cubeName = "test_kylin_cube_with_slr_ready";
+        String jobname = "merge_cuboid";
+
+        File baseFolder = File.createTempFile("kylin-f24668f6-dcff-4cb6-a89b-77f1119df8fa-", "base");
+        baseFolder.delete();
+        baseFolder.mkdir();
+        FileUtils.copyDirectory(new File("src/test/resources/data/base_cuboid"), baseFolder);
+        baseFolder.deleteOnExit();
+
+        File sixDFolder = File.createTempFile("kylin-f24668f6-dcff-4cb6-a89b-77f1119df8fa-", "6d");
+        sixDFolder.delete();
+        sixDFolder.mkdir();
+        FileUtils.copyDirectory(new File("src/test/resources/data/base_cuboid"), sixDFolder);
+        sixDFolder.deleteOnExit();
+
+        FileUtil.fullyDelete(new File(output));
+
+        // CubeManager cubeManager =
+        // CubeManager.getInstanceFromEnv(getTestConfig());
+
+        String[] args = { "-input", baseFolder.getAbsolutePath() + "," + sixDFolder.getAbsolutePath(), "-cubename", cubeName, "-segmentname", "20130331080000_20131212080000", "-output", output, "-jobname", jobname };
+        assertEquals("Job failed", 0, ToolRunner.run(conf, new MergeCuboidJob(), args));
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapperTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapperTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapperTest.java
new file mode 100644
index 0000000..a8d9670
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapperTest.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.CubeUpdate;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.dict.DictionaryGenerator;
+import org.apache.kylin.dict.DictionaryInfo;
+import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.dict.TrieDictionary;
+import org.apache.kylin.engine.mr.steps.MergeCuboidMapper;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.project.ProjectManager;
+import org.apache.kylin.source.ReadableTable.TableSignature;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author honma
+ */
+@SuppressWarnings("rawtypes")
+public class MergeCuboidMapperTest extends LocalFileMetadataTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(MergeCuboidMapperTest.class);
+
+    MapDriver<Text, Text, Text, Text> mapDriver;
+    CubeManager cubeManager;
+    CubeInstance cube;
+    DictionaryManager dictionaryManager;
+
+    TblColRef lfn;
+    TblColRef lsi;
+    TblColRef ssc;
+
+    private DictionaryInfo makeSharedDict() throws IOException {
+        TableSignature signature = new TableSignature();
+        signature.setSize(100);
+        signature.setLastModifiedTime(System.currentTimeMillis());
+        signature.setPath("fake_common_dict");
+
+        DictionaryInfo newDictInfo = new DictionaryInfo("", "", 0, "string", signature);
+
+        List<byte[]> values = new ArrayList<byte[]>();
+        values.add(new byte[] { 101, 101, 101 });
+        values.add(new byte[] { 102, 102, 102 });
+        Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueList(DataType.getInstance(newDictInfo.getDataType()), values);
+        newDictInfo.setCardinality(dict.getSize());
+        dictionaryManager.trySaveNewDict(dict, newDictInfo);
+        ((TrieDictionary) dict).dump(System.out);
+
+        return newDictInfo;
+    }
+
+    @Before
+    public void setUp() throws Exception {
+
+        createTestMetadata();
+
+        logger.info("The metadataUrl is : " + getTestConfig());
+
+        MetadataManager.clearCache();
+        CubeManager.clearCache();
+        ProjectManager.clearCache();
+        DictionaryManager.clearCache();
+
+        // hack for distributed cache
+        // CubeManager.removeInstance(KylinConfig.createInstanceFromUri("../job/meta"));//to
+        // make sure the following mapper could get latest CubeManger
+        FileUtils.deleteDirectory(new File("../job/meta"));
+
+        MergeCuboidMapper mapper = new MergeCuboidMapper();
+        mapDriver = MapDriver.newMapDriver(mapper);
+
+        cubeManager = CubeManager.getInstance(getTestConfig());
+        cube = cubeManager.getCube("test_kylin_cube_without_slr_left_join_ready_2_segments");
+        dictionaryManager = DictionaryManager.getInstance(getTestConfig());
+        lfn = cube.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "LSTG_FORMAT_NAME");
+        lsi = cube.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "CAL_DT");
+        ssc = cube.getDescriptor().findColumnRef("DEFAULT.TEST_CATEGORY_GROUPINGS", "META_CATEG_NAME");
+
+        DictionaryInfo sharedDict = makeSharedDict();
+
+        boolean isFirstSegment = true;
+        for (CubeSegment segment : cube.getSegments()) {
+
+            TableSignature signature = new TableSignature();
+            signature.setSize(100);
+            signature.setLastModifiedTime(System.currentTimeMillis());
+            signature.setPath("fake_dict_for" + lfn.getName() + segment.getName());
+
+            DictionaryInfo newDictInfo = new DictionaryInfo(lfn.getTable(), lfn.getColumnDesc().getName(), lfn.getColumnDesc().getZeroBasedIndex(), "string", signature);
+
+            List<byte[]> values = new ArrayList<byte[]>();
+            values.add(new byte[] { 97, 97, 97 });
+            if (isFirstSegment)
+                values.add(new byte[] { 99, 99, 99 });
+            else
+                values.add(new byte[] { 98, 98, 98 });
+            Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueList(DataType.getInstance(newDictInfo.getDataType()), values);
+            newDictInfo.setCardinality(dict.getSize());
+            dictionaryManager.trySaveNewDict(dict, newDictInfo);
+            ((TrieDictionary) dict).dump(System.out);
+
+            segment.putDictResPath(lfn, newDictInfo.getResourcePath());
+            segment.putDictResPath(lsi, sharedDict.getResourcePath());
+            segment.putDictResPath(ssc, sharedDict.getResourcePath());
+
+            // cubeManager.saveResource(segment.getCubeInstance());
+            // cubeManager.afterCubeUpdated(segment.getCubeInstance());
+
+            isFirstSegment = false;
+        }
+
+        CubeUpdate cubeBuilder = new CubeUpdate(cube);
+        cubeBuilder.setToUpdateSegs(cube.getSegments().toArray(new CubeSegment[cube.getSegments().size()]));
+        cube = cubeManager.updateCube(cubeBuilder);
+
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+        FileUtils.deleteDirectory(new File("../job/meta"));
+    }
+
+    @Test
+    public void test() throws IOException, ParseException {
+
+        //        String cubeName = "test_kylin_cube_without_slr_left_join_ready_2_segments";
+
+        CubeSegment newSeg = cubeManager.mergeSegments(cube, 0L, 1386835200000L, false);
+        //        String segmentName = newSeg.getName();
+
+        final Dictionary<?> dictionary = cubeManager.getDictionary(newSeg, lfn);
+        assertTrue(dictionary == null);
+        //        ((TrieDictionary) dictionary).dump(System.out);
+
+        // hack for distributed cache
+        //        File metaDir = new File("../job/meta");
+        //        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), metaDir);
+        //
+        //        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+        //        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+        //        // mapDriver.getConfiguration().set(KylinConfig.KYLIN_METADATA_URL,
+        //        // "../job/meta");
+        //
+        //        byte[] key = new byte[] { 0, 0, 0, 0, 0, 0, 0, -92, 1, 1, 1 };
+        //        byte[] value = new byte[] { 1, 2, 3 };
+        //        byte[] newkey = new byte[] { 0, 0, 0, 0, 0, 0, 0, -92, 1, 1, 2 };
+        //        byte[] newvalue = new byte[] { 1, 2, 3 };
+        //
+        //        mapDriver.withInput(new Text(key), new Text(value));
+        //        mapDriver.withOutput(new Text(newkey), new Text(newvalue));
+        //        mapDriver.setMapInputPath(new Path("/apps/hdmi-prod/b_kylin/prod/kylin-f24668f6-dcff-4cb6-a89b-77f1119df8fa/vac_sw_cube_v4/cuboid/15d_cuboid"));
+        //
+        //        mapDriver.runTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
new file mode 100644
index 0000000..847071d
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
+import org.apache.hadoop.security.Credentials;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class MockupMapContext {
+
+    public static Context create(final Configuration hconf, String metadataUrl, String cubeName, final Object[] outKV) {
+
+        hconf.set(BatchConstants.CFG_CUBE_NAME, cubeName);
+
+        return new WrappedMapper().getMapContext(new MapContext() {
+
+            @Override
+            public boolean nextKeyValue() throws IOException, InterruptedException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Object getCurrentKey() throws IOException, InterruptedException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Object getCurrentValue() throws IOException, InterruptedException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public void write(Object key, Object value) throws IOException, InterruptedException {
+                System.out.println("Write -- k:" + key + ", v:" + value);
+                if (outKV != null) {
+                    outKV[0] = key;
+                    outKV[1] = value;
+                }
+            }
+
+            @Override
+            public OutputCommitter getOutputCommitter() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public TaskAttemptID getTaskAttemptID() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public void setStatus(String msg) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getStatus() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public float getProgress() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Counter getCounter(Enum<?> counterName) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Counter getCounter(String groupName, String counterName) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Configuration getConfiguration() {
+                return hconf;
+            }
+
+            @Override
+            public Credentials getCredentials() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public JobID getJobID() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public int getNumReduceTasks() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path getWorkingDirectory() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getOutputKeyClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getOutputValueClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getMapOutputKeyClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getMapOutputValueClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getJobName() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends InputFormat<?, ?>> getInputFormatClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Mapper<?, ?, ?, ?>> getMapperClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Reducer<?, ?, ?, ?>> getCombinerClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Reducer<?, ?, ?, ?>> getReducerClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends OutputFormat<?, ?>> getOutputFormatClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Partitioner<?, ?>> getPartitionerClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public RawComparator<?> getSortComparator() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getJar() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public RawComparator<?> getGroupingComparator() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getJobSetupCleanupNeeded() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getTaskCleanupNeeded() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getProfileEnabled() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getProfileParams() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public IntegerRanges getProfileTaskRange(boolean isMap) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getUser() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getSymlink() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getArchiveClassPaths() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public URI[] getCacheArchives() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public URI[] getCacheFiles() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getLocalCacheArchives() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getLocalCacheFiles() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getFileClassPaths() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String[] getArchiveTimestamps() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String[] getFileTimestamps() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public int getMaxMapAttempts() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public int getMaxReduceAttempts() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public void progress() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public InputSplit getInputSplit() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public RawComparator<?> getCombinerKeyGroupingComparator() {
+                throw new NotImplementedException();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidJobTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidJobTest.java
new file mode 100644
index 0000000..6ed9010
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidJobTest.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.engine.mr.steps.NDCuboidJob;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class NDCuboidJobTest extends LocalFileMetadataTestCase {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws Exception {
+        conf = new Configuration();
+        conf.set("fs.default.name", "file:///");
+        conf.set("mapred.job.tracker", "local");
+
+        // for local runner out-of-memory issue
+        conf.set("mapreduce.task.io.sort.mb", "10");
+
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void testJob6D() throws Exception {
+        String input = "src/test/resources/data/base_cuboid/";
+        String output = "target/test-output/6d_cuboid";
+        String cubeName = "test_kylin_cube_with_slr_1_new_segment";
+        String segmentName = "20130331080000_20131212080000";
+        String jobname = "6d_cuboid";
+        String level = "1";
+
+        FileUtil.fullyDelete(new File(output));
+
+        String[] args = { "-input", input, "-cubename", cubeName, "-segmentname", segmentName, "-output", output, "-jobname", jobname, "-level", level };
+        assertEquals("Job failed", 0, ToolRunner.run(conf, new NDCuboidJob(), args));
+    }
+
+    @Test
+    public void testJob5D() throws Exception {
+        final String input = "src/test/resources/data/6d_cuboid/";
+        final String output = "target/test-output/5d_cuboid";
+        final String cubeName = "test_kylin_cube_with_slr_1_new_segment";
+        String segmentName = "20130331080000_20131212080000";
+        String jobname = "5d_cuboid";
+        String level = "2";
+
+        FileUtil.fullyDelete(new File(output));
+
+        String[] args = { "-input", input, "-cubename", cubeName, "-segmentname", segmentName, "-output", output, "-jobname", jobname, "-level", level };
+        assertEquals("Job failed", 0, ToolRunner.run(conf, new NDCuboidJob(), args));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidMapperTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidMapperTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidMapperTest.java
new file mode 100644
index 0000000..9f50cc6
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidMapperTest.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.steps.CuboidReducer;
+import org.apache.kylin.engine.mr.steps.NDCuboidMapper;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class NDCuboidMapperTest extends LocalFileMetadataTestCase {
+    MapReduceDriver<Text, Text, Text, Text, Text, Text> mapReduceDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    @Before
+    public void setUp() throws Exception {
+        createTestMetadata();
+
+        // hack for distributed cache
+        FileUtils.deleteDirectory(new File("../job/meta"));
+        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), new File("../job/meta"));
+
+        NDCuboidMapper mapper = new NDCuboidMapper();
+        CuboidReducer reducer = new CuboidReducer();
+        mapReduceDriver = MapReduceDriver.newMapReduceDriver(mapper, reducer);
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+        FileUtils.deleteDirectory(new File("../job/meta"));
+    }
+
+    @Test
+    public void testMapReduceWithSlr() throws IOException {
+
+        String cubeName = "test_kylin_cube_with_slr_1_new_segment";
+        String segmentName = "20130331080000_20131212080000";
+        mapReduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+        mapReduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+
+        byte[] key = { 0, 0, 0, 0, 0, 0, 1, -1, 49, 48, 48, 48, 48, 48, 48, 48, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 11, 54, -105, 55, 13, 71, 114, 65, 66, 73, 78, 9, 9, 9, 9, 9, 9, 9, 9, 0, 10, 0 };
+        byte[] value = { 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 1 };
+        Pair<Text, Text> input1 = new Pair<Text, Text>(new Text(key), new Text(value));
+
+        mapReduceDriver.addInput(input1);
+
+        List<Pair<Text, Text>> result = mapReduceDriver.run();
+
+        assertEquals(4, result.size());
+
+        byte[] resultKey = { 0, 0, 0, 0, 0, 0, 1, 127, 49, 48, 48, 48, 48, 48, 48, 48, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 55, 13, 71, 114, 65, 66, 73, 78, 9, 9, 9, 9, 9, 9, 9, 9, 0, 10, 0 };
+        byte[] resultValue = { 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 1 };
+        Pair<Text, Text> output1 = new Pair<Text, Text>(new Text(resultKey), new Text(resultValue));
+
+        //As we will truncate decimal(KYLIN-766), value will no longer equals to resultValue
+        Collection<Text> keys = Collections2.transform(result, new Function<Pair<Text, Text>, Text>() {
+            @Nullable
+            @Override
+            public Text apply(Pair<Text, Text> input) {
+                return input.getFirst();
+            }
+        });
+        assertTrue(keys.contains(output1.getFirst()));
+        assertTrue(!result.contains(output1));
+
+        long[] keySet = new long[result.size()];
+
+        System.out.println(Bytes.toLong(new byte[] { 0, 0, 0, 0, 0, 0, 1, -1 }));
+        for (int i = 0; i < result.size(); i++) {
+            byte[] bytes = new byte[result.get(i).getFirst().getLength()];
+            System.arraycopy(result.get(i).getFirst().getBytes(), 0, bytes, 0, result.get(i).getFirst().getLength());
+            System.out.println(Bytes.toLong(bytes));
+            keySet[i] = Bytes.toLong(bytes);
+        }
+
+        // refer to CuboidSchedulerTest.testGetSpanningCuboid()
+        assertArrayEquals(new long[] { 383, 447, 503, 504 }, keySet);
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/pom.xml
----------------------------------------------------------------------
diff --git a/job/pom.xml b/job/pom.xml
index cbb0772..3042778 100644
--- a/job/pom.xml
+++ b/job/pom.xml
@@ -104,6 +104,13 @@
         </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-job</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-storage-hbase</artifactId>
             <type>test-jar</type>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java b/job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java
deleted file mode 100644
index d24c99c..0000000
--- a/job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.MRBatchCubingEngine;
-import org.apache.kylin.engine.mr.MRBatchCubingEngine2;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-
-public class BuildEngineFactory {
-    
-    private static IBatchCubingEngine defaultBatchEngine;
-    
-    public static IBatchCubingEngine defaultBatchEngine() {
-        if (defaultBatchEngine == null) {
-            KylinConfig conf = KylinConfig.getInstanceFromEnv();
-            if (conf.isCubingInMem()) {
-                defaultBatchEngine = new MRBatchCubingEngine2();
-            } else {
-                defaultBatchEngine = new MRBatchCubingEngine();
-            }
-        }
-        return defaultBatchEngine;
-    }
-    
-    /** Build a new cube segment, typically its time range appends to the end of current cube. */
-    public static DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter) {
-        return defaultBatchEngine().createBatchCubingJob(newSegment, submitter);
-    }
-    
-    /** Merge multiple small segments into a big one. */
-    public static DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter) {
-        return defaultBatchEngine().createBatchMergeJob(mergeSegment, submitter);
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java b/job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
deleted file mode 100644
index 904f557..0000000
--- a/job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
+++ /dev/null
@@ -1,35 +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;
-
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-
-public interface IBatchCubingEngine {
-
-    /** Build a new cube segment, typically its time range appends to the end of current cube. */
-    public DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter);
-    
-    /** Merge multiple small segments into a big one. */
-    public DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter);
-    
-    public Class<?> getSourceInterface();
-    
-    public Class<?> getStorageInterface();
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/IStreamingCubingEngine.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/IStreamingCubingEngine.java b/job/src/main/java/org/apache/kylin/engine/IStreamingCubingEngine.java
deleted file mode 100644
index 0359ce9..0000000
--- a/job/src/main/java/org/apache/kylin/engine/IStreamingCubingEngine.java
+++ /dev/null
@@ -1,8 +0,0 @@
-package org.apache.kylin.engine;
-
-import org.apache.kylin.cube.CubeSegment;
-
-public interface IStreamingCubingEngine {
-
-    public Runnable createStreamingCubingBuilder(CubeSegment seg);
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java b/job/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
deleted file mode 100644
index a39ac74..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine.mr;
-
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
-import org.apache.kylin.engine.mr.IMROutput.IMRBatchCubingOutputSide;
-import org.apache.kylin.job.common.MapReduceExecutable;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.kylin.job.hadoop.cube.BaseCuboidJob;
-import org.apache.kylin.job.hadoop.cube.NDCuboidJob;
-
-public class BatchCubingJobBuilder extends JobBuilderSupport {
-
-    private final IMRBatchCubingInputSide inputSide;
-    private final IMRBatchCubingOutputSide outputSide;
-
-    public BatchCubingJobBuilder(CubeSegment newSegment, String submitter) {
-        super(newSegment, submitter);
-        this.inputSide = MRUtil.getBatchCubingInputSide(seg);
-        this.outputSide = MRUtil.getBatchCubingOutputSide(seg);
-    }
-
-    public CubingJob build() {
-        final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
-        final String jobId = result.getId();
-        final String cuboidRootPath = getCuboidRootPath(jobId);
-
-        // Phase 1: Create Flat Table
-        inputSide.addStepPhase1_CreateFlatTable(result);
-
-        // Phase 2: Build Dictionary
-        result.addTask(createFactDistinctColumnsStep(jobId));
-        result.addTask(createBuildDictionaryStep(jobId));
-
-        // Phase 3: Build Cube
-        final int groupRowkeyColumnsCount = seg.getCubeDesc().getRowkey().getNCuboidBuildLevels();
-        final int totalRowkeyColumnsCount = seg.getCubeDesc().getRowkey().getRowKeyColumns().length;
-        final String[] cuboidOutputTempPath = getCuboidOutputPaths(cuboidRootPath, totalRowkeyColumnsCount, groupRowkeyColumnsCount);
-        // base cuboid step
-        result.addTask(createBaseCuboidStep(cuboidOutputTempPath, jobId));
-        // n dim cuboid steps
-        for (int i = 1; i <= groupRowkeyColumnsCount; i++) {
-            int dimNum = totalRowkeyColumnsCount - i;
-            result.addTask(createNDimensionCuboidStep(cuboidOutputTempPath, dimNum, totalRowkeyColumnsCount));
-        }
-        outputSide.addStepPhase3_BuildCube(result, cuboidRootPath);
-
-        // Phase 4: Update Metadata & Cleanup
-        result.addTask(createUpdateCubeInfoAfterBuildStep(jobId));
-        inputSide.addStepPhase4_Cleanup(result);
-        outputSide.addStepPhase4_Cleanup(result);
-
-        return result;
-    }
-
-    private MapReduceExecutable createBaseCuboidStep(String[] cuboidOutputTempPath, String jobId) {
-        // base cuboid job
-        MapReduceExecutable baseCuboidStep = new MapReduceExecutable();
-
-        StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd, seg);
-
-        baseCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_BASE_CUBOID);
-
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "segmentname", seg.getName());
-        appendExecCmdParameters(cmd, "input", ""); // marks flat table input
-        appendExecCmdParameters(cmd, "output", cuboidOutputTempPath[0]);
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Base_Cuboid_Builder_" + seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "level", "0");
-
-        baseCuboidStep.setMapReduceParams(cmd.toString());
-        baseCuboidStep.setMapReduceJobClass(BaseCuboidJob.class);
-        baseCuboidStep.setCounterSaveAs(CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES);
-        return baseCuboidStep;
-    }
-
-    private MapReduceExecutable createNDimensionCuboidStep(String[] cuboidOutputTempPath, int dimNum, int totalRowkeyColumnCount) {
-        // ND cuboid job
-        MapReduceExecutable ndCuboidStep = new MapReduceExecutable();
-
-        ndCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_N_D_CUBOID + " : " + dimNum + "-Dimension");
-        StringBuilder cmd = new StringBuilder();
-
-        appendMapReduceParameters(cmd, seg);
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().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, "level", "" + (totalRowkeyColumnCount - dimNum));
-
-        ndCuboidStep.setMapReduceParams(cmd.toString());
-        ndCuboidStep.setMapReduceJobClass(NDCuboidJob.class);
-        return ndCuboidStep;
-    }
-
-    private String[] getCuboidOutputPaths(String cuboidRootPath, int totalRowkeyColumnCount, int groupRowkeyColumnsCount) {
-        String[] paths = new String[groupRowkeyColumnsCount + 1];
-        for (int i = 0; i <= groupRowkeyColumnsCount; i++) {
-            int dimNum = totalRowkeyColumnCount - i;
-            if (dimNum == totalRowkeyColumnCount) {
-                paths[i] = cuboidRootPath + "base_cuboid";
-            } else {
-                paths[i] = cuboidRootPath + dimNum + "d_cuboid";
-            }
-        }
-        return paths;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java b/job/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
deleted file mode 100644
index b6f264e..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine.mr;
-
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
-import org.apache.kylin.engine.mr.IMROutput2.IMRBatchCubingOutputSide2;
-import org.apache.kylin.engine.mr.steps.InMemCuboidJob;
-import org.apache.kylin.engine.mr.steps.SaveStatisticsStep;
-import org.apache.kylin.job.common.MapReduceExecutable;
-import org.apache.kylin.job.constant.ExecutableConstants;
-
-public class BatchCubingJobBuilder2 extends JobBuilderSupport {
-
-    private final IMRBatchCubingInputSide inputSide;
-    private final IMRBatchCubingOutputSide2 outputSide;
-
-    public BatchCubingJobBuilder2(CubeSegment newSegment, String submitter) {
-        super(newSegment, submitter);
-        this.inputSide = MRUtil.getBatchCubingInputSide(seg);
-        this.outputSide = MRUtil.getBatchCubingOutputSide2(seg);
-    }
-
-    public CubingJob build() {
-        final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
-        final String jobId = result.getId();
-
-        // Phase 1: Create Flat Table
-        inputSide.addStepPhase1_CreateFlatTable(result);
-
-        // Phase 2: Build Dictionary
-        result.addTask(createFactDistinctColumnsStepWithStats(jobId));
-        result.addTask(createBuildDictionaryStep(jobId));
-        result.addTask(createSaveStatisticsStep(jobId));
-        outputSide.addStepPhase2_BuildDictionary(result);
-
-        // Phase 3: Build Cube
-        result.addTask(createInMemCubingStep(jobId));
-        outputSide.addStepPhase3_BuildCube(result);
-
-        // Phase 4: Update Metadata & Cleanup
-        result.addTask(createUpdateCubeInfoAfterBuildStep(jobId));
-        inputSide.addStepPhase4_Cleanup(result);
-        outputSide.addStepPhase4_Cleanup(result);
-
-        return result;
-    }
-
-    private SaveStatisticsStep createSaveStatisticsStep(String jobId) {
-        SaveStatisticsStep result = new SaveStatisticsStep();
-        result.setName(ExecutableConstants.STEP_NAME_SAVE_STATISTICS);
-        result.setCubeName(seg.getCubeInstance().getName());
-        result.setSegmentId(seg.getUuid());
-        result.setStatisticsPath(getStatisticsPath(jobId));
-        return result;
-    }
-
-    private MapReduceExecutable createInMemCubingStep(String jobId) {
-        // base cuboid job
-        MapReduceExecutable cubeStep = new MapReduceExecutable();
-
-        StringBuilder cmd = new StringBuilder();
-        appendMapReduceParameters(cmd, seg);
-
-        cubeStep.setName(ExecutableConstants.STEP_NAME_BUILD_IN_MEM_CUBE);
-
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "segmentname", seg.getName());
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Cube_Builder_" + seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "jobflowid", jobId);
-
-        cubeStep.setMapReduceParams(cmd.toString());
-        cubeStep.setMapReduceJobClass(InMemCuboidJob.class);
-        cubeStep.setCounterSaveAs(CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES + "," + CubingJob.CUBE_SIZE_BYTES);
-        return cubeStep;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java b/job/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
deleted file mode 100644
index 6264ebd..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine.mr;
-
-import java.util.List;
-
-import org.apache.kylin.common.util.StringUtil;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.IMROutput.IMRBatchMergeOutputSide;
-import org.apache.kylin.job.common.MapReduceExecutable;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.kylin.job.hadoop.cube.MergeCuboidJob;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-public class BatchMergeJobBuilder extends JobBuilderSupport {
-
-    private final IMRBatchMergeOutputSide outputSide;
-
-    public BatchMergeJobBuilder(CubeSegment mergeSegment, String submitter) {
-        super(mergeSegment, submitter);
-        this.outputSide = MRUtil.getBatchMergeOutputSide(seg);
-    }
-
-    public CubingJob build() {
-        final CubingJob result = CubingJob.createMergeJob(seg, submitter, config);
-        final String jobId = result.getId();
-        final String cuboidRootPath = getCuboidRootPath(jobId);
-
-        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(seg);
-        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();
-        for (CubeSegment merging : mergingSegments) {
-            mergingSegmentIds.add(merging.getUuid());
-            mergingCuboidPaths.add(getCuboidRootPath(merging) + "*");
-        }
-
-        // Phase 1: Merge Dictionary
-        result.addTask(createMergeDictionaryStep(mergingSegmentIds));
-
-        // Phase 2: Merge Cube Files
-        String formattedPath = StringUtil.join(mergingCuboidPaths, ",");
-        result.addTask(createMergeCuboidDataStep(seg, formattedPath, cuboidRootPath));
-        outputSide.addStepPhase2_BuildCube(result, cuboidRootPath);
-
-        // Phase 3: Update Metadata & Cleanup
-        result.addTask(createUpdateCubeInfoAfterMergeStep(mergingSegmentIds, jobId));
-        outputSide.addStepPhase3_Cleanup(result);
-
-        return result;
-    }
-
-    private MapReduceExecutable createMergeCuboidDataStep(CubeSegment seg, String inputPath, String outputPath) {
-        MapReduceExecutable mergeCuboidDataStep = new MapReduceExecutable();
-        mergeCuboidDataStep.setName(ExecutableConstants.STEP_NAME_MERGE_CUBOID);
-        StringBuilder cmd = new StringBuilder();
-
-        appendMapReduceParameters(cmd, seg);
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "segmentname", seg.getName());
-        appendExecCmdParameters(cmd, "input", inputPath);
-        appendExecCmdParameters(cmd, "output", outputPath);
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Merge_Cuboid_" + seg.getCubeInstance().getName() + "_Step");
-
-        mergeCuboidDataStep.setMapReduceParams(cmd.toString());
-        mergeCuboidDataStep.setMapReduceJobClass(MergeCuboidJob.class);
-        return mergeCuboidDataStep;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java b/job/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
deleted file mode 100644
index e0fc438..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
+++ /dev/null
@@ -1,98 +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;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import org.apache.kylin.common.util.StringUtil;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.steps.MergeCuboidFromStorageJob;
-import org.apache.kylin.engine.mr.steps.MergeStatisticsStep;
-import org.apache.kylin.job.common.MapReduceExecutable;
-import org.apache.kylin.job.constant.ExecutableConstants;
-
-import java.util.List;
-
-public class BatchMergeJobBuilder2 extends JobBuilderSupport {
-
-    private final IMROutput2.IMRBatchMergeOutputSide2 outputSide;
-    
-    public BatchMergeJobBuilder2(CubeSegment mergeSegment, String submitter) {
-        super(mergeSegment, submitter);
-        this.outputSide = MRUtil.getBatchMergeOutputSide2(seg);
-    }
-
-    public CubingJob build() {
-        final CubingJob result = CubingJob.createMergeJob(seg, submitter, config);
-        final String jobId = result.getId();
-
-        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(seg);
-        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();
-        for (CubeSegment merging : mergingSegments) {
-            mergingSegmentIds.add(merging.getUuid());
-            mergingHTables.add(merging.getStorageLocationIdentifier());
-        }
-
-        // Phase 1: Merge Dictionary
-        result.addTask(createMergeDictionaryStep(mergingSegmentIds));
-        result.addTask(createMergeStatisticsStep(seg, mergingSegmentIds, getStatisticsPath(jobId)));
-        outputSide.addStepPhase1_MergeDictionary(result);
-
-        // Phase 2: Merge Cube
-        String formattedTables = StringUtil.join(mergingHTables, ",");
-        result.addTask(createMergeCuboidDataFromStorageStep(formattedTables, jobId));
-        outputSide.addStepPhase2_BuildCube(result);
-
-        // Phase 3: Update Metadata & Cleanup
-        result.addTask(createUpdateCubeInfoAfterMergeStep(mergingSegmentIds, jobId));
-        outputSide.addStepPhase3_Cleanup(result);
-
-        return result;
-    }
-
-    private MergeStatisticsStep createMergeStatisticsStep(CubeSegment seg, List<String> mergingSegmentIds, String mergedStatisticsFolder) {
-        MergeStatisticsStep result = new MergeStatisticsStep();
-        result.setName(ExecutableConstants.STEP_NAME_MERGE_STATISTICS);
-        result.setCubeName(seg.getCubeInstance().getName());
-        result.setSegmentId(seg.getUuid());
-        result.setMergingSegmentIds(mergingSegmentIds);
-        result.setMergedStatisticsPath(mergedStatisticsFolder);
-        return result;
-    }
-
-    private MapReduceExecutable createMergeCuboidDataFromStorageStep(String inputTableNames, String jobId) {
-        MapReduceExecutable mergeCuboidDataStep = new MapReduceExecutable();
-        mergeCuboidDataStep.setName(ExecutableConstants.STEP_NAME_MERGE_CUBOID);
-        StringBuilder cmd = new StringBuilder();
-
-        appendMapReduceParameters(cmd, seg);
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "segmentname", seg.getName());
-        appendExecCmdParameters(cmd, "jobname", "Kylin_Merge_Cuboid_" + seg.getCubeInstance().getName() + "_Step");
-        appendExecCmdParameters(cmd, "jobflowid", jobId);
-
-        mergeCuboidDataStep.setMapReduceParams(cmd.toString());
-        mergeCuboidDataStep.setMapReduceJobClass(MergeCuboidFromStorageJob.class);
-        mergeCuboidDataStep.setCounterSaveAs(",," + CubingJob.CUBE_SIZE_BYTES);
-        return mergeCuboidDataStep;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/ByteArrayWritable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/ByteArrayWritable.java b/job/src/main/java/org/apache/kylin/engine/mr/ByteArrayWritable.java
deleted file mode 100644
index 37a8841..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/ByteArrayWritable.java
+++ /dev/null
@@ -1,166 +0,0 @@
-package org.apache.kylin.engine.mr;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.kylin.common.util.Bytes;
-
-public class ByteArrayWritable implements WritableComparable<ByteArrayWritable> {
-
-    private byte[] data;
-    private int offset;
-    private int length;
-
-    public ByteArrayWritable() {
-        this(null, 0, 0);
-    }
-
-    public ByteArrayWritable(int capacity) {
-        this(new byte[capacity], 0, capacity);
-    }
-
-    public ByteArrayWritable(byte[] data) {
-        this(data, 0, data == null ? 0 : data.length);
-    }
-
-    public ByteArrayWritable(byte[] data, int offset, int length) {
-        this.data = data;
-        this.offset = offset;
-        this.length = length;
-    }
-
-    public byte[] array() {
-        return data;
-    }
-
-    public int offset() {
-        return offset;
-    }
-
-    public int length() {
-        return length;
-    }
-
-    public void set(byte[] array) {
-        set(array, 0, array.length);
-    }
-
-    public void set(byte[] array, int offset, int length) {
-        this.data = array;
-        this.offset = offset;
-        this.length = length;
-    }
-
-    public ByteBuffer asBuffer() {
-        if (data == null)
-            return null;
-        else if (offset == 0 && length == data.length)
-            return ByteBuffer.wrap(data);
-        else
-            return ByteBuffer.wrap(data, offset, length).slice();
-    }
-
-    @Override
-    public int hashCode() {
-        if (data == null)
-            return 0;
-        else
-            return Bytes.hashCode(data, offset, length);
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-        out.writeInt(this.length);
-        out.write(this.data, this.offset, this.length);
-    }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-        this.length = in.readInt();
-        this.data = new byte[this.length];
-        in.readFully(this.data, 0, this.length);
-        this.offset = 0;
-    }
-
-    // Below methods copied from BytesWritable
-    /**
-     * Define the sort order of the BytesWritable.
-     * @param that The other bytes writable
-     * @return Positive if left is bigger than right, 0 if they are equal, and
-     *         negative if left is smaller than right.
-     */
-    public int compareTo(ByteArrayWritable that) {
-        return WritableComparator.compareBytes(this.data, this.offset, this.length, that.data, that.offset, that.length);
-    }
-
-    /**
-     * Compares the bytes in this object to the specified byte array
-     * @param that
-     * @return Positive if left is bigger than right, 0 if they are equal, and
-     *         negative if left is smaller than right.
-     */
-    public int compareTo(final byte[] that) {
-        return WritableComparator.compareBytes(this.data, this.offset, this.length, that, 0, that.length);
-    }
-
-    /**
-     * @see java.lang.Object#equals(java.lang.Object)
-     */
-    @Override
-    public boolean equals(Object right_obj) {
-        if (right_obj instanceof byte[]) {
-            return compareTo((byte[]) right_obj) == 0;
-        }
-        if (right_obj instanceof ByteArrayWritable) {
-            return compareTo((ByteArrayWritable) right_obj) == 0;
-        }
-        return false;
-    }
-
-    /**
-     * @see java.lang.Object#toString()
-     */
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder(3 * this.length);
-        final int endIdx = this.offset + this.length;
-        for (int idx = this.offset; idx < endIdx; idx++) {
-            sb.append(' ');
-            String num = Integer.toHexString(0xff & this.data[idx]);
-            // if it is only one digit, add a leading 0.
-            if (num.length() < 2) {
-                sb.append('0');
-            }
-            sb.append(num);
-        }
-        return sb.length() > 0 ? sb.substring(1) : "";
-    }
-
-    /** A Comparator optimized for ByteArrayWritable.
-     */
-    public static class Comparator extends WritableComparator {
-        private BytesWritable.Comparator comparator = new BytesWritable.Comparator();
-
-        /** constructor */
-        public Comparator() {
-            super(ByteArrayWritable.class);
-        }
-
-        /**
-         * @see org.apache.hadoop.io.WritableComparator#compare(byte[], int, int, byte[], int, int)
-         */
-        @Override
-        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-            return comparator.compare(b1, s1, l1, b2, s2, l2);
-        }
-    }
-
-    static { // register this comparator
-        WritableComparator.define(ByteArrayWritable.class, new Comparator());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/CubingJob.java b/job/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
deleted file mode 100644
index 77cbab7..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
+++ /dev/null
@@ -1,188 +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;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.TimeZone;
-
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.job.common.MapReduceExecutable;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecutableState;
-import org.apache.kylin.job.execution.ExecuteResult;
-import org.apache.kylin.job.execution.Output;
-
-/**
- */
-public class CubingJob extends DefaultChainedExecutable {
-
-    // KEYS of Output.extraInfo map, info passed across job steps
-    public static final String SOURCE_RECORD_COUNT = "sourceRecordCount";
-    public static final String SOURCE_SIZE_BYTES = "sourceSizeBytes";
-    public static final String CUBE_SIZE_BYTES = "byteSizeBytes";
-    public static final String MAP_REDUCE_WAIT_TIME = "mapReduceWaitTime";
-    
-    private static final String CUBE_INSTANCE_NAME = "cubeName";
-    private static final String SEGMENT_ID = "segmentId";
-    
-    public static CubingJob createBuildJob(CubeSegment seg, String submitter, JobEngineConfig config) {
-        return initCubingJob(seg, "BUILD", submitter, config);
-    }
-    
-    public static CubingJob createMergeJob(CubeSegment seg, String submitter, JobEngineConfig config) {
-        return initCubingJob(seg, "MERGE", submitter, config);
-    }
-    
-    private static CubingJob initCubingJob(CubeSegment seg, String jobType, String submitter, JobEngineConfig config) {
-        CubingJob result = new CubingJob();
-        SimpleDateFormat format = new SimpleDateFormat("z yyyy-MM-dd HH:mm:ss");
-        format.setTimeZone(TimeZone.getTimeZone(config.getTimeZone()));
-        result.setCubeName(seg.getCubeInstance().getName());
-        result.setSegmentId(seg.getUuid());
-        result.setName(seg.getCubeInstance().getName() + " - " + seg.getName() + " - " + jobType + " - " + format.format(new Date(System.currentTimeMillis())));
-        result.setSubmitter(submitter);
-        result.setNotifyList(seg.getCubeInstance().getDescriptor().getNotifyList());
-        return result;
-    }
-
-    public CubingJob() {
-        super();
-    }
-    
-    void setCubeName(String name) {
-        setParam(CUBE_INSTANCE_NAME, name);
-    }
-
-    public String getCubeName() {
-        return getParam(CUBE_INSTANCE_NAME);
-    }
-
-    void setSegmentId(String segmentId) {
-        setParam(SEGMENT_ID, segmentId);
-    }
-
-    public String getSegmentId() {
-        return getParam(SEGMENT_ID);
-    }
-
-    @Override
-    protected Pair<String, String> formatNotifications(ExecutableContext context, ExecutableState state) {
-        CubeInstance cubeInstance = CubeManager.getInstance(context.getConfig()).getCube(getCubeName());
-        final Output output = jobService.getOutput(getId());
-        String logMsg;
-        state = output.getState();
-        if (state != ExecutableState.ERROR &&
-                !cubeInstance.getDescriptor().getStatusNeedNotify().contains(state.toString().toLowerCase())) {
-            logger.info("state:" + state + " no need to notify users");
-            return null;
-        }
-        switch (state) {
-            case ERROR:
-                logMsg = output.getVerboseMsg();
-                break;
-            case DISCARDED:
-                logMsg = "job has been discarded";
-                break;
-            case SUCCEED:
-                logMsg = "job has succeeded";
-                break;
-            default:
-                return null;
-        }
-        if (logMsg == null) {
-            logMsg = "no error message";
-        }
-        String content = ExecutableConstants.NOTIFY_EMAIL_TEMPLATE;
-        content = content.replaceAll("\\$\\{job_name\\}", getName());
-        content = content.replaceAll("\\$\\{result\\}", state.toString());
-        content = content.replaceAll("\\$\\{cube_name\\}", getCubeName());
-        content = content.replaceAll("\\$\\{start_time\\}", new Date(getStartTime()).toString());
-        content = content.replaceAll("\\$\\{duration\\}", getDuration() / 60000 + "mins");
-        content = content.replaceAll("\\$\\{mr_waiting\\}", getMapReduceWaitTime() / 60000 + "mins");
-        content = content.replaceAll("\\$\\{last_update_time\\}", new Date(getLastModified()).toString());
-        content = content.replaceAll("\\$\\{submitter\\}", getSubmitter());
-        content = content.replaceAll("\\$\\{error_log\\}", logMsg);
-
-        try {
-            InetAddress inetAddress = InetAddress.getLocalHost();
-            content = content.replaceAll("\\$\\{job_engine\\}", inetAddress.getCanonicalHostName());
-        } catch (UnknownHostException e) {
-            logger.warn(e.getLocalizedMessage(), e);
-        }
-
-        String title = "["+ state.toString() + "] - [Kylin Cube Build Job]-" + getCubeName();
-        return Pair.of(title, content);
-    }
-
-    @Override
-    protected void onExecuteFinished(ExecuteResult result, ExecutableContext executableContext) {
-        long time = 0L;
-        for (AbstractExecutable task: getTasks()) {
-            final ExecutableState status = task.getStatus();
-            if (status != ExecutableState.SUCCEED) {
-                break;
-            }
-            if (task instanceof MapReduceExecutable) {
-                time += ((MapReduceExecutable) task).getMapReduceWaitTime();
-            }
-        }
-        setMapReduceWaitTime(time);
-        super.onExecuteFinished(result, executableContext);
-    }
-
-    public long getMapReduceWaitTime() {
-        return getExtraInfoAsLong(MAP_REDUCE_WAIT_TIME, 0L);
-    }
-
-    public void setMapReduceWaitTime(long t) {
-        addExtraInfo(MAP_REDUCE_WAIT_TIME, t + "");
-    }
-    
-    public long findSourceRecordCount() {
-        return Long.parseLong(findExtraInfo(SOURCE_RECORD_COUNT, "0"));
-    }
-    
-    public long findSourceSizeBytes() {
-        return Long.parseLong(findExtraInfo(SOURCE_SIZE_BYTES, "0"));
-    }
-    
-    public long findCubeSizeBytes() {
-        return Long.parseLong(findExtraInfo(CUBE_SIZE_BYTES, "0"));
-    }
-    
-    private String findExtraInfo(String key, String dft) {
-        for (AbstractExecutable child : getTasks()) {
-            Output output = executableManager.getOutput(child.getId());
-            String value = output.getExtra().get(key);
-            if (value != null)
-                return value;
-        }
-        return dft;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/IMRInput.java b/job/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
deleted file mode 100644
index 0c39398..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.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;
-
-/**
- * Any ITableSource that wishes to serve as input of MapReduce build engine must adapt to this interface.
- */
-public interface IMRInput {
-
-    /** Return a helper to participate in batch cubing job flow. */
-    public IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg);
-
-    /** Return an InputFormat that reads from specified table. */
-    public IMRTableInputFormat getTableInputFormat(TableDesc table);
-    
-    /**
-     * Utility that configures mapper to read from a table.
-     */
-    public interface IMRTableInputFormat {
-        
-        /** Configure the InputFormat of given job. */
-        public void configureJob(Job job);
-        
-        /** Parse a mapper input object into column values. */
-        public String[] parseMapperInput(Object mapperInput);
-    }
-    
-    /**
-     * Participate the batch cubing flow as the input side. Responsible for creating
-     * intermediate flat table (Phase 1) and clean up any leftover (Phase 4).
-     * 
-     * - Phase 1: Create Flat Table
-     * - Phase 2: Build Dictionary (with FlatTableInputFormat)
-     * - Phase 3: Build Cube (with FlatTableInputFormat)
-     * - Phase 4: Update Metadata & Cleanup
-     */
-    public interface IMRBatchCubingInputSide {
-        
-        /** Return an InputFormat that reads from the intermediate flat table */
-        public IMRTableInputFormat getFlatTableInputFormat();
-        
-        /** Add step that creates an intermediate flat table as defined by CubeJoinedFlatTableDesc */
-        public void addStepPhase1_CreateFlatTable(DefaultChainedExecutable jobFlow);
-        
-        /** Add step that does necessary clean up, like delete the intermediate flat table */
-        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow);
-    }
-}


[17/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
new file mode 100644
index 0000000..bf3a3a4
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.common.RowKeySplitter;
+import org.apache.kylin.common.util.SplittedBytes;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.dict.DictionaryManager;
+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.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+
+/**
+ * @author ysong1, honma
+ */
+public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
+
+    private KylinConfig config;
+    private String cubeName;
+    private String segmentName;
+    private CubeManager cubeManager;
+    private CubeInstance cube;
+    private CubeDesc cubeDesc;
+    private CubeSegment mergedCubeSegment;
+    private CubeSegment sourceCubeSegment;// Must be unique during a mapper's
+    // life cycle
+
+    private Text outputKey = new Text();
+
+    private byte[] newKeyBuf;
+    private RowKeySplitter rowKeySplitter;
+
+    private HashMap<TblColRef, Boolean> dictsNeedMerging = new HashMap<TblColRef, Boolean>();
+
+    private static final Pattern JOB_NAME_PATTERN = Pattern.compile("kylin-([0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12})");
+
+    private Boolean checkNeedMerging(TblColRef col) throws IOException {
+        Boolean ret = dictsNeedMerging.get(col);
+        if (ret != null)
+            return ret;
+        else {
+            ret = cubeDesc.getRowkey().isUseDictionary(col);
+            if (ret) {
+                String dictTable = (String) DictionaryManager.getInstance(config).decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0];
+                ret = cubeDesc.getFactTable().equalsIgnoreCase(dictTable);
+            }
+            dictsNeedMerging.put(col, ret);
+            return ret;
+        }
+    }
+
+    private String extractJobIDFromPath(String path) {
+        Matcher matcher = JOB_NAME_PATTERN.matcher(path);
+        // check the first occurance
+        if (matcher.find()) {
+            return matcher.group(1);
+        } else {
+            throw new IllegalStateException("Can not extract job ID from file path : " + path);
+        }
+    }
+
+    private CubeSegment findSegmentWithUuid(String jobID, CubeInstance cubeInstance) {
+        for (CubeSegment segment : cubeInstance.getSegments()) {
+            String lastBuildJobID = segment.getLastBuildJobID();
+            if (lastBuildJobID != null && lastBuildJobID.equalsIgnoreCase(jobID)) {
+                return segment;
+            }
+        }
+
+        throw new IllegalStateException("No merging segment's last build job ID equals " + jobID);
+
+    }
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME).toUpperCase();
+
+        config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        cubeManager = CubeManager.getInstance(config);
+        cube = cubeManager.getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+        mergedCubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
+
+        // int colCount = cubeDesc.getRowkey().getRowKeyColumns().length;
+        newKeyBuf = new byte[256];// size will auto-grow
+
+        // decide which source segment
+        InputSplit inputSplit = context.getInputSplit();
+        String filePath = ((FileSplit) inputSplit).getPath().toString();
+        System.out.println("filePath:" + filePath);
+        String jobID = extractJobIDFromPath(filePath);
+        System.out.println("jobID:" + jobID);
+        sourceCubeSegment = findSegmentWithUuid(jobID, cube);
+        System.out.println(sourceCubeSegment);
+
+        this.rowKeySplitter = new RowKeySplitter(sourceCubeSegment, 65, 255);
+    }
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        long cuboidID = rowKeySplitter.split(key.getBytes(), key.getBytes().length);
+        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidID);
+
+        SplittedBytes[] splittedByteses = rowKeySplitter.getSplitBuffers();
+        int bufOffset = 0;
+        BytesUtil.writeLong(cuboidID, newKeyBuf, bufOffset, RowConstants.ROWKEY_CUBOIDID_LEN);
+        bufOffset += RowConstants.ROWKEY_CUBOIDID_LEN;
+
+        for (int i = 0; i < cuboid.getColumns().size(); ++i) {
+            TblColRef col = cuboid.getColumns().get(i);
+
+            if (this.checkNeedMerging(col)) {
+                // if dictionary on fact table column, needs rewrite
+                DictionaryManager dictMgr = DictionaryManager.getInstance(config);
+                Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(col));
+                Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(col));
+
+                while (sourceDict.getSizeOfValue() > newKeyBuf.length - bufOffset || mergedDict.getSizeOfValue() > newKeyBuf.length - bufOffset) {
+                    byte[] oldBuf = newKeyBuf;
+                    newKeyBuf = new byte[2 * newKeyBuf.length];
+                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
+                }
+
+                int idInSourceDict = BytesUtil.readUnsigned(splittedByteses[i + 1].value, 0, splittedByteses[i + 1].length);
+                int idInMergedDict;
+
+                int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBuf, bufOffset);
+                if (size < 0) {
+                    idInMergedDict = mergedDict.nullId();
+                } else {
+                    idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBuf, bufOffset, size);
+                }
+
+                BytesUtil.writeUnsigned(idInMergedDict, newKeyBuf, bufOffset, mergedDict.getSizeOfId());
+                bufOffset += mergedDict.getSizeOfId();
+            } else {
+                // keep as it is
+                while (splittedByteses[i + 1].length > newKeyBuf.length - bufOffset) {
+                    byte[] oldBuf = newKeyBuf;
+                    newKeyBuf = new byte[2 * newKeyBuf.length];
+                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
+                }
+
+                System.arraycopy(splittedByteses[i + 1].value, 0, newKeyBuf, bufOffset, splittedByteses[i + 1].length);
+                bufOffset += splittedByteses[i + 1].length;
+            }
+        }
+        byte[] newKey = Arrays.copyOf(newKeyBuf, bufOffset);
+        outputKey.set(newKey, 0, newKey.length);
+
+        context.write(outputKey, value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
new file mode 100644
index 0000000..d99cb03
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeUpdate;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.DimensionDesc;
+import org.apache.kylin.dict.DictionaryInfo;
+import org.apache.kylin.dict.DictionaryManager;
+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.TblColRef;
+
+import java.io.IOException;
+import java.util.*;
+
+public class MergeDictionaryStep extends AbstractExecutable {
+
+    private static final String CUBE_NAME = "cubeName";
+    private static final String SEGMENT_ID = "segmentId";
+    private static final String MERGING_SEGMENT_IDS = "mergingSegmentIds";
+
+    public MergeDictionaryStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        KylinConfig conf = context.getConfig();
+        final CubeManager mgr = CubeManager.getInstance(conf);
+        final CubeInstance cube = mgr.getCube(getCubeName());
+        final CubeSegment newSegment = cube.getSegmentById(getSegmentId());
+        final List<CubeSegment> mergingSegments = getMergingSegments(cube);
+        
+        Collections.sort(mergingSegments);
+        
+        try {
+            checkLookupSnapshotsMustIncremental(mergingSegments);
+            
+            makeDictForNewSegment(conf, cube, newSegment, mergingSegments);
+            makeSnapshotForNewSegment(cube, newSegment, mergingSegments);
+
+            CubeUpdate cubeBuilder = new CubeUpdate(cube);
+            cubeBuilder.setToUpdateSegs(newSegment);
+            mgr.updateCube(cubeBuilder);
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        } catch (IOException e) {
+            logger.error("fail to merge dictionary or lookup snapshots", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+    
+    private List<CubeSegment> getMergingSegments(CubeInstance cube) {
+        List<String> mergingSegmentIds = getMergingSegmentIds();
+        List<CubeSegment> result = Lists.newArrayListWithCapacity(mergingSegmentIds.size());
+        for (String id : mergingSegmentIds) {
+            result.add(cube.getSegmentById(id));
+        }
+        return result;
+    }
+
+    private void checkLookupSnapshotsMustIncremental(List<CubeSegment> mergingSegments) {
+
+        // FIXME check each newer snapshot has only NEW rows but no MODIFIED rows
+    }
+
+    /**
+     * For the new segment, we need to create dictionaries for it, too. For
+     * those dictionaries on fact table, create it by merging underlying
+     * dictionaries For those dictionaries on lookup table, just copy it from
+     * any one of the merging segments, it's guaranteed to be consistent(checked
+     * in CubeSegmentValidator)
+     *
+     * @param cube
+     * @param newSeg
+     * @throws IOException
+     */
+    private void makeDictForNewSegment(KylinConfig conf, CubeInstance cube, CubeSegment newSeg, List<CubeSegment> mergingSegments) throws IOException {
+        HashSet<TblColRef> colsNeedMeringDict = new HashSet<TblColRef>();
+        HashSet<TblColRef> colsNeedCopyDict = new HashSet<TblColRef>();
+        DictionaryManager dictMgr = DictionaryManager.getInstance(conf);
+
+        CubeDesc cubeDesc = cube.getDescriptor();
+        for (DimensionDesc dim : cubeDesc.getDimensions()) {
+            for (TblColRef col : dim.getColumnRefs()) {
+                if (newSeg.getCubeDesc().getRowkey().isUseDictionary(col)) {
+                    String dictTable = (String) dictMgr.decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0];
+                    if (cubeDesc.getFactTable().equalsIgnoreCase(dictTable)) {
+                        colsNeedMeringDict.add(col);
+                    } else {
+                        colsNeedCopyDict.add(col);
+                    }
+                }
+            }
+        }
+
+        for (TblColRef col : colsNeedMeringDict) {
+            logger.info("Merging fact table dictionary on : " + col);
+            List<DictionaryInfo> dictInfos = new ArrayList<DictionaryInfo>();
+            for (CubeSegment segment : mergingSegments) {
+                logger.info("Including fact table dictionary of segment : " + segment);
+                if (segment.getDictResPath(col) != null) {
+                    DictionaryInfo dictInfo = dictMgr.getDictionaryInfo(segment.getDictResPath(col));
+                    dictInfos.add(dictInfo);
+                }
+            }
+            mergeDictionaries(dictMgr, newSeg, dictInfos, col);
+        }
+
+        for (TblColRef col : colsNeedCopyDict) {
+            String path = mergingSegments.get(0).getDictResPath(col);
+            newSeg.putDictResPath(col, path);
+        }
+    }
+
+    private DictionaryInfo mergeDictionaries(DictionaryManager dictMgr, CubeSegment cubeSeg, List<DictionaryInfo> dicts, TblColRef col) throws IOException {
+        DictionaryInfo dictInfo = dictMgr.mergeDictionary(dicts);
+        if (dictInfo != null)
+            cubeSeg.putDictResPath(col, dictInfo.getResourcePath());
+
+        return dictInfo;
+    }
+
+    /**
+     * make snapshots for the new segment by copying from one of the underlying
+     * merging segments. it's guaranteed to be consistent(checked in
+     * CubeSegmentValidator)
+     *
+     * @param cube
+     * @param newSeg
+     */
+    private void makeSnapshotForNewSegment(CubeInstance cube, CubeSegment newSeg, List<CubeSegment> mergingSegments) {
+        CubeSegment lastSeg = mergingSegments.get(mergingSegments.size() - 1);
+        for (Map.Entry<String, String> entry : lastSeg.getSnapshots().entrySet()) {
+            newSeg.putSnapshotResPath(entry.getKey(), entry.getValue());
+        }
+    }
+
+    public void setCubeName(String cubeName) {
+        this.setParam(CUBE_NAME, cubeName);
+    }
+
+    private String getCubeName() {
+        return getParam(CUBE_NAME);
+    }
+
+    public void setSegmentId(String segmentId) {
+        this.setParam(SEGMENT_ID, segmentId);
+    }
+
+    private String getSegmentId() {
+        return getParam(SEGMENT_ID);
+    }
+
+    public void setMergingSegmentIds(List<String> ids) {
+        setParam(MERGING_SEGMENT_IDS, StringUtils.join(ids, ","));
+    }
+
+    private List<String> getMergingSegmentIds() {
+        final String ids = getParam(MERGING_SEGMENT_IDS);
+        if (ids != null) {
+            final String[] splitted = StringUtils.split(ids, ",");
+            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
+            for (String id: splitted) {
+                result.add(id);
+            }
+            return result;
+        } else {
+            return Collections.emptyList();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
new file mode 100644
index 0000000..fc89e38
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+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 com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class MergeStatisticsStep extends AbstractExecutable {
+
+    private static final String CUBE_NAME = "cubeName";
+    private static final String SEGMENT_ID = "segmentId";
+    private static final String MERGING_SEGMENT_IS = "mergingSegmentIds";
+    private static final String MERGED_STATISTICS_PATH = "mergedStatisticsPath";
+    protected Map<Long, HyperLogLogPlusCounter> cuboidHLLMap = Maps.newHashMap();
+
+    public MergeStatisticsStep() {
+        super();
+    }
+
+    @Override
+    @SuppressWarnings("deprecation")
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        KylinConfig kylinConf = context.getConfig();
+        final CubeManager mgr = CubeManager.getInstance(kylinConf);
+        final CubeInstance cube = mgr.getCube(getCubeName());
+        final CubeSegment newSegment = cube.getSegmentById(getSegmentId());
+
+        Configuration conf = new Configuration();
+        ResourceStore rs = ResourceStore.getStore(kylinConf);
+        try {
+
+            int averageSamplingPercentage = 0;
+            for (String segmentId : this.getMergingSegmentIds()) {
+                String fileKey = CubeSegment.getStatisticsResourcePath(getCubeName(), segmentId);
+                InputStream is = rs.getResource(fileKey);
+                File tempFile = null;
+                FileOutputStream tempFileStream = null;
+                try {
+                    tempFile = File.createTempFile(segmentId, ".seq");
+                    tempFileStream = new FileOutputStream(tempFile);
+                    org.apache.commons.io.IOUtils.copy(is, tempFileStream);
+                } finally {
+                    IOUtils.closeStream(is);
+                    IOUtils.closeStream(tempFileStream);
+                }
+
+                FileSystem fs = HadoopUtil.getFileSystem("file:///" + tempFile.getAbsolutePath());
+                SequenceFile.Reader reader = null;
+                try {
+                    reader = new SequenceFile.Reader(fs, new Path(tempFile.getAbsolutePath()), conf);
+                    LongWritable key = (LongWritable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+                    BytesWritable value = (BytesWritable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
+                    while (reader.next(key, value)) {
+                        if (key.get() == 0l) {
+                            // sampling percentage;
+                            averageSamplingPercentage += Bytes.toInt(value.getBytes());
+                        } else {
+                            HyperLogLogPlusCounter hll = new HyperLogLogPlusCounter(14);
+                            ByteArray byteArray = new ByteArray(value.getBytes());
+                            hll.readRegisters(byteArray.asBuffer());
+
+                            if (cuboidHLLMap.get(key.get()) != null) {
+                                cuboidHLLMap.get(key.get()).merge(hll);
+                            } else {
+                                cuboidHLLMap.put(key.get(), hll);
+                            }
+                        }
+                    }
+                } catch (Exception e) {
+                    e.printStackTrace();
+                    throw e;
+                } finally {
+                    IOUtils.closeStream(reader);
+                }
+            }
+            averageSamplingPercentage = averageSamplingPercentage / this.getMergingSegmentIds().size();
+            FactDistinctColumnsReducer.writeCuboidStatistics(conf, new Path(getMergedStatisticsPath()), cuboidHLLMap, averageSamplingPercentage);
+            Path statisticsFilePath = new Path(getMergedStatisticsPath(), BatchConstants.CFG_STATISTICS_CUBOID_ESTIMATION);
+            FileSystem fs = statisticsFilePath.getFileSystem(conf);
+            FSDataInputStream is = fs.open(statisticsFilePath);
+            try {
+                // put the statistics to metadata store
+                String statisticsFileName = newSegment.getStatisticsResourcePath();
+                rs.putResource(statisticsFileName, is, System.currentTimeMillis());
+            } finally {
+                IOUtils.closeStream(is);
+            }
+
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        } catch (IOException e) {
+            logger.error("fail to merge cuboid statistics", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+
+
+    public void setCubeName(String cubeName) {
+        this.setParam(CUBE_NAME, cubeName);
+    }
+
+    private String getCubeName() {
+        return getParam(CUBE_NAME);
+    }
+
+    public void setSegmentId(String segmentId) {
+        this.setParam(SEGMENT_ID, segmentId);
+    }
+
+    private String getSegmentId() {
+        return getParam(SEGMENT_ID);
+    }
+
+    public void setMergingSegmentIds(List<String> ids) {
+        setParam(MERGING_SEGMENT_IS, StringUtils.join(ids, ","));
+    }
+
+    private List<String> getMergingSegmentIds() {
+        final String ids = getParam(MERGING_SEGMENT_IS);
+        if (ids != null) {
+            final String[] splitted = StringUtils.split(ids, ",");
+            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
+            for (String id : splitted) {
+                result.add(id);
+            }
+            return result;
+        } else {
+            return Collections.emptyList();
+        }
+    }
+
+    public void setMergedStatisticsPath(String path) {
+        setParam(MERGED_STATISTICS_PATH, path);
+    }
+
+    private String getMergedStatisticsPath() {
+        return getParam(MERGED_STATISTICS_PATH);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MetadataCleanupJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MetadataCleanupJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MetadataCleanupJob.java
new file mode 100644
index 0000000..62dd343
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MetadataCleanupJob.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import com.google.common.collect.Lists;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ */
+public class MetadataCleanupJob extends AbstractHadoopJob {
+
+    @SuppressWarnings("static-access")
+    private static final Option OPTION_DELETE = OptionBuilder.withArgName("delete").hasArg().isRequired(false).withDescription("Delete the unused metadata").create("delete");
+
+    protected static final Logger log = LoggerFactory.getLogger(MetadataCleanupJob.class);
+
+    boolean delete = false;
+
+    private KylinConfig config = null;
+
+    public static final long TIME_THREADSHOLD = 2 * 26 * 3600 * 1000l; // 2 days
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
+     */
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        log.info("----- jobs args: " + Arrays.toString(args));
+        try {
+            options.addOption(OPTION_DELETE);
+            parseOptions(options, args);
+
+            log.info("options: '" + getOptionsAsString() + "'");
+            log.info("delete option value: '" + getOptionValue(OPTION_DELETE) + "'");
+            delete = Boolean.parseBoolean(getOptionValue(OPTION_DELETE));
+
+            config = KylinConfig.getInstanceFromEnv();
+
+            cleanup();
+
+            return 0;
+        } catch (Exception e) {
+            e.printStackTrace(System.err);
+            throw e;
+        }
+    }
+
+    private ResourceStore getStore() {
+        return ResourceStore.getStore(config);
+    }
+
+    private boolean isOlderThanThreshold(long resourceTime) {
+        long currentTime = System.currentTimeMillis();
+
+        if (currentTime - resourceTime > TIME_THREADSHOLD)
+            return true;
+        return false;
+    }
+
+    public void cleanup() throws Exception {
+        CubeManager cubeManager = CubeManager.getInstance(config);
+
+        List<String> activeResourceList = Lists.newArrayList();
+        for (org.apache.kylin.cube.CubeInstance cube : cubeManager.listAllCubes()) {
+            for (org.apache.kylin.cube.CubeSegment segment : cube.getSegments()) {
+                activeResourceList.addAll(segment.getSnapshotPaths());
+                activeResourceList.addAll(segment.getDictionaryPaths());
+                activeResourceList.add(segment.getStatisticsResourcePath());
+            }
+        }
+
+        List<String> toDeleteResource = Lists.newArrayList();
+
+        // two level resources, snapshot tables and cube statistics
+        for (String resourceRoot : new String[]{ResourceStore.SNAPSHOT_RESOURCE_ROOT, ResourceStore.CUBE_STATISTICS_ROOT}) {
+            ArrayList<String> snapshotTables = getStore().listResources(resourceRoot);
+
+            for (String snapshotTable : snapshotTables) {
+                ArrayList<String> snapshotNames = getStore().listResources(snapshotTable);
+                if (snapshotNames != null)
+                    for (String snapshot : snapshotNames) {
+                        if (!activeResourceList.contains(snapshot)) {
+                            if (isOlderThanThreshold(getStore().getResourceTimestamp(snapshot)))
+                                toDeleteResource.add(snapshot);
+                        }
+                    }
+            }
+        }
+
+        // three level resources, only dictionaries
+        ArrayList<String> dictTables = getStore().listResources(ResourceStore.DICT_RESOURCE_ROOT);
+
+        for (String table : dictTables) {
+            ArrayList<String> tableColNames = getStore().listResources(table);
+            if (tableColNames != null)
+                for (String tableCol : tableColNames) {
+                    ArrayList<String> dictionaries = getStore().listResources(tableCol);
+                    if (dictionaries != null)
+                        for (String dict : dictionaries)
+                            if (!activeResourceList.contains(dict)) {
+                                if (isOlderThanThreshold(getStore().getResourceTimestamp(dict)))
+                                    toDeleteResource.add(dict);
+                            }
+                }
+        }
+
+
+        if (toDeleteResource.size() > 0) {
+            logger.info("The following resources have no reference, will be cleaned from metadata store: \n");
+
+            for (String s : toDeleteResource) {
+                logger.info(s);
+                if (delete == true) {
+                    getStore().deleteResource(s);
+                }
+            }
+        } else {
+            logger.info("No resource to be cleaned up from metadata store;");
+        }
+
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new MetadataCleanupJob(), args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidJob.java
new file mode 100644
index 0000000..40c4dd7
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidJob.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+
+public class NDCuboidJob extends CuboidJob {
+
+    public NDCuboidJob() {
+        this.setMapperClass(NDCuboidMapper.class);
+    }
+
+    public static void main(String[] args) throws Exception {
+        CuboidJob job = new NDCuboidJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
new file mode 100644
index 0000000..af92fc5
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.common.RowKeySplitter;
+import org.apache.kylin.common.util.SplittedBytes;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
+
+    private static final Logger logger = LoggerFactory.getLogger(NDCuboidMapper.class);
+
+    private Text outputKey = new Text();
+    private String cubeName;
+    private String segmentName;
+    private CubeDesc cubeDesc;
+    private CuboidScheduler cuboidScheduler;
+
+    private int handleCounter;
+    private int skipCounter;
+
+    private byte[] keyBuf = new byte[4096];
+    private RowKeySplitter rowKeySplitter;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME).toUpperCase();
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        CubeSegment cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
+        cubeDesc = cube.getDescriptor();
+
+        // initialize CubiodScheduler
+        cuboidScheduler = new CuboidScheduler(cubeDesc);
+
+        rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 256);
+    }
+
+    private int buildKey(Cuboid parentCuboid, Cuboid childCuboid, SplittedBytes[] splitBuffers) {
+        int offset = 0;
+
+        // cuboid id
+        System.arraycopy(childCuboid.getBytes(), 0, keyBuf, offset, childCuboid.getBytes().length);
+        offset += childCuboid.getBytes().length;
+
+        // rowkey columns
+        long mask = Long.highestOneBit(parentCuboid.getId());
+        long parentCuboidId = parentCuboid.getId();
+        long childCuboidId = childCuboid.getId();
+        long parentCuboidIdActualLength = Long.SIZE - Long.numberOfLeadingZeros(parentCuboid.getId());
+        int index = 1; // skip cuboidId
+        for (int i = 0; i < parentCuboidIdActualLength; i++) {
+            if ((mask & parentCuboidId) > 0) {// if the this bit position equals
+                                              // 1
+                if ((mask & childCuboidId) > 0) {// if the child cuboid has this
+                                                 // column
+                    System.arraycopy(splitBuffers[index].value, 0, keyBuf, offset, splitBuffers[index].length);
+                    offset += splitBuffers[index].length;
+                }
+                index++;
+            }
+            mask = mask >> 1;
+        }
+
+        return offset;
+    }
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        long cuboidId = rowKeySplitter.split(key.getBytes(), key.getLength());
+        Cuboid parentCuboid = Cuboid.findById(cubeDesc, cuboidId);
+
+        Collection<Long> myChildren = cuboidScheduler.getSpanningCuboid(cuboidId);
+
+        // if still empty or null
+        if (myChildren == null || myChildren.size() == 0) {
+            context.getCounter(BatchConstants.MAPREDUCE_COUTNER_GROUP_NAME, "Skipped records").increment(1L);
+            skipCounter++;
+            if (skipCounter % BatchConstants.COUNTER_MAX == 0) {
+                logger.info("Skipped " + skipCounter + " records!");
+            }
+            return;
+        }
+
+        context.getCounter(BatchConstants.MAPREDUCE_COUTNER_GROUP_NAME, "Processed records").increment(1L);
+
+        handleCounter++;
+        if (handleCounter % BatchConstants.COUNTER_MAX == 0) {
+            logger.info("Handled " + handleCounter + " records!");
+        }
+
+        for (Long child : myChildren) {
+            Cuboid childCuboid = Cuboid.findById(cubeDesc, child);
+            int keyLength = buildKey(parentCuboid, childCuboid, rowKeySplitter.getSplitBuffers());
+            outputKey.set(keyBuf, 0, keyLength);
+            context.write(outputKey, value);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionJob.java
new file mode 100644
index 0000000..004d863
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionJob.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
+
+/**
+ * @author xjiang, ysong1
+ * 
+ */
+
+public class RangeKeyDistributionJob extends AbstractHadoopJob {
+    protected static final Logger log = LoggerFactory.getLogger(RangeKeyDistributionJob.class);
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
+     */
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+
+            parseOptions(options, args);
+
+            // start job
+            String jobName = getOptionValue(OPTION_JOB_NAME);
+            job = Job.getInstance(getConf(), jobName);
+
+            setJobClasspath(job);
+
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            FileOutputFormat.setOutputPath(job, output);
+            // job.getConfiguration().set("dfs.block.size", "67108864");
+
+            // Mapper
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapperClass(RangeKeyDistributionMapper.class);
+            job.setMapOutputKeyClass(Text.class);
+            job.setMapOutputValueClass(LongWritable.class);
+
+            // Reducer - only one
+            job.setReducerClass(RangeKeyDistributionReducer.class);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(LongWritable.class);
+            job.setNumReduceTasks(1);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            RealizationCapacity realizationCapacity = cube.getDescriptor().getModel().getCapacity();
+            job.getConfiguration().set(BatchConstants.CUBE_CAPACITY, realizationCapacity.toString());
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new RangeKeyDistributionJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionMapper.java
new file mode 100644
index 0000000..e6fb173
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionMapper.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+
+import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionMapper extends KylinMapper<Text, Text, Text, LongWritable> {
+
+    private static final long ONE_MEGA_BYTES = 1L * 1024L * 1024L;
+
+    private LongWritable outputValue = new LongWritable(0);
+
+    private long bytesRead = 0;
+
+    private Text lastKey;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+    }
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        lastKey = key;
+
+        int bytesLength = key.getLength() + value.getLength();
+        bytesRead += bytesLength;
+
+        if (bytesRead >= ONE_MEGA_BYTES) {
+            outputValue.set(bytesRead);
+            context.write(key, outputValue);
+
+            // reset bytesRead
+            bytesRead = 0;
+        }
+
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        if (lastKey != null) {
+            outputValue.set(bytesRead);
+            context.write(lastKey, outputValue);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionReducer.java
new file mode 100644
index 0000000..2e20675
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionReducer.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.engine.mr.steps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.kylin.cube.model.v1.CubeDesc.CubeCapacity;
+import org.apache.kylin.engine.mr.KylinReducer;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable, Text, LongWritable> {
+
+    public static final long ONE_GIGA_BYTES = 1024L * 1024L * 1024L;
+    public static final int SMALL_CUT = 5;  //  5 GB per region
+    public static final int MEDIUM_CUT = 10; //  10 GB per region
+    public static final int LARGE_CUT = 50; // 50 GB per region
+    
+    public static final int MAX_REGION = 1000;
+
+    private static final Logger logger = LoggerFactory.getLogger(RangeKeyDistributionReducer.class);
+
+    private LongWritable outputValue = new LongWritable(0);
+
+    private int cut;
+    private long bytesRead = 0;
+    private List<Text> gbPoints = new ArrayList<Text>();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+        CubeCapacity cubeCapacity = CubeCapacity.valueOf(context.getConfiguration().get(BatchConstants.CUBE_CAPACITY));
+        switch (cubeCapacity) {
+        case SMALL:
+            cut = SMALL_CUT;
+            break;
+        case MEDIUM:
+            cut = MEDIUM_CUT;
+            break;
+        case LARGE:
+            cut = LARGE_CUT;
+            break;
+        }
+
+        logger.info("Chosen cut for htable is " + cut);
+    }
+
+    @Override
+    public void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
+        for (LongWritable v : values) {
+            bytesRead += v.get();
+        }
+        
+        if (bytesRead >= ONE_GIGA_BYTES) {
+            gbPoints.add(new Text(key));
+            bytesRead = 0; // reset bytesRead
+        }
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        int nRegion = Math.round((float) gbPoints.size() / (float) cut);
+        nRegion = Math.max(1,  nRegion);
+        nRegion = Math.min(MAX_REGION, nRegion);
+        
+        int gbPerRegion = gbPoints.size() / nRegion;
+        gbPerRegion = Math.max(1, gbPerRegion);
+        
+        System.out.println(nRegion + " regions");
+        System.out.println(gbPerRegion + " GB per region");
+        
+        for (int i = gbPerRegion; i < gbPoints.size(); i += gbPerRegion) {
+            Text key = gbPoints.get(i);
+            outputValue.set(i);
+            System.out.println(StringUtils.byteToHexString(key.getBytes()) + "\t" + outputValue.get());
+            context.write(key, outputValue);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerJob.java
new file mode 100644
index 0000000..41712d9
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerJob.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RowKeyDistributionCheckerJob extends AbstractHadoopJob {
+
+    @SuppressWarnings("static-access")
+    protected static final Option rowKeyStatsFilePath = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("rowKeyStatsFilePath").create("rowKeyStatsFilePath");
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(rowKeyStatsFilePath);
+
+            parseOptions(options, args);
+
+            String statsFilePath = getOptionValue(rowKeyStatsFilePath);
+
+            // start job
+            String jobName = getOptionValue(OPTION_JOB_NAME);
+            job = Job.getInstance(getConf(), jobName);
+
+            setJobClasspath(job);
+
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            FileOutputFormat.setOutputPath(job, output);
+
+            // Mapper
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapperClass(RowKeyDistributionCheckerMapper.class);
+            job.setMapOutputKeyClass(Text.class);
+            job.setMapOutputValueClass(LongWritable.class);
+
+            // Reducer - only one
+            job.setReducerClass(RowKeyDistributionCheckerReducer.class);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(LongWritable.class);
+            job.setNumReduceTasks(1);
+
+            job.getConfiguration().set("rowKeyStatsFilePath", statsFilePath);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new RowKeyDistributionCheckerJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerMapper.java
new file mode 100644
index 0000000..aa25f2c
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerMapper.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.kylin.engine.mr.KylinMapper;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RowKeyDistributionCheckerMapper extends KylinMapper<Text, Text, Text, LongWritable> {
+
+    String rowKeyStatsFilePath;
+    byte[][] splitKeys;
+    Map<Text, Long> resultMap;
+    List<Text> keyList;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+        rowKeyStatsFilePath = context.getConfiguration().get("rowKeyStatsFilePath");
+        splitKeys = this.getSplits(context.getConfiguration(), new Path(rowKeyStatsFilePath));
+
+        resultMap = new HashMap<Text, Long>();
+        keyList = new ArrayList<Text>();
+        for (int i = 0; i < splitKeys.length; i++) {
+            Text key = new Text(splitKeys[i]);
+            resultMap.put(key, 0L);
+            keyList.add(new Text(splitKeys[i]));
+        }
+    }
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        for (Text t : keyList) {
+            if (key.compareTo(t) < 0) {
+                Long v = resultMap.get(t);
+                long length = key.getLength() + value.getLength();
+                v += length;
+                resultMap.put(t, v);
+                break;
+            }
+        }
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        LongWritable outputValue = new LongWritable();
+        for (Entry<Text, Long> kv : resultMap.entrySet()) {
+            outputValue.set(kv.getValue());
+            context.write(kv.getKey(), outputValue);
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    public byte[][] getSplits(Configuration conf, Path path) {
+        List<byte[]> rowkeyList = new ArrayList<byte[]>();
+        SequenceFile.Reader reader = null;
+        try {
+            reader = new SequenceFile.Reader(path.getFileSystem(conf), path, conf);
+            Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+            Writable value = (Writable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
+            while (reader.next(key, value)) {
+                byte[] tmp = ((Text) key).copyBytes();
+                if (rowkeyList.contains(tmp) == false) {
+                    rowkeyList.add(tmp);
+                }
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+        } finally {
+            IOUtils.closeStream(reader);
+        }
+
+        byte[][] retValue = rowkeyList.toArray(new byte[rowkeyList.size()][]);
+
+        return retValue;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerReducer.java
new file mode 100644
index 0000000..332cba5
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RowKeyDistributionCheckerReducer.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.engine.mr.KylinReducer;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RowKeyDistributionCheckerReducer extends KylinReducer<Text, LongWritable, Text, LongWritable> {
+
+    LongWritable outputKey = new LongWritable(0L);
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+    }
+
+    @Override
+    public void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
+
+        long length = 0;
+        for (LongWritable v : values) {
+            length += v.get();
+        }
+
+        outputKey.set(length);
+        context.write(key, outputKey);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
new file mode 100644
index 0000000..b2f3b3b
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+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;
+
+/**
+ * Save the cube segment statistic to Kylin metadata store
+ *
+ */
+public class SaveStatisticsStep extends AbstractExecutable {
+
+    private static final String CUBE_NAME = "cubeName";
+    private static final String SEGMENT_ID = "segmentId";
+    private static final String STATISTICS_PATH = "statisticsPath";
+
+    public SaveStatisticsStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        KylinConfig kylinConf = context.getConfig();
+        final CubeManager mgr = CubeManager.getInstance(kylinConf);
+        final CubeInstance cube = mgr.getCube(getCubeName());
+        final CubeSegment newSegment = cube.getSegmentById(getSegmentId());
+
+        ResourceStore rs = ResourceStore.getStore(kylinConf);
+        try {
+            Path statisticsFilePath = new Path(getStatisticsPath(), BatchConstants.CFG_STATISTICS_CUBOID_ESTIMATION);
+            FileSystem fs = FileSystem.get(HadoopUtil.getCurrentConfiguration());
+            if (!fs.exists(statisticsFilePath))
+                throw new IOException("File " + statisticsFilePath + " does not exists;");
+
+            FSDataInputStream is = fs.open(statisticsFilePath);
+            try {
+                // put the statistics to metadata store
+                String statisticsFileName = newSegment.getStatisticsResourcePath();
+                rs.putResource(statisticsFileName, is, System.currentTimeMillis());
+                fs.delete(statisticsFilePath, false);
+            } finally {
+                IOUtils.closeStream(is);
+            }
+
+
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        } catch (IOException e) {
+            logger.error("fail to save cuboid statistics", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+
+
+    public void setCubeName(String cubeName) {
+        this.setParam(CUBE_NAME, cubeName);
+    }
+
+    private String getCubeName() {
+        return getParam(CUBE_NAME);
+    }
+
+    public void setSegmentId(String segmentId) {
+        this.setParam(SEGMENT_ID, segmentId);
+    }
+
+    private String getSegmentId() {
+        return getParam(SEGMENT_ID);
+    }
+
+    public void setStatisticsPath(String path) {
+        this.setParam(STATISTICS_PATH, path);
+    }
+
+    private String getStatisticsPath() {
+        return getParam(STATISTICS_PATH);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterBuildStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterBuildStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterBuildStep.java
new file mode 100644
index 0000000..dd99a64
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterBuildStep.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+
+import org.apache.kylin.cube.CubeUpdate;
+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.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+
+/**
+ */
+public class UpdateCubeInfoAfterBuildStep extends AbstractExecutable {
+
+    private static final String SEGMENT_ID = "segmentId";
+    private static final String CUBE_NAME = "cubeName";
+    private static final String CUBING_JOB_ID = "cubingJobId";
+
+    public UpdateCubeInfoAfterBuildStep() {
+        super();
+    }
+
+    public void setCubeName(String cubeName) {
+        this.setParam(CUBE_NAME, cubeName);
+    }
+
+    private String getCubeName() {
+        return getParam(CUBE_NAME);
+    }
+
+    public void setSegmentId(String segmentId) {
+        this.setParam(SEGMENT_ID, segmentId);
+    }
+
+    private String getSegmentId() {
+        return getParam(SEGMENT_ID);
+    }
+
+    public void setCubingJobId(String id) {
+        setParam(CUBING_JOB_ID, id);
+    }
+
+    private String getCubingJobId() {
+        return getParam(CUBING_JOB_ID);
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final CubeManager cubeManager = CubeManager.getInstance(context.getConfig());
+        final CubeInstance cube = cubeManager.getCube(getCubeName());
+        final CubeSegment segment = cube.getSegmentById(getSegmentId());
+        
+        CubingJob cubingJob = (CubingJob) executableManager.getJob(getCubingJobId());
+        long sourceCount = cubingJob.findSourceRecordCount();
+        long sourceSizeBytes = cubingJob.findSourceSizeBytes();
+        long cubeSizeBytes = cubingJob.findCubeSizeBytes();
+        boolean segmentReady = cubeSizeBytes > 0; // for build+merge scenario, convert HFile not happen yet, so cube size is 0
+
+        segment.setLastBuildJobID(getCubingJobId());
+        segment.setLastBuildTime(System.currentTimeMillis());
+        segment.setSizeKB(cubeSizeBytes / 1024);
+        segment.setInputRecords(sourceCount);
+        segment.setInputRecordsSize(sourceSizeBytes);
+
+        try {
+            if (segmentReady) {
+                cubeManager.promoteNewlyBuiltSegments(cube, segment);
+            } else {
+                CubeUpdate cubeBuilder = new CubeUpdate(cube);
+                cubeBuilder.setToUpdateSegs(segment);
+                cubeManager.updateCube(cubeBuilder);
+            }
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        } catch (IOException e) {
+            logger.error("fail to update cube after build", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterMergeStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterMergeStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterMergeStep.java
new file mode 100644
index 0000000..d237908
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterMergeStep.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.steps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+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.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 com.google.common.collect.Lists;
+
+/**
+ */
+public class UpdateCubeInfoAfterMergeStep extends AbstractExecutable {
+
+    private static final String CUBE_NAME = "cubeName";
+    private static final String SEGMENT_ID = "segmentId";
+    private static final String MERGING_SEGMENT_IDS = "mergingSegmentIds";
+    private static final String CUBING_JOB_ID = "cubingJobId";
+
+    private final CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+    public UpdateCubeInfoAfterMergeStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final CubeInstance cube = cubeManager.getCube(getCubeName());
+
+        CubeSegment mergedSegment = cube.getSegmentById(getSegmentId());
+        if (mergedSegment == null) {
+            return new ExecuteResult(ExecuteResult.State.FAILED, "there is no segment with id:" + getSegmentId());
+        }
+        
+        CubingJob cubingJob = (CubingJob) executableManager.getJob(getCubingJobId());
+        long cubeSizeBytes = cubingJob.findCubeSizeBytes();
+
+        // collect source statistics
+        List<String> mergingSegmentIds = getMergingSegmentIds();
+        if (mergingSegmentIds.isEmpty()) {
+            return new ExecuteResult(ExecuteResult.State.FAILED, "there are no merging segments");
+        }
+        long sourceCount = 0L;
+        long sourceSize = 0L;
+        for (String id : mergingSegmentIds) {
+            CubeSegment segment = cube.getSegmentById(id);
+            sourceCount += segment.getInputRecords();
+            sourceSize += segment.getInputRecordsSize();
+        }
+
+        // update segment info
+        mergedSegment.setSizeKB(cubeSizeBytes / 1024);
+        mergedSegment.setInputRecords(sourceCount);
+        mergedSegment.setInputRecordsSize(sourceSize);
+        mergedSegment.setLastBuildJobID(getCubingJobId());
+        mergedSegment.setLastBuildTime(System.currentTimeMillis());
+
+        try {
+            cubeManager.promoteNewlyBuiltSegments(cube, mergedSegment);
+            return new ExecuteResult(ExecuteResult.State.SUCCEED);
+        } catch (IOException e) {
+            logger.error("fail to update cube after merge", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+
+    public void setSegmentId(String segmentId) {
+        this.setParam(SEGMENT_ID, segmentId);
+    }
+
+    private String getSegmentId() {
+        return getParam(SEGMENT_ID);
+    }
+
+    public void setCubeName(String cubeName) {
+        this.setParam(CUBE_NAME, cubeName);
+    }
+
+    private String getCubeName() {
+        return getParam(CUBE_NAME);
+    }
+
+    public void setMergingSegmentIds(List<String> ids) {
+        setParam(MERGING_SEGMENT_IDS, StringUtils.join(ids, ","));
+    }
+
+    private List<String> getMergingSegmentIds() {
+        final String ids = getParam(MERGING_SEGMENT_IDS);
+        if (ids != null) {
+            final String[] splitted = StringUtils.split(ids, ",");
+            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
+            for (String id : splitted) {
+                result.add(id);
+            }
+            return result;
+        } else {
+            return Collections.emptyList();
+        }
+    }
+
+    public void setCubingJobId(String id) {
+        setParam(CUBING_JOB_ID, id);
+    }
+
+    private String getCubingJobId() {
+        return getParam(CUBING_JOB_ID);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
new file mode 100644
index 0000000..bc0e002
--- /dev/null
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.steps.CuboidReducer;
+import org.apache.kylin.metadata.measure.LongMutable;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ */
+public class CubeReducerTest extends LocalFileMetadataTestCase {
+
+    ReduceDriver<Text, Text, Text, Text> reduceDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+    @Before
+    public void setUp() throws Exception {
+        createTestMetadata();
+
+        // hack for distributed cache
+        FileUtils.deleteDirectory(new File("../job/meta"));
+        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), new File("../job/meta"));
+
+        CuboidReducer reducer = new CuboidReducer();
+        reduceDriver = ReduceDriver.newReduceDriver(reducer);
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+        FileUtils.deleteDirectory(new File("../job/meta"));
+    }
+
+    @Test
+    public void testReducer() throws Exception {
+
+        reduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, "test_kylin_cube_with_slr_ready");
+
+        CubeDesc cubeDesc = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_ready").getDescriptor();
+        MeasureCodec codec = new MeasureCodec(cubeDesc.getMeasures());
+
+        Text key1 = new Text("72010ustech");
+        List<Text> values1 = new ArrayList<Text>();
+        values1.add(newValueText(codec, "15.09", "15.09", "15.09", 1));
+        values1.add(newValueText(codec, "20.34", "20.34", "20.34", 1));
+        values1.add(newValueText(codec, "10", "10", "10", 1));
+
+        Text key2 = new Text("1tech");
+        List<Text> values2 = new ArrayList<Text>();
+        values2.add(newValueText(codec, "15.09", "15.09", "15.09", 1));
+        values2.add(newValueText(codec, "20.34", "20.34", "20.34", 1));
+
+        Text key3 = new Text("0");
+        List<Text> values3 = new ArrayList<Text>();
+        values3.add(newValueText(codec, "146.52", "146.52", "146.52", 4));
+
+        reduceDriver.withInput(key1, values1);
+        reduceDriver.withInput(key2, values2);
+        reduceDriver.withInput(key3, values3);
+
+        List<Pair<Text, Text>> result = reduceDriver.run();
+
+        Pair<Text, Text> p1 = new Pair<Text, Text>(new Text("72010ustech"), newValueText(codec, "45.43", "10", "20.34", 3));
+        Pair<Text, Text> p2 = new Pair<Text, Text>(new Text("1tech"), newValueText(codec, "35.43", "15.09", "20.34", 2));
+        Pair<Text, Text> p3 = new Pair<Text, Text>(new Text("0"), newValueText(codec, "146.52", "146.52", "146.52", 4));
+
+        assertEquals(3, result.size());
+
+        assertTrue(result.contains(p1));
+        assertTrue(result.contains(p2));
+        assertTrue(result.contains(p3));
+    }
+
+    private Text newValueText(MeasureCodec codec, String sum, String min, String max, int count) {
+        Object[] values = new Object[] { new BigDecimal(sum), new BigDecimal(min), new BigDecimal(max), new LongMutable(count) };
+
+        buf.clear();
+        codec.encode(values, buf);
+
+        Text t = new Text();
+        t.set(buf.array(), 0, buf.position());
+        return t;
+    }
+
+}


[06/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/RandomKeyDistributionReducerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RandomKeyDistributionReducerTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/RandomKeyDistributionReducerTest.java
deleted file mode 100644
index 8e85a27..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RandomKeyDistributionReducerTest.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.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.invertedindex.RandomKeyDistributionReducer;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * @author ysong1
- * 
- */
-public class RandomKeyDistributionReducerTest {
-    ReduceDriver<Text, LongWritable, Text, LongWritable> reduceDriver;
-
-    @Before
-    @SuppressWarnings({ "rawtypes", "unchecked" })
-    public void setUp() {
-        RandomKeyDistributionReducer reducer = new RandomKeyDistributionReducer();
-        reduceDriver = ReduceDriver.newReduceDriver(reducer);
-    }
-
-    @Test
-    public void test() throws IOException {
-        List<Text> data = new ArrayList<Text>();
-        for (int i = 0; i < 1001; i++) {
-            data.add(new Text(String.valueOf(i)));
-        }
-        for (Text t : data) {
-            reduceDriver.addInput(t, new ArrayList<LongWritable>());
-        }
-
-        reduceDriver.getConfiguration().set(BatchConstants.REGION_NUMBER, "2");
-        List<Pair<Text, LongWritable>> result = reduceDriver.run();
-
-        assertEquals(2, result.size());
-
-        for (Pair<Text, LongWritable> p : result) {
-            System.out.println(p.getFirst());
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
deleted file mode 100644
index b3bcc30..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-
-/**
- * @author ysong1
- * 
- */
-public class RangeKeyDistributionJobTest extends LocalFileMetadataTestCase {
-
-    private Configuration conf;
-
-    @Before
-    public void setup() throws Exception {
-        conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-
-        // for local runner out-of-memory issue
-        conf.set("mapreduce.task.io.sort.mb", "10");
-        createTestMetadata();
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-    }
-
-    @Test
-    public void testJob() throws Exception {
-        String input = "src/test/resources/data/base_cuboid/,src/test/resources/data/6d_cuboid/";
-        String output = "target/test-output/key_distribution_range/";
-        String jobname = "calculate_splits";
-        String cubename = "test_kylin_cube_with_slr_ready";
-
-        FileUtil.fullyDelete(new File(output));
-
-        String[] args = { "-input", input, "-output", output, "-jobname", jobname, "-cubename", cubename };
-        assertEquals("Job failed", 0, ToolRunner.run(conf, new RangeKeyDistributionJob(), args));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapperTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapperTest.java
deleted file mode 100644
index 550fd6b..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionMapperTest.java
+++ /dev/null
@@ -1,111 +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.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * @author ysong1
- * 
- */
-public class RangeKeyDistributionMapperTest {
-
-    @SuppressWarnings("rawtypes")
-    MapDriver mapDriver;
-    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
-
-    @Before
-    public void setUp() {
-        RangeKeyDistributionMapper mapper = new RangeKeyDistributionMapper();
-        mapDriver = MapDriver.newMapDriver(mapper);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testMapperWithoutHeader() throws IOException {
-
-        Text inputKey1 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey2 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey3 = new Text(new byte[] { 2, 2, 2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey4 = new Text(new byte[] { 3, 3, 3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey5 = new Text(new byte[] { 4, 4, 4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey6 = new Text(new byte[] { 5, 5, 5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey7 = new Text(new byte[] { 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-
-        mapDriver.addInput(inputKey1, new Text("abc"));
-        mapDriver.addInput(inputKey2, new Text("abc"));
-        mapDriver.addInput(inputKey3, new Text("abc"));
-        mapDriver.addInput(inputKey4, new Text("abc"));
-        mapDriver.addInput(inputKey5, new Text("abc"));
-        mapDriver.addInput(inputKey6, new Text("abc"));
-        mapDriver.addInput(inputKey7, new Text("abc"));
-
-        List<Pair<Text, LongWritable>> result = mapDriver.run();
-
-        assertEquals(1, result.size());
-
-        byte[] key1 = result.get(0).getFirst().getBytes();
-        LongWritable value1 = result.get(0).getSecond();
-        assertArrayEquals(new byte[] { 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 }, key1);
-        assertEquals(147, value1.get());
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testMapperWithHeader() throws IOException {
-
-        Text inputKey1 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey2 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0, 0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey3 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 2, 2, 2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey4 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 3, 3, 3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey5 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 4, 4, 4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey6 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 5, 5, 5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-        Text inputKey7 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-
-        mapDriver.addInput(inputKey1, new Text("abc"));
-        mapDriver.addInput(inputKey2, new Text("abc"));
-        mapDriver.addInput(inputKey3, new Text("abc"));
-        mapDriver.addInput(inputKey4, new Text("abc"));
-        mapDriver.addInput(inputKey5, new Text("abc"));
-        mapDriver.addInput(inputKey6, new Text("abc"));
-        mapDriver.addInput(inputKey7, new Text("abc"));
-
-        List<Pair<Text, LongWritable>> result = mapDriver.run();
-
-        assertEquals(1, result.size());
-
-        byte[] key1 = result.get(0).getFirst().getBytes();
-        LongWritable value1 = result.get(0).getSecond();
-        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 }, key1);
-        assertEquals(273, value1.get());
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducerTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducerTest.java
deleted file mode 100644
index 38fb132..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducerTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * @author ysong1
- * 
- */
-public class RangeKeyDistributionReducerTest {
-
-    ReduceDriver<Text, LongWritable, Text, LongWritable> reduceDriver;
-    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
-
-    @Before
-    public void setUp() {
-        RangeKeyDistributionReducer reducer = new RangeKeyDistributionReducer();
-        reduceDriver = ReduceDriver.newReduceDriver(reducer);
-    }
-
-    @Test
-    public void testReducer() throws IOException {
-        // TODO
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/CubeSamplingTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/CubeSamplingTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/CubeSamplingTest.java
deleted file mode 100644
index 386d858..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/CubeSamplingTest.java
+++ /dev/null
@@ -1,134 +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.cubev2;
-
-import com.google.common.collect.Lists;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hasher;
-import com.google.common.hash.Hashing;
-import org.apache.commons.lang.RandomStringUtils;
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-
-/**
- */
-public class CubeSamplingTest {
-
-    private static final int ROW_LENGTH = 10;
-
-    private final List<String> row = new ArrayList<String>(ROW_LENGTH);
-    private final ByteArray[] row_index = new ByteArray[ROW_LENGTH];
-
-    private Integer[][] allCuboidsBitSet;
-    private HashFunction hf = null;
-    private long baseCuboidId;
-    private HyperLogLogPlusCounter[] allCuboidsHLL = null;
-    private final byte[] seperator = Bytes.toBytes(",");
-
-    @Before
-    public void setup() {
-
-        baseCuboidId = (1l << ROW_LENGTH) - 1;
-        List<Long> allCuboids = Lists.newArrayList();
-        List<Integer[]> allCuboidsBitSetList = Lists.newArrayList();
-        for (long i = 1; i < baseCuboidId; i++) {
-            allCuboids.add(i);
-            addCuboidBitSet(i, allCuboidsBitSetList);
-        }
-
-        allCuboidsBitSet = allCuboidsBitSetList.toArray(new Integer[allCuboidsBitSetList.size()][]);
-        System.out.println("Totally have " + allCuboidsBitSet.length + " cuboids.");
-        allCuboidsHLL = new HyperLogLogPlusCounter[allCuboids.size()];
-        for (int i = 0; i < allCuboids.size(); i++) {
-            allCuboidsHLL[i] = new HyperLogLogPlusCounter(14);
-        }
-
-        //  hf = Hashing.goodFastHash(32);
-//        hf = Hashing.md5();
-        hf = Hashing.murmur3_32();
-
-        for (int i = 0; i < ROW_LENGTH; i++) {
-            row_index[i] = new ByteArray();
-        }
-    }
-
-    private void addCuboidBitSet(long cuboidId, List<Integer[]> allCuboidsBitSet) {
-        BitSet bitSet = BitSet.valueOf(new long[]{cuboidId});
-        Integer[] indice = new Integer[bitSet.cardinality()];
-
-        long mask = Long.highestOneBit(baseCuboidId);
-        int position = 0;
-        for (int i = 0; i < ROW_LENGTH; i++) {
-            if ((mask & cuboidId) > 0) {
-                indice[position] = i;
-                position++;
-            }
-            mask = mask >> 1;
-        }
-
-        allCuboidsBitSet.add(indice);
-
-    }
-
-    @Test
-    public void test() {
-
-        long start = System.currentTimeMillis();
-        List<String> row;
-        for (int i = 0; i < 10000; i++) {
-            row = getRandomRow();
-            putRowKeyToHLL(row);
-        }
-
-        long duration = System.currentTimeMillis() - start;
-        System.out.println("The test takes " + duration / 1000 + "seconds.");
-    }
-
-    private void putRowKeyToHLL(List<String> row) {
-        int x = 0;
-        for (String field : row) {
-            Hasher hc = hf.newHasher();
-            row_index[x++].set(hc.putString(field).hash().asBytes());
-        }
-
-        for (int i = 0, n = allCuboidsBitSet.length; i < n; i++) {
-            Hasher hc = hf.newHasher();
-            for (int position = 0; position < allCuboidsBitSet[i].length; position++) {
-                hc.putBytes(row_index[allCuboidsBitSet[i][position]].array());
-                hc.putBytes(seperator);
-            }
-            allCuboidsHLL[i].add(hc.hash().asBytes());
-        }
-    }
-
-    private List<String> getRandomRow() {
-        row.clear();
-        for (int i = 0; i < ROW_LENGTH; i++) {
-            row.add(RandomStringUtils.random(10));
-        }
-        return row;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/FactDistinctColumnsReducerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/FactDistinctColumnsReducerTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/FactDistinctColumnsReducerTest.java
deleted file mode 100644
index b85afcc..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cubev2/FactDistinctColumnsReducerTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-package org.apache.kylin.job.hadoop.cubev2;
-
-import com.google.common.collect.Maps;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.job.hadoop.cube.FactDistinctColumnsReducer;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.UUID;
-
-/**
- */
-public class FactDistinctColumnsReducerTest {
-
-
-    @Test
-    public void testWriteCuboidStatistics() throws IOException {
-
-        final Configuration conf = HadoopUtil.getCurrentConfiguration();
-        final Path outputPath = new Path("file:///tmp/kylin/cuboidstatistics/" + UUID.randomUUID().toString());
-        if (!FileSystem.getLocal(conf).exists(outputPath)) {
-//            FileSystem.getLocal(conf).create(outputPath);
-        }
-
-        System.out.println(outputPath);
-        Map<Long, HyperLogLogPlusCounter> cuboidHLLMap = Maps.newHashMap();
-        FactDistinctColumnsReducer.writeCuboidStatistics(conf, outputPath, cuboidHLLMap, 100);
-
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
deleted file mode 100644
index 5f43c2a..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.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.hbase;
-
-import static org.junit.Assert.*;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-public class CreateHTableTest extends LocalFileMetadataTestCase {
-
-    private Configuration conf;
-
-    @Before
-    public void setup() throws Exception {
-        conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-        this.createTestMetadata();
-
-    }
-
-    @After
-    public void after() throws Exception {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void testGetSplits() throws IllegalArgumentException, Exception {
-        CreateHTableJob c = new CreateHTableJob();
-
-        String input = "src/test/resources/partition_list/part-r-00000";
-
-        byte[][] splits = c.getSplits(conf, new Path(input));
-
-        assertEquals(497, splits.length);
-        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 15, -1, 11, 51, -45, 2 }, splits[0]);
-        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 3, -1, -1, -54, -61, 109, -44, 1 }, splits[496]);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
deleted file mode 100644
index 3232a80..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
+++ /dev/null
@@ -1,101 +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.hbase;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.kylin.common.util.Bytes;
-
-/**
- */
-public class TestHbaseClient {
-
-    private static boolean reverse = false;
-
-    public static void foo(int n, int k) {
-        int t = k;
-        if (n - k < k) {
-            t = n - k;
-            reverse = true;
-        }
-        boolean[] flags = new boolean[n];
-        inner(flags, 0, t);
-    }
-
-    private static void print(boolean[] flags) {
-        for (int i = 0; i < flags.length; i++) {
-            if (!reverse) {
-                if (flags[i])
-                    System.out.print("0");
-                else
-                    System.out.print("1");
-            } else {
-                if (flags[i])
-                    System.out.print("1");
-                else
-                    System.out.print("0");
-
-            }
-        }
-        System.out.println();
-
-    }
-
-    private static void inner(boolean[] flags, int start, int remaining) {
-        if (remaining <= 0) {
-            print(flags);
-            return;
-        }
-
-        if (flags.length - start < remaining) {
-            return;
-        }
-
-        // write at flags[start]
-        flags[start] = true;
-        inner(flags, start + 1, remaining - 1);
-
-        // not write at flags[start]
-        flags[start] = false;
-        inner(flags, start + 1, remaining);
-    }
-
-    public static void main(String[] args) throws IOException {
-        foo(6, 5);
-        foo(5, 2);
-        foo(3, 0);
-
-        Configuration conf = HBaseConfiguration.create();
-        conf.set("hbase.zookeeper.quorum", "hbase_host");
-        conf.set("zookeeper.znode.parent", "/hbase-unsecure");
-
-        HTable table = new HTable(conf, "test1");
-        Put put = new Put(Bytes.toBytes("row1"));
-
-        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual1"), Bytes.toBytes("val1"));
-        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual2"), Bytes.toBytes("val2"));
-
-        table.put(put);
-        table.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/hdfs/ITHdfsOpsTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/hdfs/ITHdfsOpsTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/hdfs/ITHdfsOpsTest.java
deleted file mode 100644
index 69519e0..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/hdfs/ITHdfsOpsTest.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.hdfs;
-
-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.kylin.common.KylinConfig;
-import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-/**
- */
-public class ITHdfsOpsTest extends HBaseMetadataTestCase {
-
-    FileSystem fileSystem;
-
-    @Before
-    public void setup() throws Exception {
-
-        this.createTestMetadata();
-
-        Configuration hconf = new Configuration();
-
-        fileSystem = FileSystem.get(hconf);
-    }
-
-    @Test
-    public void TestPath() throws IOException {
-        String hdfsWorkingDirectory = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory();
-        Path coprocessorDir = new Path(hdfsWorkingDirectory, "test");
-        fileSystem.mkdirs(coprocessorDir);
-
-        Path newFile = new Path(coprocessorDir, "test_file");
-        newFile = newFile.makeQualified(fileSystem.getUri(), null);
-        FSDataOutputStream stream = fileSystem.create(newFile);
-        stream.write(new byte[] { 0, 1, 2 });
-        stream.close();
-    }
-
-    @After
-    public void after() throws Exception {
-        this.cleanupTestMetadata();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/hive/JoinedFlatTableTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/hive/JoinedFlatTableTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/hive/JoinedFlatTableTest.java
deleted file mode 100644
index 21f6a71..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/hive/JoinedFlatTableTest.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.hive;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-
-import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.job.JoinedFlatTable;
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-@Ignore("This test case doesn't have much value, ignore it.")
-public class JoinedFlatTableTest extends LocalFileMetadataTestCase {
-
-    CubeInstance cube = null;
-    CubeJoinedFlatTableDesc intermediateTableDesc = null;
-    String fakeJobUUID = "abc-def";
-    CubeSegment cubeSegment = null;
-
-    @Before
-    public void setUp() throws Exception {
-        this.createTestMetadata();
-        cube = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_ready");
-        cubeSegment = cube.getSegments().get(0);
-        intermediateTableDesc = new CubeJoinedFlatTableDesc(cube.getDescriptor(), cubeSegment);
-    }
-
-    @After
-    public void after() throws Exception {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void testGenCreateTableDDL() {
-        String ddl = JoinedFlatTable.generateCreateTableStatement(intermediateTableDesc, "/tmp");
-        System.out.println(ddl);
-
-        System.out.println("The length for the ddl is " + ddl.length());
-    }
-
-    @Test
-    public void testGenDropTableDDL() {
-        String ddl = JoinedFlatTable.generateDropTableStatement(intermediateTableDesc);
-        System.out.println(ddl);
-        assertEquals(107, ddl.length());
-    }
-
-    @Test
-    public void testGenerateInsertSql() throws IOException {
-        String sqls = JoinedFlatTable.generateInsertDataStatement(intermediateTableDesc, new JobEngineConfig(KylinConfig.getInstanceFromEnv()));
-        System.out.println(sqls);
-
-        int length = sqls.length();
-        assertEquals(1155, length);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java b/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
index 20208f2..1a503e1 100644
--- a/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
+++ b/job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
@@ -26,7 +26,7 @@ import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
-import org.apache.kylin.storage.hbase.ZookeeperJobLock;
+import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/inmemcubing/ConcurrentDiskStoreTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/inmemcubing/ConcurrentDiskStoreTest.java b/job/src/test/java/org/apache/kylin/job/inmemcubing/ConcurrentDiskStoreTest.java
deleted file mode 100644
index 2fbcd94..0000000
--- a/job/src/test/java/org/apache/kylin/job/inmemcubing/ConcurrentDiskStoreTest.java
+++ /dev/null
@@ -1,93 +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.inmemcubing;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.kylin.storage.gridtable.GTBuilder;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.GridTable;
-import org.apache.kylin.storage.gridtable.IGTScanner;
-import org.apache.kylin.storage.gridtable.UnitTestSupport;
-import org.junit.Test;
-
-public class ConcurrentDiskStoreTest {
-
-    final GTInfo info = UnitTestSupport.advancedInfo();
-    final List<GTRecord> data = UnitTestSupport.mockupData(info, 1000000); // converts to about 34 MB data
-
-    @Test
-    public void testSingleThreadRead() throws IOException, InterruptedException {
-        long start = System.currentTimeMillis();
-        verifyOneTableWriteAndRead(1);
-        long end = System.currentTimeMillis();
-        System.out.println("Cost " + (end - start) + " millis");
-    }
-
-    @Test
-    public void testMultiThreadRead() throws IOException, InterruptedException {
-        long start = System.currentTimeMillis();
-        verifyOneTableWriteAndRead(20);
-        long end = System.currentTimeMillis();
-        System.out.println("Cost " + (end - start) + " millis");
-    }
-    
-    private void verifyOneTableWriteAndRead(int readThreads) throws IOException, InterruptedException {
-        ConcurrentDiskStore store = new ConcurrentDiskStore(info);
-        GridTable table = new GridTable(info, store);
-        verifyWriteAndRead(table, readThreads);
-    }
-
-    private void verifyWriteAndRead(final GridTable table, int readThreads) throws IOException, InterruptedException {
-        GTBuilder builder = table.rebuild();
-        for (GTRecord r : data) {
-            builder.write(r);
-        }
-        builder.close();
-
-        int nThreads = readThreads;
-        Thread[] t = new Thread[nThreads];
-        for (int i = 0; i < nThreads; i++) {
-            t[i] = new Thread() {
-                public void run() {
-                    try {
-                        IGTScanner scanner = table.scan(new GTScanRequest(table.getInfo()));
-                        int i = 0;
-                        for (GTRecord r : scanner) {
-                            assertEquals(data.get(i++), r);
-                        }
-                        scanner.close();
-                    } catch (Exception ex) {
-                        ex.printStackTrace();
-                    }
-                }
-            };
-            t[i].start();
-        }
-        for (int i = 0; i < nThreads; i++) {
-            t[i].join();
-        }
-        
-        ((ConcurrentDiskStore) table.getStore()).close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilderStressTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilderStressTest.java b/job/src/test/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilderStressTest.java
deleted file mode 100644
index d5563b7..0000000
--- a/job/src/test/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilderStressTest.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements. See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License. You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.kylin.job.inmemcubing;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-public class DoggedCubeBuilderStressTest extends LocalFileMetadataTestCase {
-
-    @SuppressWarnings("unused")
-    private static final Logger logger = LoggerFactory.getLogger(DoggedCubeBuilderStressTest.class);
-
-    // CI sandbox memory is no more than 512MB, this many input should hit memory threshold
-    private static final int INPUT_ROWS = 200000;
-    private static final int THREADS = 4;
-
-    private static CubeInstance cube;
-    private static String flatTable;
-    private static Map<TblColRef, Dictionary<?>> dictionaryMap;
-
-    @BeforeClass
-    public static void before() throws IOException {
-        staticCreateTestMetadata();
-
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        CubeManager cubeManager = CubeManager.getInstance(kylinConfig);
-
-        cube = cubeManager.getCube("test_kylin_cube_without_slr_left_join_empty");
-        flatTable = "../examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv";
-        dictionaryMap = InMemCubeBuilderTest.getDictionaryMap(cube, flatTable);
-    }
-
-    @AfterClass
-    public static void after() throws Exception {
-        staticCleanupTestMetadata();
-    }
-
-    @Test
-    public void test() throws Exception {
-
-        ArrayBlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(1000);
-        ExecutorService executorService = Executors.newSingleThreadExecutor();
-        long randSeed = System.currentTimeMillis();
-
-        DoggedCubeBuilder doggedBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap);
-        doggedBuilder.setConcurrentThreads(THREADS);
-
-        {
-            Future<?> future = executorService.submit(doggedBuilder.buildAsRunnable(queue, new NoopWriter()));
-            InMemCubeBuilderTest.feedData(cube, flatTable, queue, INPUT_ROWS, randSeed);
-            future.get();
-        }
-    }
-
-    class NoopWriter implements ICuboidWriter {
-        @Override
-        public void write(long cuboidId, GTRecord record) throws IOException {
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilderTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilderTest.java b/job/src/test/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilderTest.java
deleted file mode 100644
index a87f950..0000000
--- a/job/src/test/java/org/apache/kylin/job/inmemcubing/DoggedCubeBuilderTest.java
+++ /dev/null
@@ -1,154 +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.inmemcubing;
-
-import static org.junit.Assert.*;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.PrintWriter;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-public class DoggedCubeBuilderTest extends LocalFileMetadataTestCase {
-
-    @SuppressWarnings("unused")
-    private static final Logger logger = LoggerFactory.getLogger(DoggedCubeBuilderTest.class);
-
-    private static final int INPUT_ROWS = 10000;
-    private static final int SPLIT_ROWS = 5000;
-    private static final int THREADS = 4;
-
-    private static CubeInstance cube;
-    private static String flatTable;
-    private static Map<TblColRef, Dictionary<?>> dictionaryMap;
-
-    @BeforeClass
-    public static void before() throws IOException {
-        staticCreateTestMetadata();
-
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        CubeManager cubeManager = CubeManager.getInstance(kylinConfig);
-
-        cube = cubeManager.getCube("test_kylin_cube_without_slr_left_join_empty");
-        flatTable = "../examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv";
-        dictionaryMap = InMemCubeBuilderTest.getDictionaryMap(cube, flatTable);
-    }
-
-    @AfterClass
-    public static void after() throws Exception {
-        staticCleanupTestMetadata();
-    }
-
-    @Test
-    public void test() throws Exception {
-
-        ArrayBlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(1000);
-        ExecutorService executorService = Executors.newSingleThreadExecutor();
-        long randSeed = System.currentTimeMillis();
-
-        DoggedCubeBuilder doggedBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap);
-        doggedBuilder.setConcurrentThreads(THREADS);
-        doggedBuilder.setSplitRowThreshold(SPLIT_ROWS);
-        FileRecordWriter doggedResult = new FileRecordWriter();
-
-        {
-            Future<?> future = executorService.submit(doggedBuilder.buildAsRunnable(queue, doggedResult));
-            InMemCubeBuilderTest.feedData(cube, flatTable, queue, INPUT_ROWS, randSeed);
-            future.get();
-            doggedResult.close();
-        }
-
-        InMemCubeBuilder inmemBuilder = new InMemCubeBuilder(cube.getDescriptor(), dictionaryMap);
-        inmemBuilder.setConcurrentThreads(THREADS);
-        FileRecordWriter inmemResult = new FileRecordWriter();
-
-        {
-            Future<?> future = executorService.submit(inmemBuilder.buildAsRunnable(queue, inmemResult));
-            InMemCubeBuilderTest.feedData(cube, flatTable, queue, INPUT_ROWS, randSeed);
-            future.get();
-            inmemResult.close();
-        }
-
-        fileCompare(doggedResult.file, inmemResult.file);
-        doggedResult.file.delete();
-        inmemResult.file.delete();
-    }
-
-    private void fileCompare(File file, File file2) throws IOException {
-        BufferedReader r1 = new BufferedReader(new InputStreamReader(new FileInputStream(file), "UTF-8"));
-        BufferedReader r2 = new BufferedReader(new InputStreamReader(new FileInputStream(file2), "UTF-8"));
-
-        String line1, line2;
-        do {
-            line1 = r1.readLine();
-            line2 = r2.readLine();
-            
-            assertEquals(line1, line2);
-            
-        } while (line1 != null || line2 != null);
-
-        r1.close();
-        r2.close();
-    }
-
-    class FileRecordWriter implements ICuboidWriter {
-
-        File file;
-        PrintWriter writer;
-
-        FileRecordWriter() throws IOException {
-            file = File.createTempFile("DoggedCubeBuilderTest_", ".data");
-            writer = new PrintWriter(file, "UTF-8");
-        }
-
-        @Override
-        public void write(long cuboidId, GTRecord record) throws IOException {
-            writer.print(cuboidId);
-            writer.print(", ");
-            writer.print(record.toString());
-            writer.println();
-        }
-
-        public void close() {
-            writer.close();
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilderTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilderTest.java b/job/src/test/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilderTest.java
deleted file mode 100644
index 3caa1b0..0000000
--- a/job/src/test/java/org/apache/kylin/job/inmemcubing/InMemCubeBuilderTest.java
+++ /dev/null
@@ -1,207 +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.inmemcubing;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.dict.DictionaryGenerator;
-import org.apache.kylin.engine.mr.DFSFileTableReader;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-/**
- */
-public class InMemCubeBuilderTest extends LocalFileMetadataTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(InMemCubeBuilderTest.class);
-
-    private static final int INPUT_ROWS = 70000;
-    private static final int THREADS = 4;
-    
-    private static CubeInstance cube;
-    private static String flatTable;
-    private static Map<TblColRef, Dictionary<?>> dictionaryMap;
-    
-    @BeforeClass
-    public static void before() throws IOException {
-        staticCreateTestMetadata();
-        
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        CubeManager cubeManager = CubeManager.getInstance(kylinConfig);
-        
-        cube = cubeManager.getCube("test_kylin_cube_without_slr_left_join_empty");
-        flatTable = "../examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv";
-        dictionaryMap = getDictionaryMap(cube, flatTable);
-    }
-
-    @AfterClass
-    public static void after() throws Exception {
-        staticCleanupTestMetadata();
-    }
-
-    @Test
-    public void test() throws Exception {
-
-        InMemCubeBuilder cubeBuilder = new InMemCubeBuilder(cube.getDescriptor(), dictionaryMap);
-        //DoggedCubeBuilder cubeBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap);
-        cubeBuilder.setConcurrentThreads(THREADS);
-        
-        ArrayBlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(1000);
-        ExecutorService executorService = Executors.newSingleThreadExecutor();
-
-        try {
-            // round 1
-            {
-                Future<?> future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new ConsoleGTRecordWriter()));
-                feedData(cube, flatTable, queue, INPUT_ROWS);
-                future.get();
-            }
-            
-            // round 2, zero input
-            {
-                Future<?> future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new ConsoleGTRecordWriter()));
-                feedData(cube, flatTable, queue, 0);
-                future.get();
-            }
-            
-            // round 3
-            {
-                Future<?> future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new ConsoleGTRecordWriter()));
-                feedData(cube, flatTable, queue, INPUT_ROWS);
-                future.get();
-            }
-            
-        } catch (Exception e) {
-            logger.error("stream build failed", e);
-            throw new IOException("Failed to build cube ", e);
-        }
-    }
-
-    static void feedData(final CubeInstance cube, final String flatTable, ArrayBlockingQueue<List<String>> queue, int count) throws IOException, InterruptedException {
-        feedData(cube, flatTable, queue, count, 0);
-    }
-    
-    static void feedData(final CubeInstance cube, final String flatTable, ArrayBlockingQueue<List<String>> queue, int count, long randSeed) throws IOException, InterruptedException {
-        CubeJoinedFlatTableDesc flatTableDesc = new CubeJoinedFlatTableDesc(cube.getDescriptor(), null);
-        int nColumns = flatTableDesc.getColumnList().size();
-
-        @SuppressWarnings("unchecked")
-        Set<String>[] distinctSets = new Set[nColumns];
-        for (int i = 0; i < nColumns; i++)
-            distinctSets[i] = new TreeSet<String>();
-
-        // get distinct values on each column
-        DFSFileTableReader reader = new DFSFileTableReader(flatTable, nColumns);
-        while (count > 0 && reader.next()) {
-            String[] row = reader.getRow();
-            for (int i = 0; i < nColumns; i++)
-                distinctSets[i].add(row[i]);
-        }
-        reader.close();
-
-        List<String[]> distincts = new ArrayList<String[]>();
-        for (int i = 0; i < nColumns; i++) {
-            distincts.add((String[]) distinctSets[i].toArray(new String[distinctSets[i].size()]));
-        }
-
-        Random rand = new Random();
-        if (randSeed != 0)
-            rand.setSeed(randSeed);
-        
-        // output with random data
-        for (; count > 0; count--) {
-            ArrayList<String> row = new ArrayList<String>(nColumns);
-            for (int i = 0; i < nColumns; i++) {
-                String[] candidates = distincts.get(i);
-                row.add(candidates[rand.nextInt(candidates.length)]);
-            }
-            queue.put(row);
-        }
-        queue.put(new ArrayList<String>(0));
-    }
-
-    static Map<TblColRef, Dictionary<?>> getDictionaryMap(CubeInstance cube, String flatTable) throws IOException {
-        Map<TblColRef, Dictionary<?>> result = Maps.newHashMap();
-        CubeDesc desc = cube.getDescriptor();
-        CubeJoinedFlatTableDesc flatTableDesc = new CubeJoinedFlatTableDesc(desc, null);
-        int nColumns = flatTableDesc.getColumnList().size();
-
-        List<TblColRef> columns = Cuboid.getBaseCuboid(desc).getColumns();
-        for (int c = 0; c < columns.size(); c++) {
-            TblColRef col = columns.get(c);
-            if (desc.getRowkey().isUseDictionary(col)) {
-                logger.info("Building dictionary for " + col);
-                List<byte[]> valueList = readValueList(flatTable, nColumns, flatTableDesc.getRowKeyColumnIndexes()[c]);
-                Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueList(col.getType(), valueList);
-                result.put(col, dict);
-            }
-        }
-        return result;
-    }
-
-    private static List<byte[]> readValueList(String flatTable, int nColumns, int c) throws IOException {
-        List<byte[]> result = Lists.newArrayList();
-        DFSFileTableReader reader = new DFSFileTableReader(flatTable, nColumns);
-        while (reader.next()) {
-            String[] row = reader.getRow();
-            if (row[c] != null) {
-                result.add(Bytes.toBytes(row[c]));
-            }
-        }
-        reader.close();
-        return result;
-    }
-
-    class ConsoleGTRecordWriter implements ICuboidWriter {
-
-        boolean verbose = false;
-
-        @Override
-        public void write(long cuboidId, GTRecord record) throws IOException {
-            if (verbose)
-                System.out.println(record.toString());
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/inmemcubing/MemDiskStoreTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/inmemcubing/MemDiskStoreTest.java b/job/src/test/java/org/apache/kylin/job/inmemcubing/MemDiskStoreTest.java
deleted file mode 100644
index fd848f2..0000000
--- a/job/src/test/java/org/apache/kylin/job/inmemcubing/MemDiskStoreTest.java
+++ /dev/null
@@ -1,97 +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.inmemcubing;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.kylin.common.util.MemoryBudgetController;
-import org.apache.kylin.storage.gridtable.GTBuilder;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.GridTable;
-import org.apache.kylin.storage.gridtable.IGTScanner;
-import org.apache.kylin.storage.gridtable.UnitTestSupport;
-import org.junit.Test;
-
-public class MemDiskStoreTest {
-
-    final MemoryBudgetController budgetCtrl = new MemoryBudgetController(20);
-    final GTInfo info = UnitTestSupport.advancedInfo();
-    final List<GTRecord> data = UnitTestSupport.mockupData(info, 1000000); // converts to about 34 MB data
-
-    @Test
-    public void testSingleThreadWriteRead() throws IOException {
-        long start = System.currentTimeMillis();
-        verifyOneTableWriteAndRead();
-        long end = System.currentTimeMillis();
-        System.out.println("Cost " + (end - start) + " millis");
-    }
-
-    @Test
-    public void testMultiThreadWriteRead() throws IOException, InterruptedException {
-        long start = System.currentTimeMillis();
-
-        int nThreads = 5;
-        Thread[] t = new Thread[nThreads];
-        for (int i = 0; i < nThreads; i++) {
-            t[i] = new Thread() {
-                public void run() {
-                    try {
-                        verifyOneTableWriteAndRead();
-                    } catch (Exception ex) {
-                        ex.printStackTrace();
-                    }
-                }
-            };
-            t[i].start();
-        }
-        for (int i = 0; i < nThreads; i++) {
-            t[i].join();
-        }
-
-        long end = System.currentTimeMillis();
-        System.out.println("Cost " + (end - start) + " millis");
-    }
-
-    private void verifyOneTableWriteAndRead() throws IOException {
-        MemDiskStore store = new MemDiskStore(info, budgetCtrl);
-        GridTable table = new GridTable(info, store);
-        verifyWriteAndRead(table);
-    }
-
-    private void verifyWriteAndRead(GridTable table) throws IOException {
-        GTInfo info = table.getInfo();
-
-        GTBuilder builder = table.rebuild();
-        for (GTRecord r : data) {
-            builder.write(r);
-        }
-        builder.close();
-
-        IGTScanner scanner = table.scan(new GTScanRequest(info));
-        int i = 0;
-        for (GTRecord r : scanner) {
-            assertEquals(data.get(i++), r);
-        }
-        scanner.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/inmemcubing/MemoryBudgetControllerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/inmemcubing/MemoryBudgetControllerTest.java b/job/src/test/java/org/apache/kylin/job/inmemcubing/MemoryBudgetControllerTest.java
deleted file mode 100644
index 6f86c95..0000000
--- a/job/src/test/java/org/apache/kylin/job/inmemcubing/MemoryBudgetControllerTest.java
+++ /dev/null
@@ -1,98 +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.inmemcubing;
-
-import org.apache.kylin.common.util.MemoryBudgetController;
-import org.apache.kylin.common.util.MemoryBudgetController.NotEnoughBudgetException;
-import org.junit.Test;
-
-import java.util.ArrayList;
-
-import static org.junit.Assert.*;
-
-public class MemoryBudgetControllerTest {
-
-    @Test
-    public void test() {
-        final int n = MemoryBudgetController.getSystemAvailMB() / 2;
-        final MemoryBudgetController mbc = new MemoryBudgetController(n);
-
-        ArrayList<Consumer> mbList = new ArrayList<Consumer>();
-        for (int i = 0; i < n; i++) {
-            mbList.add(new Consumer(mbc));
-            assertEquals(mbList.size(), mbc.getTotalReservedMB());
-        }
-
-        // a's reservation will free up all the previous
-        final Consumer a = new Consumer();
-        mbc.reserve(a, n);
-        for (int i = 0; i < n; i++) {
-            assertEquals(null, mbList.get(i).data);
-        }
-        
-        // cancel a in 2 seconds
-        new Thread() {
-            @Override
-            public void run() {
-                try {
-                    Thread.sleep(2000);
-                } catch (InterruptedException e) {
-                    e.printStackTrace();
-                }
-                mbc.reserve(a, 0);
-            }
-        }.start();
-        
-        // b will success after some wait
-        long bWaitStart = System.currentTimeMillis();
-        final Consumer b = new Consumer();
-        mbc.reserveInsist(b, n);
-        assertTrue(System.currentTimeMillis() - bWaitStart > 1000);
-
-        try {
-            mbc.reserve(a, 1);
-            fail();
-        } catch (NotEnoughBudgetException ex) {
-            // expected
-        }
-    }
-
-    class Consumer implements MemoryBudgetController.MemoryConsumer {
-
-        byte[] data;
-
-        Consumer() {
-        }
-
-        Consumer(MemoryBudgetController mbc) {
-            mbc.reserve(this, 1);
-            data = new byte[MemoryBudgetController.ONE_MB - 24]; // 24 is object shell of this + object shell of data + reference of data 
-        }
-
-        @Override
-        public int freeUp(int mb) {
-            if (data != null) {
-                data = null;
-                return 1;
-            } else {
-                return 0;
-            }
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/manager/ExecutableManagerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/manager/ExecutableManagerTest.java b/job/src/test/java/org/apache/kylin/job/manager/ExecutableManagerTest.java
deleted file mode 100644
index 1d28fc8..0000000
--- a/job/src/test/java/org/apache/kylin/job/manager/ExecutableManagerTest.java
+++ /dev/null
@@ -1,143 +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.manager;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.job.BaseTestExecutable;
-import org.apache.kylin.job.SucceedTestExecutable;
-import org.apache.kylin.job.exception.IllegalStateTranferException;
-import org.apache.kylin.job.execution.ChainedExecutable;
-import org.apache.kylin.job.execution.Executable;
-import org.apache.kylin.job.execution.ExecutableState;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-/**
- */
-public class ExecutableManagerTest extends LocalFileMetadataTestCase {
-
-    private ExecutableManager service;
-
-    @Before
-    public void setup() throws Exception {
-        createTestMetadata();
-        service = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
-
-        for (String jobId: service.getAllJobIds()) {
-            System.out.println("deleting " + jobId);
-            service.deleteJob(jobId);
-        }
-
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-    }
-
-    @Test
-    public void test() throws Exception {
-        assertNotNull(service);
-        BaseTestExecutable executable = new SucceedTestExecutable();
-        executable.setParam("test1", "test1");
-        executable.setParam("test2", "test2");
-        executable.setParam("test3", "test3");
-        service.addJob(executable);
-        List<AbstractExecutable> result = service.getAllExecutables();
-        assertEquals(1, result.size());
-        AbstractExecutable another = service.getJob(executable.getId());
-        assertJobEqual(executable, another);
-
-        service.updateJobOutput(executable.getId(), ExecutableState.RUNNING, null, "test output");
-        assertJobEqual(executable, service.getJob(executable.getId()));
-    }
-
-    @Test
-    public void testDefaultChainedExecutable() throws Exception {
-        DefaultChainedExecutable job = new DefaultChainedExecutable();
-        job.addTask(new SucceedTestExecutable());
-        job.addTask(new SucceedTestExecutable());
-
-        service.addJob(job);
-        assertEquals(2, job.getTasks().size());
-        AbstractExecutable anotherJob = service.getJob(job.getId());
-        assertEquals(DefaultChainedExecutable.class, anotherJob.getClass());
-        assertEquals(2, ((DefaultChainedExecutable) anotherJob).getTasks().size());
-        assertJobEqual(job, anotherJob);
-    }
-
-    @Test
-    public void testValidStateTransfer() throws Exception {
-        SucceedTestExecutable job = new SucceedTestExecutable();
-        String id = job.getId();
-        service.addJob(job);
-        service.updateJobOutput(id, ExecutableState.RUNNING, null, null);
-        service.updateJobOutput(id, ExecutableState.ERROR, null, null);
-        service.updateJobOutput(id, ExecutableState.READY, null, null);
-        service.updateJobOutput(id, ExecutableState.RUNNING, null, null);
-        service.updateJobOutput(id, ExecutableState.READY, null, null);
-        service.updateJobOutput(id, ExecutableState.RUNNING, null, null);
-        service.updateJobOutput(id, ExecutableState.SUCCEED, null, null);
-    }
-
-    @Test(expected = IllegalStateTranferException.class)
-    public void testInvalidStateTransfer(){
-        SucceedTestExecutable job = new SucceedTestExecutable();
-        service.addJob(job);
-        service.updateJobOutput(job.getId(), ExecutableState.RUNNING, null, null);
-        service.updateJobOutput(job.getId(), ExecutableState.STOPPED, null, null);
-    }
-
-
-
-    private static void assertJobEqual(Executable one, Executable another) {
-        assertEquals(one.getClass(), another.getClass());
-        assertEquals(one.getId(), another.getId());
-        assertEquals(one.getStatus(), another.getStatus());
-        assertEquals(one.isRunnable(), another.isRunnable());
-        assertEquals(one.getOutput(), another.getOutput());
-        assertTrue((one.getParams() == null && another.getParams() == null) || (one.getParams() != null && another.getParams() != null));
-        if (one.getParams() != null) {
-            assertEquals(one.getParams().size(), another.getParams().size());
-            for (String key : one.getParams().keySet()) {
-                assertEquals(one.getParams().get(key), another.getParams().get(key));
-            }
-        }
-        if (one instanceof ChainedExecutable) {
-            assertTrue(another instanceof ChainedExecutable);
-            List<? extends Executable> onesSubs = ((ChainedExecutable) one).getTasks();
-            List<? extends Executable> anotherSubs = ((ChainedExecutable) another).getTasks();
-            assertTrue((onesSubs == null && anotherSubs == null) || (onesSubs != null && anotherSubs != null));
-            if (onesSubs != null) {
-                assertEquals(onesSubs.size(), anotherSubs.size());
-                for (int i = 0; i < onesSubs.size(); ++i) {
-                    assertJobEqual(onesSubs.get(i), anotherSubs.get(i));
-                }
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
index 1e55b2c..325cbee 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -42,7 +42,7 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.engine.BuildEngineFactory;
 import org.apache.kylin.engine.mr.CubingJob;
 import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.job.common.HadoopShellExecutable;
+import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
 import org.apache.kylin.job.exception.JobException;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
@@ -65,7 +65,7 @@ import org.apache.kylin.rest.response.MetricsResponse;
 import org.apache.kylin.rest.security.AclPermission;
 import org.apache.kylin.source.hive.HiveSourceTableLoader;
 import org.apache.kylin.storage.hbase.HBaseConnection;
-import org.apache.kylin.storage.hbase.HBaseRegionSizeCalculator;
+import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/server/src/main/java/org/apache/kylin/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/JobService.java b/server/src/main/java/org/apache/kylin/rest/service/JobService.java
index 96b615f..bb17bd2 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -31,9 +31,9 @@ import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.model.CubeBuildTypeEnum;
 import org.apache.kylin.engine.BuildEngineFactory;
 import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
 import org.apache.kylin.job.JobInstance;
-import org.apache.kylin.job.common.HadoopShellExecutable;
-import org.apache.kylin.job.common.MapReduceExecutable;
 import org.apache.kylin.job.common.ShellExecutable;
 import org.apache.kylin.job.constant.JobStatusEnum;
 import org.apache.kylin.job.constant.JobStepStatusEnum;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/source-hive/pom.xml
----------------------------------------------------------------------
diff --git a/source-hive/pom.xml b/source-hive/pom.xml
index 17a918e..8b4a973 100644
--- a/source-hive/pom.xml
+++ b/source-hive/pom.xml
@@ -48,6 +48,11 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.hive.hcatalog</groupId>
             <artifactId>hive-hcatalog-core</artifactId>
             <version>${hive-hcatalog.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..c40924e
--- /dev/null
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.source.hive;
+
+import java.io.IOException;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hive.hcatalog.data.HCatRecord;
+import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.engine.mr.IMRInput;
+import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.job.JoinedFlatTable;
+import org.apache.kylin.job.cmd.ShellCmdOutput;
+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.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+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;
+
+public class HiveMRInput implements IMRInput {
+
+    @Override
+    public IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg) {
+        return new BatchCubingInputSide(seg);
+    }
+    
+    @Override
+    public IMRTableInputFormat getTableInputFormat(TableDesc table) {
+        return new HiveTableInputFormat(table.getIdentity());
+    }
+    
+    public static class HiveTableInputFormat implements IMRTableInputFormat {
+        final String dbName;
+        final String tableName;
+
+        public HiveTableInputFormat(String hiveTable) {
+            String[] parts = HadoopUtil.parseHiveTableName(hiveTable);
+            dbName = parts[0];
+            tableName = parts[1];
+        }
+
+        @Override
+        public void configureJob(Job job) {
+            try {
+                HCatInputFormat.setInput(job, dbName, tableName);
+                job.setInputFormatClass(HCatInputFormat.class);
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        @Override
+        public String[] parseMapperInput(Object mapperInput) {
+            return HiveTableReader.getRowAsStringArray((HCatRecord) mapperInput);
+        }
+        
+    }
+
+    public static class BatchCubingInputSide implements IMRBatchCubingInputSide {
+        
+        final JobEngineConfig conf;
+        final CubeSegment seg;
+        final CubeJoinedFlatTableDesc flatHiveTableDesc;
+
+        public BatchCubingInputSide(CubeSegment seg) {
+            this.conf = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
+            this.seg = seg;
+            this.flatHiveTableDesc = new CubeJoinedFlatTableDesc(seg.getCubeDesc(), seg);
+        }
+
+        @Override
+        public void addStepPhase1_CreateFlatTable(DefaultChainedExecutable jobFlow) {
+            jobFlow.addTask(createFlatHiveTableStep(conf, flatHiveTableDesc, jobFlow.getId()));
+        }
+        
+        public static AbstractExecutable createFlatHiveTableStep(JobEngineConfig conf, IJoinedFlatTableDesc flatTableDesc, String jobId) {
+
+            final String dropTableHql = JoinedFlatTable.generateDropTableStatement(flatTableDesc);
+            final String createTableHql = JoinedFlatTable.generateCreateTableStatement(flatTableDesc, JobBuilderSupport.getJobWorkingDir(conf, jobId));
+            String insertDataHqls;
+            try {
+                insertDataHqls = JoinedFlatTable.generateInsertDataStatement(flatTableDesc, conf);
+            } catch (IOException e) {
+                throw new RuntimeException("Failed to generate insert data SQL for intermediate table.", e);
+            }
+
+            ShellExecutable step = new ShellExecutable();
+            StringBuffer buf = new StringBuffer();
+            buf.append("hive -e \"");
+            buf.append(dropTableHql + "\n");
+            buf.append(createTableHql + "\n");
+            buf.append(insertDataHqls + "\n");
+            buf.append("\"");
+
+            step.setCmd(buf.toString());
+            step.setName(ExecutableConstants.STEP_NAME_CREATE_FLAT_HIVE_TABLE);
+
+            return step;
+        }
+
+        @Override
+        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
+            GarbageCollectionStep step = new GarbageCollectionStep();
+            step.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
+            step.setOldHiveTable(flatHiveTableDesc.getTableName());
+            jobFlow.addTask(step);
+        }
+
+        @Override
+        public IMRTableInputFormat getFlatTableInputFormat() {
+            return new HiveTableInputFormat(flatHiveTableDesc.getTableName());
+        }
+        
+    }
+    
+    public static class GarbageCollectionStep extends AbstractExecutable {
+
+        @Override
+        protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+            StringBuffer output = new StringBuffer();
+
+            final String hiveTable = this.getOldHiveTable();
+            if (StringUtils.isNotEmpty(hiveTable)) {
+                final String dropHiveCMD = "hive -e \"DROP TABLE IF EXISTS  " + hiveTable + ";\"";
+                ShellCmdOutput shellCmdOutput = new ShellCmdOutput();
+                try {
+                    context.getConfig().getCliCommandExecutor().execute(dropHiveCMD, shellCmdOutput);
+                    output.append("Hive table " + hiveTable + " is dropped. \n");
+                } catch (IOException e) {
+                    logger.error("job:" + getId() + " execute finished with exception", e);
+                    output.append(shellCmdOutput.getOutput()).append("\n").append(e.getLocalizedMessage());
+                    return new ExecuteResult(ExecuteResult.State.ERROR, output.toString());
+                }
+            }
+
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, output.toString());
+        }
+
+        public void setOldHiveTable(String hiveTable) {
+            setParam("oldHiveTable", hiveTable);
+        }
+
+        private String getOldHiveTable() {
+            return getParam("oldHiveTable");
+        }
+    }
+
+}


[27/28] incubator-kylin git commit: minor, bug fix, rollback a previous refactor in InMemCubeBuilder that causes OOM on Windows

Posted by li...@apache.org.
minor, bug fix, rollback a previous refactor in InMemCubeBuilder that causes OOM on Windows


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

Branch: refs/heads/0.8
Commit: 94676a1ba17d9a75dec5454e40f686614d8c4a46
Parents: 2ef9831
Author: Yang Li <li...@apache.org>
Authored: Fri Jul 24 06:44:44 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Fri Jul 24 06:44:44 2015 +0800

----------------------------------------------------------------------
 .../cube/inmemcubing/InMemCubeBuilder.java      | 24 +++++++++++++-------
 1 file changed, 16 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/94676a1b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
index 85ac47a..660cd23 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
@@ -23,11 +23,10 @@ import java.util.BitSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeSet;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.kylin.common.util.Bytes;
@@ -83,7 +82,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
     private MemoryBudgetController memBudget;
     private Thread[] taskThreads;
     private Throwable[] taskThreadExceptions;
-    private LinkedBlockingQueue<CuboidTask> taskPending;
+    private TreeSet<CuboidTask> taskPending;
     private AtomicInteger taskCuboidCompleted = new AtomicInteger(0);
 
     private CuboidResult baseResult;
@@ -195,7 +194,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         logger.info("In Mem Cube Build start, " + cubeDesc.getName());
 
         // multiple threads to compute cuboid in parallel
-        taskPending = new LinkedBlockingQueue<>();
+        taskPending = new TreeSet<CuboidTask>();
         taskCuboidCompleted.set(0);
         taskThreads = prepareTaskThreads();
         taskThreadExceptions = new Throwable[taskThreadCount];
@@ -293,8 +292,12 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
             try {
                 while (!isAllCuboidDone()) {
                     CuboidTask task = null;
-                    while (task == null && taskHasNoException()) {
-                        task = taskPending.poll(15, TimeUnit.SECONDS);
+                    synchronized (taskPending) {
+                        while (task == null && taskHasNoException()) {
+                            task = taskPending.pollFirst();
+                            if (task == null)
+                                taskPending.wait(60000);
+                        }
                     }
                     // if task error occurs
                     if (task == null)
@@ -328,8 +331,13 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
 
     private void addChildTasks(CuboidResult parent) {
         List<Long> children = cuboidScheduler.getSpanningCuboid(parent.cuboidId);
-        for (Long child : children) {
-            taskPending.add(new CuboidTask(parent, child));
+        if (!children.isEmpty()) {
+            synchronized (taskPending) {
+                for (Long child : children) {
+                    taskPending.add(new CuboidTask(parent, child));
+                }
+                taskPending.notifyAll();
+            }
         }
     }
 


[04/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
new file mode 100644
index 0000000..ed67109
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
@@ -0,0 +1,85 @@
+package org.apache.kylin.storage.hbase.steps;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.util.DeployCoprocessorCLI;
+import org.apache.kylin.storage.hbase.util.LZOSupportnessChecker;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ */
+public class CubeHTableUtil {
+
+    private static final Logger logger = LoggerFactory.getLogger(CubeHTableUtil.class);
+
+    public static void createHTable(CubeDesc cubeDesc, String tableName, byte[][] splitKeys) throws IOException {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+
+        HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(tableName));
+        // https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.html
+        tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName());
+        tableDesc.setValue(IRealizationConstants.HTableTag, kylinConfig.getMetadataUrlPrefix());
+
+        Configuration conf = HBaseConfiguration.create();
+        HBaseAdmin admin = new HBaseAdmin(conf);
+
+        try {
+            if (User.isHBaseSecurityEnabled(conf)) {
+                // add coprocessor for bulk load
+                tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
+            }
+
+            for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
+                HColumnDescriptor cf = new HColumnDescriptor(cfDesc.getName());
+                cf.setMaxVersions(1);
+
+                if (LZOSupportnessChecker.getSupportness()) {
+                    logger.info("hbase will use lzo to compress cube data");
+                    cf.setCompressionType(Compression.Algorithm.LZO);
+                } else {
+                    logger.info("hbase will not use lzo to compress cube data");
+                }
+
+                cf.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
+                cf.setInMemory(false);
+                cf.setBlocksize(4 * 1024 * 1024); // set to 4MB
+                tableDesc.addFamily(cf);
+            }
+
+            if (admin.tableExists(tableName)) {
+                // admin.disableTable(tableName);
+                // admin.deleteTable(tableName);
+                throw new RuntimeException("HBase table " + tableName + " exists!");
+            }
+
+            DeployCoprocessorCLI.deployCoprocessor(tableDesc);
+
+            admin.createTable(tableDesc, splitKeys);
+            Preconditions.checkArgument(admin.isTableAvailable(tableName), "table " + tableName + " created, but is not available due to some reasons");
+            logger.info("create hbase table " + tableName + " done.");
+        } catch (Exception e) {
+            logger.error("Failed to create HTable", e);
+            throw e;
+        } finally {
+            admin.close();
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..e42f709
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
@@ -0,0 +1,135 @@
+package org.apache.kylin.storage.hbase.steps;
+
+import java.util.List;
+
+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.engine.mr.steps.RangeKeyDistributionJob;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class HBaseMRSteps extends JobBuilderSupport {
+    
+    public HBaseMRSteps(CubeSegment seg) {
+        super(seg, null);
+    }
+
+    public void addSaveCuboidToHTableSteps(DefaultChainedExecutable jobFlow, String cuboidRootPath) {
+        String jobId = jobFlow.getId();
+        
+        // calculate key distribution
+        jobFlow.addTask(createRangeRowkeyDistributionStep(cuboidRootPath + "*", jobId));
+        // create htable step
+        jobFlow.addTask(createCreateHTableStep(jobId));
+        // generate hfiles step
+        jobFlow.addTask(createConvertCuboidToHfileStep(cuboidRootPath + "*", jobId));
+        // bulk load step
+        jobFlow.addTask(createBulkLoadStep(jobId));
+    }
+
+    public MapReduceExecutable createRangeRowkeyDistributionStep(String inputPath, String jobId) {
+        MapReduceExecutable rowkeyDistributionStep = new MapReduceExecutable();
+        rowkeyDistributionStep.setName(ExecutableConstants.STEP_NAME_GET_CUBOID_KEY_DISTRIBUTION);
+        StringBuilder cmd = new StringBuilder();
+
+        appendMapReduceParameters(cmd, seg);
+        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");
+
+        rowkeyDistributionStep.setMapReduceParams(cmd.toString());
+        rowkeyDistributionStep.setMapReduceJobClass(RangeKeyDistributionJob.class);
+        return rowkeyDistributionStep;
+    }
+
+    public HadoopShellExecutable createCreateHTableStep(String jobId) {
+        return createCreateHTableStep(jobId, false);
+    }
+    
+    public HadoopShellExecutable createCreateHTableStepWithStats(String jobId) {
+        return createCreateHTableStep(jobId, true);
+    }
+    
+    private HadoopShellExecutable createCreateHTableStep(String jobId, boolean withStats) {
+        HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
+        createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
+        StringBuilder cmd = new StringBuilder();
+        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, "segmentname", seg.getName());
+        appendExecCmdParameters(cmd, "input", getRowkeyDistributionOutputPath(jobId) + "/part-r-00000");
+        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
+        appendExecCmdParameters(cmd, "statisticsenabled", String.valueOf(withStats));
+
+        createHtableStep.setJobParams(cmd.toString());
+        createHtableStep.setJobClass(CreateHTableJob.class);
+
+        return createHtableStep;
+    }
+
+    public MapReduceExecutable createConvertCuboidToHfileStep(String inputPath, String jobId) {
+        MapReduceExecutable createHFilesStep = new MapReduceExecutable();
+        createHFilesStep.setName(ExecutableConstants.STEP_NAME_CONVERT_CUBOID_TO_HFILE);
+        StringBuilder cmd = new StringBuilder();
+
+        appendMapReduceParameters(cmd, seg);
+        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().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");
+
+        createHFilesStep.setMapReduceParams(cmd.toString());
+        createHFilesStep.setMapReduceJobClass(CubeHFileJob.class);
+        createHFilesStep.setCounterSaveAs(",," + CubingJob.CUBE_SIZE_BYTES);
+
+        return createHFilesStep;
+    }
+
+    public HadoopShellExecutable createBulkLoadStep(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, "cubename", seg.getCubeInstance().getName());
+
+        bulkLoadStep.setJobParams(cmd.toString());
+        bulkLoadStep.setJobClass(BulkLoadJob.class);
+
+        return bulkLoadStep;
+    }
+    
+    public MergeGCStep createMergeGCStep() {
+        MergeGCStep result = new MergeGCStep();
+        result.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
+        result.setOldHTables(getMergingHTables());
+        return result;
+    }
+
+    public List<String> getMergingHTables() {
+        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(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) {
+            mergingHTables.add(merging.getStorageLocationIdentifier());
+        }
+        return mergingHTables;
+    }
+    
+    public String getHFilePath(String jobId) {
+        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/hfile/";
+    }
+
+    public String getRowkeyDistributionOutputPath(String jobId) {
+        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/InMemKeyValueCreator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/InMemKeyValueCreator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/InMemKeyValueCreator.java
new file mode 100644
index 0000000..b2d691e
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/InMemKeyValueCreator.java
@@ -0,0 +1,73 @@
+package org.apache.kylin.storage.hbase.steps;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+import com.google.common.collect.Lists;
+
+public class InMemKeyValueCreator {
+    byte[] cfBytes;
+    byte[] qBytes;
+    long timestamp;
+
+
+    MeasureCodec codec;
+    Object[] colValues;
+    ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+    int startPosition = 0;
+
+    public InMemKeyValueCreator(HBaseColumnDesc colDesc, int startPosition) {
+
+        cfBytes = Bytes.toBytes(colDesc.getColumnFamilyName());
+        qBytes = Bytes.toBytes(colDesc.getQualifier());
+        timestamp = System.currentTimeMillis();
+
+        List<MeasureDesc> measures = Lists.newArrayList();
+        for (MeasureDesc measure : colDesc.getMeasures()) {
+            measures.add(measure);
+        }
+        codec = new MeasureCodec(measures);
+        colValues = new Object[measures.size()];
+
+        this.startPosition = startPosition;
+
+    }
+
+    public KeyValue create(Text key, Object[] measureValues) {
+        return create(key.getBytes(), 0, key.getLength(), measureValues);
+    }
+
+    public KeyValue create(byte[] keyBytes, int keyOffset, int keyLength, Object[] measureValues) {
+        for (int i = 0; i < colValues.length; i++) {
+            colValues[i] = measureValues[startPosition + i];
+        }
+
+        valueBuf.clear();
+        codec.encode(colValues, valueBuf);
+
+        return create(keyBytes, keyOffset, keyLength, valueBuf.array(), 0, valueBuf.position());
+    }
+
+
+    public KeyValue create(byte[] keyBytes, int keyOffset, int keyLength, byte[] value, int voffset, int vlen) {
+        return new KeyValue(keyBytes, keyOffset, keyLength, //
+                cfBytes, 0, cfBytes.length, //
+                qBytes, 0, qBytes.length, //
+                timestamp, KeyValue.Type.Put, //
+                value, voffset, vlen);
+    }
+
+    public KeyValue create(Text key, byte[] value, int voffset, int vlen) {
+        return create(key.getBytes(), 0, key.getLength(), value, voffset, vlen);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/KeyValueCreator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/KeyValueCreator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/KeyValueCreator.java
new file mode 100644
index 0000000..3e49e27
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/KeyValueCreator.java
@@ -0,0 +1,104 @@
+package org.apache.kylin.storage.hbase.steps;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class KeyValueCreator {
+    byte[] cfBytes;
+    byte[] qBytes;
+    long timestamp;
+
+    int[] refIndex;
+    MeasureDesc[] refMeasures;
+
+    MeasureCodec codec;
+    Object[] colValues;
+    ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+    public boolean isFullCopy;
+
+    public KeyValueCreator(CubeDesc cubeDesc, HBaseColumnDesc colDesc) {
+
+        cfBytes = Bytes.toBytes(colDesc.getColumnFamilyName());
+        qBytes = Bytes.toBytes(colDesc.getQualifier());
+        timestamp = System.currentTimeMillis();
+
+        List<MeasureDesc> measures = cubeDesc.getMeasures();
+        String[] measureNames = getMeasureNames(cubeDesc);
+        String[] refs = colDesc.getMeasureRefs();
+
+        refIndex = new int[refs.length];
+        refMeasures = new MeasureDesc[refs.length];
+        for (int i = 0; i < refs.length; i++) {
+            refIndex[i] = indexOf(measureNames, refs[i]);
+            refMeasures[i] = measures.get(refIndex[i]);
+        }
+
+        codec = new MeasureCodec(refMeasures);
+        colValues = new Object[refs.length];
+
+        isFullCopy = true;
+        for (int i = 0; i < measures.size(); i++) {
+            if (refIndex.length <= i || refIndex[i] != i)
+                isFullCopy = false;
+        }
+    }
+
+    public KeyValue create(Text key, Object[] measureValues) {
+        return create(key.getBytes(), 0, key.getLength(), measureValues);
+    }
+
+    public KeyValue create(byte[] keyBytes, int keyOffset, int keyLength, Object[] measureValues) {
+        for (int i = 0; i < colValues.length; i++) {
+            colValues[i] = measureValues[refIndex[i]];
+        }
+
+        valueBuf.clear();
+        codec.encode(colValues, valueBuf);
+
+        return create(keyBytes, keyOffset, keyLength, valueBuf.array(), 0, valueBuf.position());
+    }
+
+
+    public KeyValue create(byte[] keyBytes, int keyOffset, int keyLength, byte[] value, int voffset, int vlen) {
+        return new KeyValue(keyBytes, keyOffset, keyLength, //
+                cfBytes, 0, cfBytes.length, //
+                qBytes, 0, qBytes.length, //
+                timestamp, KeyValue.Type.Put, //
+                value, voffset, vlen);
+    }
+
+    public KeyValue create(Text key, byte[] value, int voffset, int vlen) {
+        return create(key.getBytes(), 0, key.getLength(), value, voffset, vlen);
+    }
+
+    private int indexOf(String[] measureNames, String ref) {
+        for (int i = 0; i < measureNames.length; i++)
+            if (measureNames[i].equalsIgnoreCase(ref))
+                return i;
+
+        throw new IllegalArgumentException("Measure '" + ref + "' not found in " + Arrays.toString(measureNames));
+    }
+
+    private String[] getMeasureNames(CubeDesc cubeDesc) {
+        List<MeasureDesc> measures = cubeDesc.getMeasures();
+        String[] result = new String[measures.size()];
+        for (int i = 0; i < measures.size(); i++)
+            result[i] = measures.get(i).getName();
+        return result;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
new file mode 100644
index 0000000..df42560
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase.steps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Drop HBase tables that is no longer needed
+ */
+public class MergeGCStep extends AbstractExecutable {
+
+    private static final String OLD_HTABLES = "oldHTables";
+
+    private static final Logger logger = LoggerFactory.getLogger(MergeGCStep.class);
+
+    public MergeGCStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+
+        StringBuffer output = new StringBuffer();
+
+        List<String> oldTables = getOldHTables();
+        if (oldTables != null && oldTables.size() > 0) {
+            String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
+            Configuration conf = HBaseConfiguration.create();
+            HBaseAdmin admin = null;
+            try {
+                admin = new HBaseAdmin(conf);
+                for (String table : oldTables) {
+                    if (admin.tableExists(table)) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                        String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
+                        if (metadataUrlPrefix.equalsIgnoreCase(host)) {
+                            if (admin.isTableEnabled(table)) {
+                                admin.disableTable(table);
+                            }
+                            admin.deleteTable(table);
+                            logger.debug("Dropped htable: " + table);
+                            output.append("HBase table " + table + " is dropped. \n");
+                        } else {
+                            logger.debug("Skip htable: " + table);
+                            output.append("Skip htable: " + table + ". \n");
+                        }
+                    }
+                }
+
+            } catch (IOException e) {
+                output.append("Got error when drop HBase table, exiting... \n");
+                // This should not block the merge job; Orphans should be cleaned up in StorageCleanupJob
+                return new ExecuteResult(ExecuteResult.State.ERROR, output.append(e.getLocalizedMessage()).toString());
+            } finally {
+                if (admin != null)
+                    try {
+                        admin.close();
+                    } catch (IOException e) {
+                        logger.error(e.getLocalizedMessage());
+                    }
+            }
+        }
+
+        return new ExecuteResult(ExecuteResult.State.SUCCEED, output.toString());
+    }
+
+    public void setOldHTables(List<String> ids) {
+        setParam(OLD_HTABLES, StringUtils.join(ids, ","));
+    }
+
+    private List<String> getOldHTables() {
+        final String ids = getParam(OLD_HTABLES);
+        if (ids != null) {
+            final String[] splitted = StringUtils.split(ids, ",");
+            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
+            for (String id : splitted) {
+                result.add(id);
+            }
+            return result;
+        } else {
+            return Collections.emptyList();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/RowValueDecoder.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/RowValueDecoder.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/RowValueDecoder.java
new file mode 100644
index 0000000..877b71b
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/RowValueDecoder.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.metadata.measure.DoubleMutable;
+import org.apache.kylin.metadata.measure.LongMutable;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Collection;
+
+/**
+ */
+public class RowValueDecoder implements Cloneable {
+
+    private final HBaseColumnDesc hbaseColumn;
+    private final byte[] hbaseColumnFamily;
+    private final byte[] hbaseColumnQualifier;
+
+    private final MeasureCodec codec;
+    private final BitSet projectionIndex;
+    private final MeasureDesc[] measures;
+    private Object[] values;
+
+    public RowValueDecoder(HBaseColumnDesc hbaseColumn) {
+        this.hbaseColumn = hbaseColumn;
+        this.hbaseColumnFamily = Bytes.toBytes(hbaseColumn.getColumnFamilyName());
+        this.hbaseColumnQualifier = Bytes.toBytes(hbaseColumn.getQualifier());
+        this.projectionIndex = new BitSet();
+        this.measures = hbaseColumn.getMeasures();
+        this.codec = new MeasureCodec(measures);
+        this.values = new Object[measures.length];
+    }
+
+    public void decode(Result hbaseRow) {
+        decode(hbaseRow, true);
+    }
+
+    public void decode(Result hbaseRow, boolean convertToJavaObject) {
+        decode(hbaseRow.getValueAsByteBuffer(hbaseColumnFamily, hbaseColumnQualifier), convertToJavaObject);
+    }
+
+    public void decode(byte[] bytes) {
+        decode(bytes, true);
+    }
+
+    public void decode(byte[] bytes, boolean convertToJavaObject) {
+        decode(ByteBuffer.wrap(bytes), convertToJavaObject);
+    }
+
+    private void decode(ByteBuffer buffer, boolean convertToJavaObject) {
+        codec.decode(buffer, values);
+        if (convertToJavaObject) {
+            convertToJavaObjects(values, values, convertToJavaObject);
+        }
+    }
+
+    private void convertToJavaObjects(Object[] mapredObjs, Object[] results, boolean convertToJavaObject) {
+        for (int i = 0; i < mapredObjs.length; i++) {
+            Object o = mapredObjs[i];
+
+            if (o instanceof LongMutable)
+                o = ((LongMutable) o).get();
+            else if (o instanceof DoubleMutable)
+                o = ((DoubleMutable) o).get();
+
+            results[i] = o;
+        }
+    }
+
+    public void setIndex(int bitIndex) {
+        projectionIndex.set(bitIndex);
+    }
+
+    public HBaseColumnDesc getHBaseColumn() {
+        return hbaseColumn;
+    }
+
+    public BitSet getProjectionIndex() {
+        return projectionIndex;
+    }
+
+    public Object[] getValues() {
+        return values;
+    }
+
+    public MeasureDesc[] getMeasures() {
+        return measures;
+    }
+
+    public boolean hasMemHungryCountDistinct() {
+        for (int i = projectionIndex.nextSetBit(0); i >= 0; i = projectionIndex.nextSetBit(i + 1)) {
+            FunctionDesc func = measures[i].getFunction();
+            if (func.isCountDistinct() && !func.isHolisticCountDistinct()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public static boolean hasMemHungryCountDistinct(Collection<RowValueDecoder> rowValueDecoders) {
+        for (RowValueDecoder decoder : rowValueDecoders) {
+            if (decoder.hasMemHungryCountDistinct())
+                return true;
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
new file mode 100644
index 0000000..5b3949f
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IIManager;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.realization.RealizationStatusEnum;
+import org.apache.kylin.storage.hbase.HBaseConnection;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.*;
+import java.util.regex.Matcher;
+
+/**
+ * @author yangli9
+ */
+public class DeployCoprocessorCLI {
+
+    private static final Logger logger = LoggerFactory.getLogger(DeployCoprocessorCLI.class);
+
+    public static final String OBSERVER_CLS_NAME = "org.apache.kylin.storage.hbase.coprocessor.observer.AggregateRegionObserver";
+    public static final String ENDPOINT_CLS_NAMAE = "org.apache.kylin.storage.hbase.coprocessor.endpoint.IIEndpoint";
+
+    public static void main(String[] args) throws IOException {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        Configuration hconf = HBaseConnection.newHBaseConfiguration(kylinConfig.getStorageUrl());
+        FileSystem fileSystem = FileSystem.get(hconf);
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
+
+        String localCoprocessorJar = new File(args[0]).getAbsolutePath();
+        logger.info("Identify coprocessor jar " + localCoprocessorJar);
+
+        List<String> tableNames = getHTableNames(kylinConfig);
+        logger.info("Identify tables " + tableNames);
+
+        Set<String> oldJarPaths = getCoprocessorJarPaths(hbaseAdmin, tableNames);
+        logger.info("Old coprocessor jar: " + oldJarPaths);
+
+        Path hdfsCoprocessorJar = uploadCoprocessorJar(localCoprocessorJar, fileSystem, oldJarPaths);
+        logger.info("New coprocessor jar: " + hdfsCoprocessorJar);
+
+        List<String> processedTables = resetCoprocessorOnHTables(hbaseAdmin, hdfsCoprocessorJar, tableNames);
+
+        // Don't remove old jars, missing coprocessor jar will fail hbase
+        // removeOldJars(oldJarPaths, fileSystem);
+
+        hbaseAdmin.close();
+
+        logger.info("Processed " + processedTables);
+        logger.info("Active coprocessor jar: " + hdfsCoprocessorJar);
+    }
+
+    public static void deployCoprocessor(HTableDescriptor tableDesc) {
+        try {
+            initHTableCoprocessor(tableDesc);
+            logger.info("hbase table " + tableDesc.getName() + " deployed with coprocessor.");
+
+        } catch (Exception ex) {
+            logger.error("Error deploying coprocessor on " + tableDesc.getName(), ex);
+            logger.error("Will try creating the table without coprocessor.");
+        }
+    }
+
+    private static void initHTableCoprocessor(HTableDescriptor desc) throws IOException {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        Configuration hconf = HadoopUtil.getCurrentConfiguration();
+        FileSystem fileSystem = FileSystem.get(hconf);
+
+        String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();
+        Path hdfsCoprocessorJar = DeployCoprocessorCLI.uploadCoprocessorJar(localCoprocessorJar, fileSystem, null);
+
+        DeployCoprocessorCLI.addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
+    }
+
+    public static void addCoprocessorOnHTable(HTableDescriptor desc, Path hdfsCoprocessorJar) throws IOException {
+        logger.info("Add coprocessor on " + desc.getNameAsString());
+        desc.addCoprocessor(ENDPOINT_CLS_NAMAE, hdfsCoprocessorJar, 1000, null);
+        desc.addCoprocessor(OBSERVER_CLS_NAME, hdfsCoprocessorJar, 1001, null);
+    }
+
+    public static void resetCoprocessor(String tableName, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
+        logger.info("Disable " + tableName);
+        hbaseAdmin.disableTable(tableName);
+
+        logger.info("Unset coprocessor on " + tableName);
+        HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+        while (desc.hasCoprocessor(OBSERVER_CLS_NAME)) {
+            desc.removeCoprocessor(OBSERVER_CLS_NAME);
+        }
+        while (desc.hasCoprocessor(ENDPOINT_CLS_NAMAE)) {
+            desc.removeCoprocessor(ENDPOINT_CLS_NAMAE);
+        }
+
+        addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
+        hbaseAdmin.modifyTable(tableName, desc);
+
+        logger.info("Enable " + tableName);
+        hbaseAdmin.enableTable(tableName);
+    }
+
+    private static List<String> resetCoprocessorOnHTables(HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
+        List<String> processed = new ArrayList<String>();
+
+        for (String tableName : tableNames) {
+            try {
+                resetCoprocessor(tableName, hbaseAdmin, hdfsCoprocessorJar);
+                processed.add(tableName);
+            } catch (IOException ex) {
+                logger.error("Error processing " + tableName, ex);
+            }
+        }
+        return processed;
+    }
+
+    public static Path getNewestCoprocessorJar(KylinConfig config, FileSystem fileSystem) throws IOException {
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, config);
+        FileStatus newestJar = null;
+        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
+            if (fileStatus.getPath().toString().endsWith(".jar")) {
+                if (newestJar == null) {
+                    newestJar = fileStatus;
+                } else {
+                    if (newestJar.getModificationTime() < fileStatus.getModificationTime())
+                        newestJar = fileStatus;
+                }
+            }
+        }
+        if (newestJar == null)
+            return null;
+
+        Path path = newestJar.getPath().makeQualified(fileSystem.getUri(), null);
+        logger.info("The newest coprocessor is " + path.toString());
+        return path;
+    }
+
+    public static Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem, Set<String> oldJarPaths) throws IOException {
+        Path uploadPath = null;
+        File localCoprocessorFile = new File(localCoprocessorJar);
+
+        // check existing jars
+        if (oldJarPaths == null) {
+            oldJarPaths = new HashSet<String>();
+        }
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, KylinConfig.getInstanceFromEnv());
+        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
+            if (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;
+    }
+
+    private static Set<String> getCoprocessorJarPaths(HBaseAdmin hbaseAdmin, List<String> tableNames) throws IOException {
+        HashSet<String> result = new HashSet<String>();
+
+        for (String tableName : tableNames) {
+            HTableDescriptor tableDescriptor = null;
+            try {
+                tableDescriptor = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            } catch (TableNotFoundException e) {
+                logger.warn("Table not found " + tableName, e);
+                continue;
+            }
+
+            Matcher keyMatcher;
+            Matcher valueMatcher;
+            for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : tableDescriptor.getValues().entrySet()) {
+                keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
+                if (!keyMatcher.matches()) {
+                    continue;
+                }
+                valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes.toString(e.getValue().get()));
+                if (!valueMatcher.matches()) {
+                    continue;
+                }
+
+                String jarPath = valueMatcher.group(1).trim();
+                String clsName = valueMatcher.group(2).trim();
+
+                if (OBSERVER_CLS_NAME.equals(clsName)) {
+                    result.add(jarPath);
+                }
+            }
+        }
+
+        return result;
+    }
+
+    private static List<String> getHTableNames(KylinConfig config) {
+        CubeManager cubeMgr = CubeManager.getInstance(config);
+
+        ArrayList<String> result = new ArrayList<String>();
+        for (CubeInstance cube : cubeMgr.listAllCubes()) {
+            for (CubeSegment seg : cube.getSegments(SegmentStatusEnum.READY)) {
+                String tableName = seg.getStorageLocationIdentifier();
+                if (StringUtils.isBlank(tableName) == false) {
+                    result.add(tableName);
+                    System.out.println("added new table: " + tableName);
+                }
+            }
+        }
+
+        for (IIInstance ii : IIManager.getInstance(config).listAllIIs()) {
+            if (ii.getStatus() == RealizationStatusEnum.READY) {
+                for (IISegment seg : ii.getSegments()) {//streaming segment is never "READY"
+                    String tableName = seg.getStorageLocationIdentifier();
+                    if (StringUtils.isBlank(tableName) == false) {
+                        result.add(tableName);
+                        System.out.println("added new table: " + tableName);
+                    }
+                }
+            }
+        }
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
new file mode 100644
index 0000000..ba0da00
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+/** This class will come with HBase 2.0 in package org.apache.hadoop.hbase.util **/
+package org.apache.kylin.storage.hbase.util;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HBaseRegionSizeCalculator {
+
+    private static final Logger logger = LoggerFactory.getLogger(HBaseRegionSizeCalculator.class);
+
+    /**
+     * Maps each region to its size in bytes.
+     **/
+    private final Map<byte[], Long> sizeMap = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
+
+    static final String ENABLE_REGIONSIZECALCULATOR = "hbase.regionsizecalculator.enable";
+
+    /**
+     * Computes size of each region for table and given column families.
+     * */
+    public HBaseRegionSizeCalculator(HTable table) throws IOException {
+        this(table, new HBaseAdmin(table.getConfiguration()));
+    }
+
+    /** Constructor for unit testing */
+    HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
+
+        try {
+            if (!enabled(table.getConfiguration())) {
+                logger.info("Region size calculation disabled.");
+                return;
+            }
+
+            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
+
+            // Get regions for table.
+            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
+            Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+
+            for (HRegionInfo regionInfo : tableRegionInfos) {
+                tableRegions.add(regionInfo.getRegionName());
+            }
+
+            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
+            Collection<ServerName> servers = clusterStatus.getServers();
+            final long megaByte = 1024L * 1024L;
+
+            // Iterate all cluster regions, filter regions from our table and
+            // compute their size.
+            for (ServerName serverName : servers) {
+                ServerLoad serverLoad = clusterStatus.getLoad(serverName);
+
+                for (RegionLoad regionLoad : serverLoad.getRegionsLoad().values()) {
+                    byte[] regionId = regionLoad.getName();
+
+                    if (tableRegions.contains(regionId)) {
+
+                        long regionSizeBytes = regionLoad.getStorefileSizeMB() * megaByte;
+                        sizeMap.put(regionId, regionSizeBytes);
+
+                        // logger.info("Region " + regionLoad.getNameAsString()
+                        // + " has size " + regionSizeBytes);
+                    }
+                }
+            }
+        } finally {
+            hBaseAdmin.close();
+        }
+
+    }
+
+    boolean enabled(Configuration configuration) {
+        return configuration.getBoolean(ENABLE_REGIONSIZECALCULATOR, true);
+    }
+
+    /**
+     * Returns size of given region in bytes. Returns 0 if region was not found.
+     **/
+    public long getRegionSize(byte[] regionId) {
+        Long size = sizeMap.get(regionId);
+        if (size == null) {
+            logger.info("Unknown region:" + Arrays.toString(regionId));
+            return 0;
+        } else {
+            return size;
+        }
+    }
+
+    public Map<byte[], Long> getRegionSizeMap() {
+        return Collections.unmodifiableMap(sizeMap);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/LZOSupportnessChecker.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/LZOSupportnessChecker.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/LZOSupportnessChecker.java
new file mode 100644
index 0000000..3645bed
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/LZOSupportnessChecker.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase.util;
+
+import org.apache.hadoop.hbase.util.CompressionTest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
+/**
+ */
+public class LZOSupportnessChecker {
+    private static final Logger log = LoggerFactory.getLogger(LZOSupportnessChecker.class);
+
+    public static boolean getSupportness() {
+        try {
+            File temp = File.createTempFile("test", ".tmp");
+            CompressionTest.main(new String[] { "file://" + temp.getAbsolutePath(), "lzo" });
+        } catch (Exception e) {
+            log.warn("LZO support is disabled. Fail to compress file with lzo: " + e.toString());
+            return false;
+        }
+        return true;
+    }
+
+    public static void main(String[] args) throws Exception {
+        System.out.println("LZO supported by current env? " + getSupportness());
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/CreateHTableTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/CreateHTableTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/CreateHTableTest.java
new file mode 100644
index 0000000..8b09f05
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/CreateHTableTest.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.storage.hbase;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.storage.hbase.steps.CreateHTableJob;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CreateHTableTest extends LocalFileMetadataTestCase {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws Exception {
+        conf = new Configuration();
+        conf.set("fs.default.name", "file:///");
+        conf.set("mapred.job.tracker", "local");
+        this.createTestMetadata();
+
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testGetSplits() throws IllegalArgumentException, Exception {
+        CreateHTableJob c = new CreateHTableJob();
+
+        String input = "src/test/resources/partition_list/part-r-00000";
+
+        byte[][] splits = c.getSplits(conf, new Path(input));
+
+        assertEquals(497, splits.length);
+        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 15, -1, 11, 51, -45, 2 }, splits[0]);
+        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 3, -1, -1, -54, -61, 109, -44, 1 }, splits[496]);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ITHdfsOpsTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ITHdfsOpsTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ITHdfsOpsTest.java
new file mode 100644
index 0000000..5381187
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ITHdfsOpsTest.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.KylinConfig;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ */
+public class ITHdfsOpsTest extends HBaseMetadataTestCase {
+
+    FileSystem fileSystem;
+
+    @Before
+    public void setup() throws Exception {
+
+        this.createTestMetadata();
+
+        Configuration hconf = new Configuration();
+
+        fileSystem = FileSystem.get(hconf);
+    }
+
+    @Test
+    public void TestPath() throws IOException {
+        String hdfsWorkingDirectory = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory();
+        Path coprocessorDir = new Path(hdfsWorkingDirectory, "test");
+        fileSystem.mkdirs(coprocessorDir);
+
+        Path newFile = new Path(coprocessorDir, "test_file");
+        newFile = newFile.makeQualified(fileSystem.getUri(), null);
+        FSDataOutputStream stream = fileSystem.create(newFile);
+        stream.write(new byte[] { 0, 1, 2 });
+        stream.close();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/RowValueDecoderTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/RowValueDecoderTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/RowValueDecoderTest.java
index dcc5b06..3ff1fc1 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/RowValueDecoderTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/RowValueDecoderTest.java
@@ -35,6 +35,7 @@ import org.apache.kylin.metadata.measure.LongMutable;
 import org.apache.kylin.metadata.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.storage.hbase.steps.RowValueDecoder;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/TestHbaseClient.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/TestHbaseClient.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/TestHbaseClient.java
new file mode 100644
index 0000000..0ea037b
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/TestHbaseClient.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.kylin.common.util.Bytes;
+
+/**
+ */
+public class TestHbaseClient {
+
+    private static boolean reverse = false;
+
+    public static void foo(int n, int k) {
+        int t = k;
+        if (n - k < k) {
+            t = n - k;
+            reverse = true;
+        }
+        boolean[] flags = new boolean[n];
+        inner(flags, 0, t);
+    }
+
+    private static void print(boolean[] flags) {
+        for (int i = 0; i < flags.length; i++) {
+            if (!reverse) {
+                if (flags[i])
+                    System.out.print("0");
+                else
+                    System.out.print("1");
+            } else {
+                if (flags[i])
+                    System.out.print("1");
+                else
+                    System.out.print("0");
+
+            }
+        }
+        System.out.println();
+
+    }
+
+    private static void inner(boolean[] flags, int start, int remaining) {
+        if (remaining <= 0) {
+            print(flags);
+            return;
+        }
+
+        if (flags.length - start < remaining) {
+            return;
+        }
+
+        // write at flags[start]
+        flags[start] = true;
+        inner(flags, start + 1, remaining - 1);
+
+        // not write at flags[start]
+        flags[start] = false;
+        inner(flags, start + 1, remaining);
+    }
+
+    public static void main(String[] args) throws IOException {
+        foo(6, 5);
+        foo(5, 2);
+        foo(3, 0);
+
+        Configuration conf = HBaseConfiguration.create();
+        conf.set("hbase.zookeeper.quorum", "hbase_host");
+        conf.set("zookeeper.znode.parent", "/hbase-unsecure");
+
+        HTable table = new HTable(conf, "test1");
+        Put put = new Put(Bytes.toBytes("row1"));
+
+        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual1"), Bytes.toBytes("val1"));
+        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual2"), Bytes.toBytes("val2"));
+
+        table.put(put);
+        table.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper2Test.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper2Test.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper2Test.java
new file mode 100644
index 0000000..733de1d
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper2Test.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.storage.hbase.steps;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class CubeHFileMapper2Test extends LocalFileMetadataTestCase {
+
+    String cubeName = "test_kylin_cube_with_slr_ready";
+
+    MeasureCodec codec;
+    ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+    Object[] outKV = new Object[2];
+
+    @Before
+    public void setup() throws Exception {
+        this.createTestMetadata();
+        // hack for distributed cache
+        FileUtils.deleteDirectory(new File("../job/meta"));
+        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), new File("../job/meta"));
+        CubeDesc desc = CubeManager.getInstance(getTestConfig()).getCube(cubeName).getDescriptor();
+        codec = new MeasureCodec(desc.getMeasures());
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+        FileUtils.deleteDirectory(new File("../job/meta"));
+    }
+
+    @Test
+    public void testBasic() throws Exception {
+
+        Configuration hconf = new Configuration();
+        Context context = MockupMapContext.create(hconf, getTestConfig().getMetadataUrl(), cubeName, outKV);
+
+        CubeHFileMapper mapper = new CubeHFileMapper();
+        mapper.setup(context);
+
+        Text key = new Text("not important");
+        Text value = new Text(new byte[] { 2, 2, 51, -79, 1 });
+
+        mapper.map(key, value, context);
+
+        ImmutableBytesWritable outKey = (ImmutableBytesWritable) outKV[0];
+        KeyValue outValue = (KeyValue) outKV[1];
+
+        assertTrue(Bytes.compareTo(key.getBytes(), 0, key.getLength(), outKey.get(), outKey.getOffset(), outKey.getLength()) == 0);
+
+        assertTrue(Bytes.compareTo(value.getBytes(), 0, value.getLength(), outValue.getValueArray(), outValue.getValueOffset(), outValue.getValueLength()) == 0);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapperTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapperTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapperTest.java
new file mode 100644
index 0000000..0c148e7
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapperTest.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.steps;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.storage.hbase.steps.CubeHFileMapper;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CubeHFileMapperTest {
+
+    MapDriver<Text, Text, ImmutableBytesWritable, KeyValue> mapDriver;
+
+    private String cube_name = "FLAT_ITEM_CUBE";
+
+    @Before
+    public void setUp() {
+        CubeHFileMapper mapper = new CubeHFileMapper();
+        mapDriver = MapDriver.newMapDriver(mapper);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    @Ignore("not maintaining")
+    public void testMapper2() throws IOException {
+        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cube_name);
+
+        mapDriver.addInput(new Text("52010tech"), new Text("35.432"));
+
+        List<Pair<ImmutableBytesWritable, KeyValue>> result = mapDriver.run();
+
+        assertEquals(2, result.size());
+
+        byte[] bytes = { 0, 0, 0, 0, 0, 0, 0, 119, 33, 0, 22, 1, 0, 121, 7 };
+        ImmutableBytesWritable key = new ImmutableBytesWritable(bytes);
+
+        Pair<ImmutableBytesWritable, KeyValue> p1 = result.get(0);
+        Pair<ImmutableBytesWritable, KeyValue> p2 = result.get(1);
+
+        assertEquals(key, p1.getFirst());
+        assertEquals("cf1", new String(p1.getSecond().getFamily()));
+        assertEquals("usd_amt", new String(p1.getSecond().getQualifier()));
+        assertEquals("35.43", new String(p1.getSecond().getValue()));
+
+        assertEquals(key, p2.getFirst());
+        assertEquals("cf1", new String(p2.getSecond().getFamily()));
+        assertEquals("item_count", new String(p2.getSecond().getQualifier()));
+        assertEquals("2", new String(p2.getSecond().getValue()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
new file mode 100644
index 0000000..d5c3f60
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
+import org.apache.hadoop.security.Credentials;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class MockupMapContext {
+
+    public static Context create(final Configuration hconf, String metadataUrl, String cubeName, final Object[] outKV) {
+
+        hconf.set(BatchConstants.CFG_CUBE_NAME, cubeName);
+
+        return new WrappedMapper().getMapContext(new MapContext() {
+
+            @Override
+            public boolean nextKeyValue() throws IOException, InterruptedException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Object getCurrentKey() throws IOException, InterruptedException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Object getCurrentValue() throws IOException, InterruptedException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public void write(Object key, Object value) throws IOException, InterruptedException {
+                System.out.println("Write -- k:" + key + ", v:" + value);
+                if (outKV != null) {
+                    outKV[0] = key;
+                    outKV[1] = value;
+                }
+            }
+
+            @Override
+            public OutputCommitter getOutputCommitter() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public TaskAttemptID getTaskAttemptID() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public void setStatus(String msg) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getStatus() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public float getProgress() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Counter getCounter(Enum<?> counterName) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Counter getCounter(String groupName, String counterName) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Configuration getConfiguration() {
+                return hconf;
+            }
+
+            @Override
+            public Credentials getCredentials() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public JobID getJobID() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public int getNumReduceTasks() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path getWorkingDirectory() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getOutputKeyClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getOutputValueClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getMapOutputKeyClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getMapOutputValueClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getJobName() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends InputFormat<?, ?>> getInputFormatClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Mapper<?, ?, ?, ?>> getMapperClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Reducer<?, ?, ?, ?>> getCombinerClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Reducer<?, ?, ?, ?>> getReducerClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends OutputFormat<?, ?>> getOutputFormatClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Partitioner<?, ?>> getPartitionerClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public RawComparator<?> getSortComparator() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getJar() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public RawComparator<?> getGroupingComparator() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getJobSetupCleanupNeeded() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getTaskCleanupNeeded() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getProfileEnabled() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getProfileParams() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public IntegerRanges getProfileTaskRange(boolean isMap) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getUser() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getSymlink() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getArchiveClassPaths() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public URI[] getCacheArchives() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public URI[] getCacheFiles() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getLocalCacheArchives() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getLocalCacheFiles() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getFileClassPaths() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String[] getArchiveTimestamps() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String[] getFileTimestamps() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public int getMaxMapAttempts() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public int getMaxReduceAttempts() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public void progress() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public InputSplit getInputSplit() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public RawComparator<?> getCombinerKeyGroupingComparator() {
+                throw new NotImplementedException();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionJobTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionJobTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionJobTest.java
new file mode 100644
index 0000000..78ea71f
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RangeKeyDistributionJobTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.steps;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.engine.mr.steps.RangeKeyDistributionJob;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionJobTest extends LocalFileMetadataTestCase {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws Exception {
+        conf = new Configuration();
+        conf.set("fs.default.name", "file:///");
+        conf.set("mapred.job.tracker", "local");
+
+        // for local runner out-of-memory issue
+        conf.set("mapreduce.task.io.sort.mb", "10");
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void testJob() throws Exception {
+        String input = "src/test/resources/data/base_cuboid/,src/test/resources/data/6d_cuboid/";
+        String output = "target/test-output/key_distribution_range/";
+        String jobname = "calculate_splits";
+        String cubename = "test_kylin_cube_with_slr_ready";
+
+        FileUtil.fullyDelete(new File(output));
+
+        String[] args = { "-input", input, "-output", output, "-jobname", jobname, "-cubename", cubename };
+        assertEquals("Job failed", 0, ToolRunner.run(conf, new RangeKeyDistributionJob(), args));
+    }
+
+}


[07/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/storage/hbase/MergeGCStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/storage/hbase/MergeGCStep.java b/job/src/main/java/org/apache/kylin/storage/hbase/MergeGCStep.java
deleted file mode 100644
index 5fc57e3..0000000
--- a/job/src/main/java/org/apache/kylin/storage/hbase/MergeGCStep.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage.hbase;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-import org.apache.kylin.metadata.realization.IRealizationConstants;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-
-/**
- * Drop HBase tables that is no longer needed
- */
-public class MergeGCStep extends AbstractExecutable {
-
-    private static final String OLD_HTABLES = "oldHTables";
-
-    private static final Logger logger = LoggerFactory.getLogger(MergeGCStep.class);
-
-    public MergeGCStep() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-
-        StringBuffer output = new StringBuffer();
-
-        List<String> oldTables = getOldHTables();
-        if (oldTables != null && oldTables.size() > 0) {
-            String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Configuration conf = HBaseConfiguration.create();
-            HBaseAdmin admin = null;
-            try {
-                admin = new HBaseAdmin(conf);
-                for (String table : oldTables) {
-                    if (admin.tableExists(table)) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
-                        String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
-                        if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(table)) {
-                                admin.disableTable(table);
-                            }
-                            admin.deleteTable(table);
-                            logger.debug("Dropped htable: " + table);
-                            output.append("HBase table " + table + " is dropped. \n");
-                        } else {
-                            logger.debug("Skip htable: " + table);
-                            output.append("Skip htable: " + table + ". \n");
-                        }
-                    }
-                }
-
-            } catch (IOException e) {
-                output.append("Got error when drop HBase table, exiting... \n");
-                // This should not block the merge job; Orphans should be cleaned up in StorageCleanupJob
-                return new ExecuteResult(ExecuteResult.State.ERROR, output.append(e.getLocalizedMessage()).toString());
-            } finally {
-                if (admin != null)
-                    try {
-                        admin.close();
-                    } catch (IOException e) {
-                        logger.error(e.getLocalizedMessage());
-                    }
-            }
-        }
-
-        return new ExecuteResult(ExecuteResult.State.SUCCEED, output.toString());
-    }
-
-    public void setOldHTables(List<String> ids) {
-        setParam(OLD_HTABLES, StringUtils.join(ids, ","));
-    }
-
-    private List<String> getOldHTables() {
-        final String ids = getParam(OLD_HTABLES);
-        if (ids != null) {
-            final String[] splitted = StringUtils.split(ids, ",");
-            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
-            for (String id : splitted) {
-                result.add(id);
-            }
-            return result;
-        } else {
-            return Collections.emptyList();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/BaseTestExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BaseTestExecutable.java b/job/src/test/java/org/apache/kylin/job/BaseTestExecutable.java
deleted file mode 100644
index 5ab97c9..0000000
--- a/job/src/test/java/org/apache/kylin/job/BaseTestExecutable.java
+++ /dev/null
@@ -1,32 +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;
-
-import org.apache.kylin.job.execution.AbstractExecutable;
-
-/**
- */
-public abstract class BaseTestExecutable extends AbstractExecutable {
-
-
-    public BaseTestExecutable() {
-        super();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java b/job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java
deleted file mode 100644
index d034a73..0000000
--- a/job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java
+++ /dev/null
@@ -1,24 +0,0 @@
-package org.apache.kylin.job;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- */
-public class BasicLocalMetaTest extends LocalFileMetadataTestCase {
-    @Before
-    public void setUp() throws Exception {
-        this.createTestMetadata();
-    }
-
-    @After
-    public void after() throws Exception {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void basicTest() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java b/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
index 1b626d2..8966742 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
@@ -49,7 +49,7 @@ import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.impl.threadpool.DefaultScheduler;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
-import org.apache.kylin.storage.hbase.ZookeeperJobLock;
+import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java b/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
index 41457ed..4c335e6 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
@@ -39,7 +39,7 @@ import org.apache.kylin.job.invertedindex.IIJobBuilder;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.storage.hbase.HBaseMetadataTestCase;
-import org.apache.kylin.storage.hbase.ZookeeperJobLock;
+import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
 import org.junit.*;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/DeployUtil.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/DeployUtil.java b/job/src/test/java/org/apache/kylin/job/DeployUtil.java
index 7ebfac0..ab172c0 100644
--- a/job/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/job/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -18,7 +18,13 @@
 
 package org.apache.kylin.job;
 
-import com.google.common.collect.Lists;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
@@ -29,11 +35,10 @@ import org.apache.kylin.common.persistence.ResourceTool;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.CliCommandExecutor;
 import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.job.dataGen.FactTableGenerator;
-import org.apache.kylin.job.hadoop.hive.SqlHiveDataTypeMapping;
 import org.apache.kylin.job.streaming.KafkaDataLoader;
 import org.apache.kylin.job.streaming.StreamingTableDataGenerator;
 import org.apache.kylin.metadata.MetadataManager;
@@ -49,8 +54,7 @@ import org.apache.maven.model.io.xpp3.MavenXpp3Reader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.*;
-import java.util.List;
+import com.google.common.collect.Lists;
 
 public class DeployUtil {
     @SuppressWarnings("unused")
@@ -262,7 +266,7 @@ public class DeployUtil {
             if (i > 0) {
                 ddl.append(",");
             }
-            ddl.append(col.getName() + " " + SqlHiveDataTypeMapping.getHiveDataType((col.getDatatype())) + "\n");
+            ddl.append(col.getName() + " " + getHiveDataType((col.getDatatype())) + "\n");
         }
 
         ddl.append(")" + "\n");
@@ -272,4 +276,10 @@ public class DeployUtil {
         return new String[] { dropsql, ddl.toString() };
     }
 
+    private static String getHiveDataType(String javaDataType) {
+        String hiveDataType = javaDataType.toLowerCase().startsWith("varchar") ? "string" : javaDataType;
+        hiveDataType = javaDataType.toLowerCase().startsWith("integer") ? "int" : hiveDataType;
+
+        return hiveDataType.toLowerCase();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/ErrorTestExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/ErrorTestExecutable.java b/job/src/test/java/org/apache/kylin/job/ErrorTestExecutable.java
deleted file mode 100644
index ac4828e..0000000
--- a/job/src/test/java/org/apache/kylin/job/ErrorTestExecutable.java
+++ /dev/null
@@ -1,41 +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;
-
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-
-/**
- */
-public class ErrorTestExecutable extends BaseTestExecutable {
-
-    public ErrorTestExecutable() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        try {
-            Thread.sleep(1000);
-        } catch (InterruptedException e) {
-        }
-        throw new RuntimeException("test error");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/FailedTestExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/FailedTestExecutable.java b/job/src/test/java/org/apache/kylin/job/FailedTestExecutable.java
deleted file mode 100644
index e18cc0d..0000000
--- a/job/src/test/java/org/apache/kylin/job/FailedTestExecutable.java
+++ /dev/null
@@ -1,41 +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;
-
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-
-/**
- */
-public class FailedTestExecutable extends BaseTestExecutable {
-
-    public FailedTestExecutable() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        try {
-            Thread.sleep(1000);
-        } catch (InterruptedException e) {
-        }
-        return new ExecuteResult(ExecuteResult.State.FAILED, "failed");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/SelfStopExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/SelfStopExecutable.java b/job/src/test/java/org/apache/kylin/job/SelfStopExecutable.java
deleted file mode 100644
index b4f6a98..0000000
--- a/job/src/test/java/org/apache/kylin/job/SelfStopExecutable.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job;
-
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-
-/**
- */
-public class SelfStopExecutable extends BaseTestExecutable {
-
-    public SelfStopExecutable() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        try {
-            Thread.sleep(5000);
-        } catch (InterruptedException e) {
-        }
-        if (isDiscarded()) {
-            return new ExecuteResult(ExecuteResult.State.STOPPED, "stopped");
-        } else {
-            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/SucceedTestExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/SucceedTestExecutable.java b/job/src/test/java/org/apache/kylin/job/SucceedTestExecutable.java
deleted file mode 100644
index 1421f10..0000000
--- a/job/src/test/java/org/apache/kylin/job/SucceedTestExecutable.java
+++ /dev/null
@@ -1,41 +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;
-
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-
-/**
- */
-public class SucceedTestExecutable extends BaseTestExecutable {
-
-    public SucceedTestExecutable() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        try {
-            Thread.sleep(1000);
-        } catch (InterruptedException e) {
-        }
-        return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/CopySeq.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/CopySeq.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/CopySeq.java
deleted file mode 100644
index 505cdf8..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/CopySeq.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.cube;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.SequenceFile.Writer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * @author yangli9
- * 
- */
-public class CopySeq {
-
-    public static void main(String[] args) throws IOException {
-        copyTo64MB(args[0], args[1]);
-    }
-
-    public static void copyTo64MB(String src, String dst) throws IOException {
-        Configuration hconf = new Configuration();
-        Path srcPath = new Path(src);
-        Path dstPath = new Path(dst);
-
-        FileSystem fs = FileSystem.get(hconf);
-        long srcSize = fs.getFileStatus(srcPath).getLen();
-        int copyTimes = (int) (67108864 / srcSize); // 64 MB
-        System.out.println("Copy " + copyTimes + " times");
-
-        Reader reader = new Reader(hconf, SequenceFile.Reader.file(srcPath));
-        Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), hconf);
-        Text value = new Text();
-
-        Writer writer = SequenceFile.createWriter(hconf, Writer.file(dstPath), Writer.keyClass(key.getClass()), Writer.valueClass(Text.class), Writer.compression(CompressionType.BLOCK, getLZOCodec(hconf)));
-
-        int count = 0;
-        while (reader.next(key, value)) {
-            for (int i = 0; i < copyTimes; i++) {
-                writer.append(key, value);
-                count++;
-            }
-        }
-
-        System.out.println("Len: " + writer.getLength());
-        System.out.println("Rows: " + count);
-
-        reader.close();
-        writer.close();
-    }
-
-    static CompressionCodec getLZOCodec(Configuration hconf) {
-        CompressionCodecFactory factory = new CompressionCodecFactory(hconf);
-        return factory.getCodecByClassName("org.apache.hadoop.io.compress.LzoCodec");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper2Test.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper2Test.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper2Test.java
deleted file mode 100644
index 5262936..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper2Test.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.nio.ByteBuffer;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Mapper.Context;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.apache.kylin.cube.model.CubeDesc;
-
-/**
- * @author yangli9
- * 
- */
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class CubeHFileMapper2Test extends LocalFileMetadataTestCase {
-
-    String cubeName = "test_kylin_cube_with_slr_ready";
-
-    MeasureCodec codec;
-    ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-    Object[] outKV = new Object[2];
-
-    @Before
-    public void setup() throws Exception {
-        this.createTestMetadata();
-        // hack for distributed cache
-        FileUtils.deleteDirectory(new File("../job/meta"));
-        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), new File("../job/meta"));
-        CubeDesc desc = CubeManager.getInstance(getTestConfig()).getCube(cubeName).getDescriptor();
-        codec = new MeasureCodec(desc.getMeasures());
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-        FileUtils.deleteDirectory(new File("../job/meta"));
-    }
-
-    @Test
-    public void testBasic() throws Exception {
-
-        Configuration hconf = new Configuration();
-        Context context = MockupMapContext.create(hconf, getTestConfig().getMetadataUrl(), cubeName, outKV);
-
-        CubeHFileMapper mapper = new CubeHFileMapper();
-        mapper.setup(context);
-
-        Text key = new Text("not important");
-        Text value = new Text(new byte[] { 2, 2, 51, -79, 1 });
-
-        mapper.map(key, value, context);
-
-        ImmutableBytesWritable outKey = (ImmutableBytesWritable) outKV[0];
-        KeyValue outValue = (KeyValue) outKV[1];
-
-        assertTrue(Bytes.compareTo(key.getBytes(), 0, key.getLength(), outKey.get(), outKey.getOffset(), outKey.getLength()) == 0);
-
-        assertTrue(Bytes.compareTo(value.getBytes(), 0, value.getLength(), outValue.getValueArray(), outValue.getValueOffset(), outValue.getValueLength()) == 0);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapperTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapperTest.java
deleted file mode 100644
index 7b86f1b..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapperTest.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.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import org.apache.kylin.job.constant.BatchConstants;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-public class CubeHFileMapperTest {
-
-    MapDriver<Text, Text, ImmutableBytesWritable, KeyValue> mapDriver;
-
-    private String cube_name = "FLAT_ITEM_CUBE";
-
-    @Before
-    public void setUp() {
-        CubeHFileMapper mapper = new CubeHFileMapper();
-        mapDriver = MapDriver.newMapDriver(mapper);
-    }
-
-    @SuppressWarnings("deprecation")
-    @Test
-    @Ignore("not maintaining")
-    public void testMapper2() throws IOException {
-        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cube_name);
-
-        mapDriver.addInput(new Text("52010tech"), new Text("35.432"));
-
-        List<Pair<ImmutableBytesWritable, KeyValue>> result = mapDriver.run();
-
-        assertEquals(2, result.size());
-
-        byte[] bytes = { 0, 0, 0, 0, 0, 0, 0, 119, 33, 0, 22, 1, 0, 121, 7 };
-        ImmutableBytesWritable key = new ImmutableBytesWritable(bytes);
-
-        Pair<ImmutableBytesWritable, KeyValue> p1 = result.get(0);
-        Pair<ImmutableBytesWritable, KeyValue> p2 = result.get(1);
-
-        assertEquals(key, p1.getFirst());
-        assertEquals("cf1", new String(p1.getSecond().getFamily()));
-        assertEquals("usd_amt", new String(p1.getSecond().getQualifier()));
-        assertEquals("35.43", new String(p1.getSecond().getValue()));
-
-        assertEquals(key, p2.getFirst());
-        assertEquals("cf1", new String(p2.getSecond().getFamily()));
-        assertEquals("item_count", new String(p2.getSecond().getQualifier()));
-        assertEquals("2", new String(p2.getSecond().getValue()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeReducerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeReducerTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeReducerTest.java
deleted file mode 100644
index 5369284..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/CubeReducerTest.java
+++ /dev/null
@@ -1,122 +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.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- */
-public class CubeReducerTest extends LocalFileMetadataTestCase {
-
-    ReduceDriver<Text, Text, Text, Text> reduceDriver;
-    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
-
-    ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-
-    @Before
-    public void setUp() throws Exception {
-        createTestMetadata();
-
-        // hack for distributed cache
-        FileUtils.deleteDirectory(new File("../job/meta"));
-        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), new File("../job/meta"));
-
-        CuboidReducer reducer = new CuboidReducer();
-        reduceDriver = ReduceDriver.newReduceDriver(reducer);
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-        FileUtils.deleteDirectory(new File("../job/meta"));
-    }
-
-    @Test
-    public void testReducer() throws Exception {
-
-        reduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, "test_kylin_cube_with_slr_ready");
-
-        CubeDesc cubeDesc = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_ready").getDescriptor();
-        MeasureCodec codec = new MeasureCodec(cubeDesc.getMeasures());
-
-        Text key1 = new Text("72010ustech");
-        List<Text> values1 = new ArrayList<Text>();
-        values1.add(newValueText(codec, "15.09", "15.09", "15.09", 1));
-        values1.add(newValueText(codec, "20.34", "20.34", "20.34", 1));
-        values1.add(newValueText(codec, "10", "10", "10", 1));
-
-        Text key2 = new Text("1tech");
-        List<Text> values2 = new ArrayList<Text>();
-        values2.add(newValueText(codec, "15.09", "15.09", "15.09", 1));
-        values2.add(newValueText(codec, "20.34", "20.34", "20.34", 1));
-
-        Text key3 = new Text("0");
-        List<Text> values3 = new ArrayList<Text>();
-        values3.add(newValueText(codec, "146.52", "146.52", "146.52", 4));
-
-        reduceDriver.withInput(key1, values1);
-        reduceDriver.withInput(key2, values2);
-        reduceDriver.withInput(key3, values3);
-
-        List<Pair<Text, Text>> result = reduceDriver.run();
-
-        Pair<Text, Text> p1 = new Pair<Text, Text>(new Text("72010ustech"), newValueText(codec, "45.43", "10", "20.34", 3));
-        Pair<Text, Text> p2 = new Pair<Text, Text>(new Text("1tech"), newValueText(codec, "35.43", "15.09", "20.34", 2));
-        Pair<Text, Text> p3 = new Pair<Text, Text>(new Text("0"), newValueText(codec, "146.52", "146.52", "146.52", 4));
-
-        assertEquals(3, result.size());
-
-        assertTrue(result.contains(p1));
-        assertTrue(result.contains(p2));
-        assertTrue(result.contains(p3));
-    }
-
-    private Text newValueText(MeasureCodec codec, String sum, String min, String max, int count) {
-        Object[] values = new Object[] { new BigDecimal(sum), new BigDecimal(min), new BigDecimal(max), new LongMutable(count) };
-
-        buf.clear();
-        codec.encode(values, buf);
-
-        Text t = new Text();
-        t.set(buf.array(), 0, buf.position());
-        return t;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/HiveToBaseCuboidMapperPerformanceTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/HiveToBaseCuboidMapperPerformanceTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/HiveToBaseCuboidMapperPerformanceTest.java
deleted file mode 100644
index cf9cfe0..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/HiveToBaseCuboidMapperPerformanceTest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Mapper.Context;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * @author yangli9
- * 
- */
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class HiveToBaseCuboidMapperPerformanceTest {
-
-    String metadataUrl = "hbase:yadesk00:2181:/hbase-unsecure";
-    String cubeName = "test_kylin_cube_with_slr";
-    Path srcPath = new Path("/download/test_kylin_cube_with_slr_intermediate_table_64mb.seq");
-
-    @Ignore("convenient trial tool for dev")
-    @Test
-    public void test() throws IOException, InterruptedException {
-        Configuration hconf = new Configuration();
-        HiveToBaseCuboidMapper mapper = new HiveToBaseCuboidMapper();
-        Context context = MockupMapContext.create(hconf, metadataUrl, cubeName, null);
-
-        mapper.setup(context);
-
-        Reader reader = new Reader(hconf, SequenceFile.Reader.file(srcPath));
-        Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), hconf);
-        Text value = new Text();
-
-        while (reader.next(key, value)) {
-            mapper.map(key, value, context);
-        }
-
-        reader.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
deleted file mode 100644
index 932670f..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
+++ /dev/null
@@ -1,82 +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.cube;
-
-///*
-// * Copyright 2013-2014 eBay Software Foundation
-// *
-// * Licensed under the Apache License, Version 2.0 (the "License");
-// * you may not use this file except in compliance with the License.
-// * You may obtain a copy of the License at
-// *
-// *   http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT 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.index.cube;
-//
-//import static org.junit.Assert.*;
-//
-//import java.io.File;
-//import java.io.IOException;
-//
-//import org.apache.hadoop.conf.Configuration;
-//import org.apache.hadoop.fs.FileUtil;
-//import org.apache.hadoop.util.ToolRunner;
-//import org.junit.Before;
-//import org.junit.Test;
-//
-//import org.apache.kylin.metadata.MetadataManager;
-//
-///**
-// * @author xjiang
-// *
-// */
-//public class KeyDistributionJobTest {
-//
-//    private Configuration conf;
-//
-//    @Before
-//    public void setup() throws IOException {
-//        conf = new Configuration();
-//        conf.set("fs.default.name", "file:///");
-//        conf.set("mapred.job.tracker", "local");
-//    }
-//
-//    @Test
-//    public void testJob() throws Exception {
-//        final String input = "src/test/resources/data/base_cuboid/,src/test/resources/data/6d_cuboid/";
-//        final String output = "target/test-output/key_distribution/";
-//        final String cubeName = "test_kylin_cube_with_slr";
-//        final String metadata = MetadataManager.getMetadataUrlFromEnv();
-//
-//        FileUtil.fullyDelete(new File(output));
-//
-//        String[] args =
-//                { "-input", input, "-cubename", cubeName, "-output", output, "-metadata", metadata,
-//                        "-columnpercentage", "50", "-splitnumber", "10" };
-//        assertEquals("Job failed", 0, ToolRunner.run(new KeyDistributionJob(), args));
-//    }
-//
-// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionMapperTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionMapperTest.java
deleted file mode 100644
index 001953f..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionMapperTest.java
+++ /dev/null
@@ -1,171 +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.cube;
-
-///*
-// * Copyright 2013-2014 eBay Software Foundation
-// *
-// * Licensed under the Apache License, Version 2.0 (the "License");
-// * you may not use this file except in compliance with the License.
-// * You may obtain a copy of the License at
-// *
-// *   http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT 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.index.cube;
-//
-//import static org.junit.Assert.*;
-//
-//import java.io.File;
-//import java.io.IOException;
-//import java.util.List;
-//
-//import org.apache.hadoop.io.LongWritable;
-//import org.apache.hadoop.io.Text;
-//import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-//import org.apache.hadoop.mrunit.types.Pair;
-//import org.junit.Before;
-//import org.junit.Test;
-//
-//import org.apache.kylin.index.BatchConstants;
-//import org.apache.kylin.metadata.MetadataManager;
-//
-///**
-// * @author ysong1
-// *
-// */
-//public class KeyDistributionMapperTest {
-//    @SuppressWarnings("rawtypes")
-//    MapDriver mapDriver;
-//    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
-//
-//    @Before
-//    public void setUp() {
-//        KeyDistributionMapper mapper = new KeyDistributionMapper();
-//        mapDriver = MapDriver.newMapDriver(mapper);
-//    }
-//
-//    @SuppressWarnings("unchecked")
-//    @Test
-//    public void testMapperWithoutHeader() throws IOException {
-//        String matadata = MetadataManager.getMetadataUrlFromEnv();
-//        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, "test_kylin_cube_without_slr");
-//        mapDriver.getConfiguration().set(BatchConstants.CFG_METADATA_URL, matadata);
-//        mapDriver.getConfiguration().set(KeyDistributionJob.KEY_COLUMN_PERCENTAGE, "7");
-//        mapDriver.getConfiguration().set(KeyDistributionJob.KEY_HEADER_LENGTH, "8");
-//
-//        Text inputKey1 =
-//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey2 =
-//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey3 =
-//                new Text(new byte[] { 2, 2, 2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey4 =
-//                new Text(new byte[] { 3, 3, 3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey5 =
-//                new Text(new byte[] { 4, 4, 4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey6 =
-//                new Text(new byte[] { 5, 5, 5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey7 =
-//                new Text(new byte[] { 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//
-//        mapDriver.addInput(inputKey1, new Text("abc"));
-//        mapDriver.addInput(inputKey2, new Text("abc"));
-//        mapDriver.addInput(inputKey3, new Text("abc"));
-//        mapDriver.addInput(inputKey4, new Text("abc"));
-//        mapDriver.addInput(inputKey5, new Text("abc"));
-//        mapDriver.addInput(inputKey6, new Text("abc"));
-//        mapDriver.addInput(inputKey7, new Text("abc"));
-//
-//        List<Pair<Text, LongWritable>> result = mapDriver.run();
-//
-//        assertEquals(7, result.size());
-//
-//        byte[] key1 = result.get(0).getFirst().getBytes();
-//        LongWritable value1 = result.get(0).getSecond();
-//        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11 }, key1);
-//        assertEquals(2, value1.get());
-//
-//        byte[] key7 = result.get(6).getFirst().getBytes();
-//        LongWritable value7 = result.get(6).getSecond();
-//        assertArrayEquals(new byte[] { 0 }, key7);
-//        assertEquals(7, value7.get());
-//    }
-//
-//    @SuppressWarnings("unchecked")
-//    @Test
-//    public void testMapperWithHeader() throws IOException {
-//        String matadata = MetadataManager.getMetadataUrlFromEnv();
-//        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, "test_kylin_cube_with_slr");
-//        mapDriver.getConfiguration().set(BatchConstants.CFG_METADATA_URL, matadata);
-//        mapDriver.getConfiguration().set(KeyDistributionJob.KEY_COLUMN_PERCENTAGE, "7");
-//        mapDriver.getConfiguration().set(KeyDistributionJob.KEY_HEADER_LENGTH, "26");
-//
-//        Text inputKey1 =
-//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0,
-//                        0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey2 =
-//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0,
-//                        0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey3 =
-//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 2, 2,
-//                        2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey4 =
-//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 3, 3,
-//                        3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey5 =
-//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 4, 4,
-//                        4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey6 =
-//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 5, 5,
-//                        5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//        Text inputKey7 =
-//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 6, 6,
-//                        6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
-//
-//        mapDriver.addInput(inputKey1, new Text("abc"));
-//        mapDriver.addInput(inputKey2, new Text("abc"));
-//        mapDriver.addInput(inputKey3, new Text("abc"));
-//        mapDriver.addInput(inputKey4, new Text("abc"));
-//        mapDriver.addInput(inputKey5, new Text("abc"));
-//        mapDriver.addInput(inputKey6, new Text("abc"));
-//        mapDriver.addInput(inputKey7, new Text("abc"));
-//
-//        List<Pair<Text, LongWritable>> result = mapDriver.run();
-//
-//        assertEquals(7, result.size());
-//
-//        byte[] key1 = result.get(0).getFirst().getBytes();
-//        LongWritable value1 = result.get(0).getSecond();
-//        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0,
-//                0, 0, 0, 0, 0, 127, 11 }, key1);
-//        assertEquals(2, value1.get());
-//
-//        byte[] key7 = result.get(6).getFirst().getBytes();
-//        LongWritable value7 = result.get(6).getSecond();
-//        assertArrayEquals(new byte[] { 0 }, key7);
-//        assertEquals(7, value7.get());
-//    }
-// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
deleted file mode 100644
index aae54b4..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-
-/**
- * @author ysong1
- */
-public class MergeCuboidJobTest extends LocalFileMetadataTestCase {
-
-    private Configuration conf;
-
-    @Before
-    public void setup() throws Exception {
-        conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-
-        // for local runner out-of-memory issue
-        conf.set("mapreduce.task.io.sort.mb", "10");
-        createTestMetadata();
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-    }
-
-    @Test
-    public void test() throws Exception {
-        // String input =
-        // "src/test/resources/data/base_cuboid,src/test/resources/data/6d_cuboid";
-        String output = "target/test-output/merged_cuboid";
-        String cubeName = "test_kylin_cube_with_slr_ready";
-        String jobname = "merge_cuboid";
-
-        File baseFolder = File.createTempFile("kylin-f24668f6-dcff-4cb6-a89b-77f1119df8fa-", "base");
-        baseFolder.delete();
-        baseFolder.mkdir();
-        FileUtils.copyDirectory(new File("src/test/resources/data/base_cuboid"), baseFolder);
-        baseFolder.deleteOnExit();
-
-        File sixDFolder = File.createTempFile("kylin-f24668f6-dcff-4cb6-a89b-77f1119df8fa-", "6d");
-        sixDFolder.delete();
-        sixDFolder.mkdir();
-        FileUtils.copyDirectory(new File("src/test/resources/data/base_cuboid"), sixDFolder);
-        sixDFolder.deleteOnExit();
-
-        FileUtil.fullyDelete(new File(output));
-
-        // CubeManager cubeManager =
-        // CubeManager.getInstanceFromEnv(getTestConfig());
-
-        String[] args = { "-input", baseFolder.getAbsolutePath() + "," + sixDFolder.getAbsolutePath(), "-cubename", cubeName, "-segmentname", "20130331080000_20131212080000", "-output", output, "-jobname", jobname };
-        assertEquals("Job failed", 0, ToolRunner.run(conf, new MergeCuboidJob(), args));
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapperTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapperTest.java
deleted file mode 100644
index 4c782e7..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidMapperTest.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.io.IOException;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.CubeUpdate;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.dict.DictionaryGenerator;
-import org.apache.kylin.dict.DictionaryInfo;
-import org.apache.kylin.dict.DictionaryManager;
-import org.apache.kylin.dict.TrieDictionary;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.project.ProjectManager;
-import org.apache.kylin.source.ReadableTable.TableSignature;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author honma
- */
-@SuppressWarnings("rawtypes")
-public class MergeCuboidMapperTest extends LocalFileMetadataTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(MergeCuboidMapperTest.class);
-
-    MapDriver<Text, Text, Text, Text> mapDriver;
-    CubeManager cubeManager;
-    CubeInstance cube;
-    DictionaryManager dictionaryManager;
-
-    TblColRef lfn;
-    TblColRef lsi;
-    TblColRef ssc;
-
-    private DictionaryInfo makeSharedDict() throws IOException {
-        TableSignature signature = new TableSignature();
-        signature.setSize(100);
-        signature.setLastModifiedTime(System.currentTimeMillis());
-        signature.setPath("fake_common_dict");
-
-        DictionaryInfo newDictInfo = new DictionaryInfo("", "", 0, "string", signature);
-
-        List<byte[]> values = new ArrayList<byte[]>();
-        values.add(new byte[] { 101, 101, 101 });
-        values.add(new byte[] { 102, 102, 102 });
-        Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueList(DataType.getInstance(newDictInfo.getDataType()), values);
-        newDictInfo.setCardinality(dict.getSize());
-        dictionaryManager.trySaveNewDict(dict, newDictInfo);
-        ((TrieDictionary) dict).dump(System.out);
-
-        return newDictInfo;
-    }
-
-    @Before
-    public void setUp() throws Exception {
-
-        createTestMetadata();
-
-        logger.info("The metadataUrl is : " + getTestConfig());
-
-        MetadataManager.clearCache();
-        CubeManager.clearCache();
-        ProjectManager.clearCache();
-        DictionaryManager.clearCache();
-
-        // hack for distributed cache
-        // CubeManager.removeInstance(KylinConfig.createInstanceFromUri("../job/meta"));//to
-        // make sure the following mapper could get latest CubeManger
-        FileUtils.deleteDirectory(new File("../job/meta"));
-
-        MergeCuboidMapper mapper = new MergeCuboidMapper();
-        mapDriver = MapDriver.newMapDriver(mapper);
-
-        cubeManager = CubeManager.getInstance(getTestConfig());
-        cube = cubeManager.getCube("test_kylin_cube_without_slr_left_join_ready_2_segments");
-        dictionaryManager = DictionaryManager.getInstance(getTestConfig());
-        lfn = cube.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "LSTG_FORMAT_NAME");
-        lsi = cube.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "CAL_DT");
-        ssc = cube.getDescriptor().findColumnRef("DEFAULT.TEST_CATEGORY_GROUPINGS", "META_CATEG_NAME");
-
-        DictionaryInfo sharedDict = makeSharedDict();
-
-        boolean isFirstSegment = true;
-        for (CubeSegment segment : cube.getSegments()) {
-
-            TableSignature signature = new TableSignature();
-            signature.setSize(100);
-            signature.setLastModifiedTime(System.currentTimeMillis());
-            signature.setPath("fake_dict_for" + lfn.getName() + segment.getName());
-
-            DictionaryInfo newDictInfo = new DictionaryInfo(lfn.getTable(), lfn.getColumnDesc().getName(), lfn.getColumnDesc().getZeroBasedIndex(), "string", signature);
-
-            List<byte[]> values = new ArrayList<byte[]>();
-            values.add(new byte[] { 97, 97, 97 });
-            if (isFirstSegment)
-                values.add(new byte[] { 99, 99, 99 });
-            else
-                values.add(new byte[] { 98, 98, 98 });
-            Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueList(DataType.getInstance(newDictInfo.getDataType()), values);
-            newDictInfo.setCardinality(dict.getSize());
-            dictionaryManager.trySaveNewDict(dict, newDictInfo);
-            ((TrieDictionary) dict).dump(System.out);
-
-            segment.putDictResPath(lfn, newDictInfo.getResourcePath());
-            segment.putDictResPath(lsi, sharedDict.getResourcePath());
-            segment.putDictResPath(ssc, sharedDict.getResourcePath());
-
-            // cubeManager.saveResource(segment.getCubeInstance());
-            // cubeManager.afterCubeUpdated(segment.getCubeInstance());
-
-            isFirstSegment = false;
-        }
-
-        CubeUpdate cubeBuilder = new CubeUpdate(cube);
-        cubeBuilder.setToUpdateSegs(cube.getSegments().toArray(new CubeSegment[cube.getSegments().size()]));
-        cube = cubeManager.updateCube(cubeBuilder);
-
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-        FileUtils.deleteDirectory(new File("../job/meta"));
-    }
-
-    @Test
-    public void test() throws IOException, ParseException {
-
-        //        String cubeName = "test_kylin_cube_without_slr_left_join_ready_2_segments";
-
-        CubeSegment newSeg = cubeManager.mergeSegments(cube, 0L, 1386835200000L, false);
-        //        String segmentName = newSeg.getName();
-
-        final Dictionary<?> dictionary = cubeManager.getDictionary(newSeg, lfn);
-        assertTrue(dictionary == null);
-        //        ((TrieDictionary) dictionary).dump(System.out);
-
-        // hack for distributed cache
-        //        File metaDir = new File("../job/meta");
-        //        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), metaDir);
-        //
-        //        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
-        //        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
-        //        // mapDriver.getConfiguration().set(KylinConfig.KYLIN_METADATA_URL,
-        //        // "../job/meta");
-        //
-        //        byte[] key = new byte[] { 0, 0, 0, 0, 0, 0, 0, -92, 1, 1, 1 };
-        //        byte[] value = new byte[] { 1, 2, 3 };
-        //        byte[] newkey = new byte[] { 0, 0, 0, 0, 0, 0, 0, -92, 1, 1, 2 };
-        //        byte[] newvalue = new byte[] { 1, 2, 3 };
-        //
-        //        mapDriver.withInput(new Text(key), new Text(value));
-        //        mapDriver.withOutput(new Text(newkey), new Text(newvalue));
-        //        mapDriver.setMapInputPath(new Path("/apps/hdmi-prod/b_kylin/prod/kylin-f24668f6-dcff-4cb6-a89b-77f1119df8fa/vac_sw_cube_v4/cuboid/15d_cuboid"));
-        //
-        //        mapDriver.runTest();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/MockupMapContext.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MockupMapContext.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/MockupMapContext.java
deleted file mode 100644
index debcf3e..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MockupMapContext.java
+++ /dev/null
@@ -1,314 +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.cube;
-
-import java.io.IOException;
-import java.net.URI;
-
-import org.apache.commons.lang.NotImplementedException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configuration.IntegerRanges;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.MapContext;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Mapper.Context;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.Partitioner;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
-import org.apache.hadoop.security.Credentials;
-
-import org.apache.kylin.job.constant.BatchConstants;
-
-/**
- * @author yangli9
- * 
- */
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class MockupMapContext {
-
-    public static Context create(final Configuration hconf, String metadataUrl, String cubeName, final Object[] outKV) {
-
-        hconf.set(BatchConstants.CFG_CUBE_NAME, cubeName);
-
-        return new WrappedMapper().getMapContext(new MapContext() {
-
-            @Override
-            public boolean nextKeyValue() throws IOException, InterruptedException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Object getCurrentKey() throws IOException, InterruptedException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Object getCurrentValue() throws IOException, InterruptedException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public void write(Object key, Object value) throws IOException, InterruptedException {
-                System.out.println("Write -- k:" + key + ", v:" + value);
-                if (outKV != null) {
-                    outKV[0] = key;
-                    outKV[1] = value;
-                }
-            }
-
-            @Override
-            public OutputCommitter getOutputCommitter() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public TaskAttemptID getTaskAttemptID() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public void setStatus(String msg) {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public String getStatus() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public float getProgress() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Counter getCounter(Enum<?> counterName) {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Counter getCounter(String groupName, String counterName) {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Configuration getConfiguration() {
-                return hconf;
-            }
-
-            @Override
-            public Credentials getCredentials() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public JobID getJobID() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public int getNumReduceTasks() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Path getWorkingDirectory() throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<?> getOutputKeyClass() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<?> getOutputValueClass() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<?> getMapOutputKeyClass() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<?> getMapOutputValueClass() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public String getJobName() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<? extends InputFormat<?, ?>> getInputFormatClass() throws ClassNotFoundException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<? extends Mapper<?, ?, ?, ?>> getMapperClass() throws ClassNotFoundException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<? extends Reducer<?, ?, ?, ?>> getCombinerClass() throws ClassNotFoundException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<? extends Reducer<?, ?, ?, ?>> getReducerClass() throws ClassNotFoundException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<? extends OutputFormat<?, ?>> getOutputFormatClass() throws ClassNotFoundException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Class<? extends Partitioner<?, ?>> getPartitionerClass() throws ClassNotFoundException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public RawComparator<?> getSortComparator() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public String getJar() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public RawComparator<?> getGroupingComparator() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public boolean getJobSetupCleanupNeeded() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public boolean getTaskCleanupNeeded() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public boolean getProfileEnabled() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public String getProfileParams() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public IntegerRanges getProfileTaskRange(boolean isMap) {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public String getUser() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public boolean getSymlink() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Path[] getArchiveClassPaths() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public URI[] getCacheArchives() throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public URI[] getCacheFiles() throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Path[] getLocalCacheArchives() throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Path[] getLocalCacheFiles() throws IOException {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public Path[] getFileClassPaths() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public String[] getArchiveTimestamps() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public String[] getFileTimestamps() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public int getMaxMapAttempts() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public int getMaxReduceAttempts() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public void progress() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public InputSplit getInputSplit() {
-                throw new NotImplementedException();
-            }
-
-            @Override
-            public RawComparator<?> getCombinerKeyGroupingComparator() {
-                throw new NotImplementedException();
-            }
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
deleted file mode 100644
index c8a3d51..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-public class NDCuboidJobTest extends LocalFileMetadataTestCase {
-
-    private Configuration conf;
-
-    @Before
-    public void setup() throws Exception {
-        conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-
-        // for local runner out-of-memory issue
-        conf.set("mapreduce.task.io.sort.mb", "10");
-
-        createTestMetadata();
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-    }
-
-    @Test
-    public void testJob6D() throws Exception {
-        String input = "src/test/resources/data/base_cuboid/";
-        String output = "target/test-output/6d_cuboid";
-        String cubeName = "test_kylin_cube_with_slr_1_new_segment";
-        String segmentName = "20130331080000_20131212080000";
-        String jobname = "6d_cuboid";
-        String level = "1";
-
-        FileUtil.fullyDelete(new File(output));
-
-        String[] args = { "-input", input, "-cubename", cubeName, "-segmentname", segmentName, "-output", output, "-jobname", jobname, "-level", level };
-        assertEquals("Job failed", 0, ToolRunner.run(conf, new NDCuboidJob(), args));
-    }
-
-    @Test
-    public void testJob5D() throws Exception {
-        final String input = "src/test/resources/data/6d_cuboid/";
-        final String output = "target/test-output/5d_cuboid";
-        final String cubeName = "test_kylin_cube_with_slr_1_new_segment";
-        String segmentName = "20130331080000_20131212080000";
-        String jobname = "5d_cuboid";
-        String level = "2";
-
-        FileUtil.fullyDelete(new File(output));
-
-        String[] args = { "-input", input, "-cubename", cubeName, "-segmentname", segmentName, "-output", output, "-jobname", jobname, "-level", level };
-        assertEquals("Job failed", 0, ToolRunner.run(conf, new NDCuboidJob(), args));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapperTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapperTest.java
deleted file mode 100644
index a6b0372..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidMapperTest.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Collections2;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import javax.annotation.Nullable;
-import java.io.File;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-/**
- * @author George Song (ysong1)
- */
-public class NDCuboidMapperTest extends LocalFileMetadataTestCase {
-    MapReduceDriver<Text, Text, Text, Text, Text, Text> mapReduceDriver;
-    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
-
-    @Before
-    public void setUp() throws Exception {
-        createTestMetadata();
-
-        // hack for distributed cache
-        FileUtils.deleteDirectory(new File("../job/meta"));
-        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), new File("../job/meta"));
-
-        NDCuboidMapper mapper = new NDCuboidMapper();
-        CuboidReducer reducer = new CuboidReducer();
-        mapReduceDriver = MapReduceDriver.newMapReduceDriver(mapper, reducer);
-    }
-
-    @After
-    public void after() throws Exception {
-        cleanupTestMetadata();
-        FileUtils.deleteDirectory(new File("../job/meta"));
-    }
-
-    @Test
-    public void testMapReduceWithSlr() throws IOException {
-
-        String cubeName = "test_kylin_cube_with_slr_1_new_segment";
-        String segmentName = "20130331080000_20131212080000";
-        mapReduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
-        mapReduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
-
-        byte[] key = { 0, 0, 0, 0, 0, 0, 1, -1, 49, 48, 48, 48, 48, 48, 48, 48, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 11, 54, -105, 55, 13, 71, 114, 65, 66, 73, 78, 9, 9, 9, 9, 9, 9, 9, 9, 0, 10, 0 };
-        byte[] value = { 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 1 };
-        Pair<Text, Text> input1 = new Pair<Text, Text>(new Text(key), new Text(value));
-
-        mapReduceDriver.addInput(input1);
-
-        List<Pair<Text, Text>> result = mapReduceDriver.run();
-
-        assertEquals(4, result.size());
-
-        byte[] resultKey = { 0, 0, 0, 0, 0, 0, 1, 127, 49, 48, 48, 48, 48, 48, 48, 48, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 55, 13, 71, 114, 65, 66, 73, 78, 9, 9, 9, 9, 9, 9, 9, 9, 0, 10, 0 };
-        byte[] resultValue = { 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 1 };
-        Pair<Text, Text> output1 = new Pair<Text, Text>(new Text(resultKey), new Text(resultValue));
-
-        //As we will truncate decimal(KYLIN-766), value will no longer equals to resultValue
-        Collection<Text> keys = Collections2.transform(result, new Function<Pair<Text, Text>, Text>() {
-            @Nullable
-            @Override
-            public Text apply(Pair<Text, Text> input) {
-                return input.getFirst();
-            }
-        });
-        assertTrue(keys.contains(output1.getFirst()));
-        assertTrue(!result.contains(output1));
-
-        long[] keySet = new long[result.size()];
-
-        System.out.println(Bytes.toLong(new byte[] { 0, 0, 0, 0, 0, 0, 1, -1 }));
-        for (int i = 0; i < result.size(); i++) {
-            byte[] bytes = new byte[result.get(i).getFirst().getLength()];
-            System.arraycopy(result.get(i).getFirst().getBytes(), 0, bytes, 0, result.get(i).getFirst().getLength());
-            System.out.println(Bytes.toLong(bytes));
-            keySet[i] = Bytes.toLong(bytes);
-        }
-
-        // refer to CuboidSchedulerTest.testGetSpanningCuboid()
-        assertArrayEquals(new long[] { 383, 447, 503, 504 }, keySet);
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/test/java/org/apache/kylin/job/hadoop/cube/RandomKeyDistributionMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RandomKeyDistributionMapperTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/RandomKeyDistributionMapperTest.java
deleted file mode 100644
index 4aae185..0000000
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RandomKeyDistributionMapperTest.java
+++ /dev/null
@@ -1,72 +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.cube;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.invertedindex.RandomKeyDistributionMapper;
-
-/**
- * @author ysong1
- * 
- */
-public class RandomKeyDistributionMapperTest {
-
-    MapDriver<Text, Text, Text, LongWritable> mapDriver;
-
-    @Before
-    @SuppressWarnings({ "rawtypes", "unchecked" })
-    public void setUp() {
-        RandomKeyDistributionMapper mapper = new RandomKeyDistributionMapper();
-        mapDriver = MapDriver.newMapDriver(mapper);
-    }
-
-    @Test
-    public void test() throws IOException {
-        List<Text> data = new ArrayList<Text>();
-        for (int i = 0; i < 1001; i++) {
-            data.add(new Text(String.valueOf(i)));
-        }
-
-        for (Text t : data) {
-            mapDriver.addInput(t, new Text("abc"));
-        }
-
-        mapDriver.getConfiguration().set(BatchConstants.MAPPER_SAMPLE_NUMBER, "100");
-        List<Pair<Text, LongWritable>> result = mapDriver.run();
-        assertEquals(100, result.size());
-
-        for (Pair<Text, LongWritable> p : result) {
-            System.out.println(p.getFirst());
-        }
-    }
-
-}


[08/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusChecker.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusChecker.java b/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusChecker.java
deleted file mode 100644
index 54e9cb3..0000000
--- a/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusChecker.java
+++ /dev/null
@@ -1,99 +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 org.apache.kylin.job.constant.JobStepStatusEnum;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-/**
- * @author xduo
- * 
- */
-public class HadoopStatusChecker {
-
-    protected static final Logger logger = LoggerFactory.getLogger(HadoopStatusChecker.class);
-
-    private final String yarnUrl;
-    private final String mrJobID;
-    private final StringBuilder output;
-
-    public HadoopStatusChecker(String yarnUrl, String mrJobID, StringBuilder output) {
-        this.yarnUrl = yarnUrl;
-        this.mrJobID = mrJobID;
-        this.output = output;
-    }
-
-    public JobStepStatusEnum checkStatus() {
-        if (null == mrJobID) {
-            this.output.append("Skip status check with empty job id..\n");
-            return JobStepStatusEnum.WAITING;
-        }
-        JobStepStatusEnum status = null;
-        try {
-            final Pair<RMAppState, FinalApplicationStatus> result = new HadoopStatusGetter(yarnUrl, mrJobID).get();
-            logger.debug("State of Hadoop job: " + mrJobID + ":" + result.getLeft() + "-" + result.getRight());
-            output.append(new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S").format(new Date()) + " - State of Hadoop job: " + mrJobID + ":" + result.getLeft() + " - " + result.getRight() + "\n");
-
-            switch (result.getRight()) {
-            case SUCCEEDED:
-                status = JobStepStatusEnum.FINISHED;
-                break;
-            case FAILED:
-                status = JobStepStatusEnum.ERROR;
-                break;
-            case KILLED:
-                status = JobStepStatusEnum.KILLED;
-                break;
-            case UNDEFINED:
-                switch (result.getLeft()) {
-                case NEW:
-                case NEW_SAVING:
-                case SUBMITTED:
-                case ACCEPTED:
-                    status = JobStepStatusEnum.WAITING;
-                    break;
-                case RUNNING:
-                    status = JobStepStatusEnum.RUNNING;
-                    break;
-                case FINAL_SAVING:
-                case FINISHING:
-                case FINISHED:
-                case FAILED:
-                case KILLING:
-                case KILLED:
-                }
-                break;
-            }
-        } catch (Exception e) {
-            logger.error("error check status", e);
-            output.append("Exception: " + e.getLocalizedMessage() + "\n");
-            status = JobStepStatusEnum.ERROR;
-        }
-
-        return status;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusGetter.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusGetter.java b/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusGetter.java
deleted file mode 100644
index 8b80e6c..0000000
--- a/job/src/main/java/org/apache/kylin/job/tools/HadoopStatusGetter.java
+++ /dev/null
@@ -1,116 +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 org.apache.commons.httpclient.Header;
-import org.apache.commons.httpclient.HttpClient;
-import org.apache.commons.httpclient.HttpMethod;
-import org.apache.commons.httpclient.methods.GetMethod;
-import org.apache.commons.httpclient.protocol.Protocol;
-import org.apache.commons.httpclient.protocol.ProtocolSocketFactory;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- */
-public class HadoopStatusGetter {
-
-    private final String mrJobId;
-    private final String yarnUrl;
-
-    protected static final Logger log = LoggerFactory.getLogger(HadoopStatusChecker.class);
-
-    public HadoopStatusGetter(String yarnUrl, String mrJobId) {
-        this.yarnUrl = yarnUrl;
-        this.mrJobId = mrJobId;
-    }
-
-    public Pair<RMAppState, FinalApplicationStatus> get() throws IOException {
-        String applicationId = mrJobId.replace("job", "application");
-        String url = yarnUrl.replace("${job_id}", applicationId);
-        JsonNode root = new ObjectMapper().readTree(getHttpResponse(url));
-        RMAppState state = RMAppState.valueOf(root.findValue("state").getTextValue());
-        FinalApplicationStatus finalStatus = FinalApplicationStatus.valueOf(root.findValue("finalStatus").getTextValue());
-        return Pair.of(state, finalStatus);
-    }
-
-    private String getHttpResponse(String url) throws IOException {
-        HttpClient client = new HttpClient();
-
-        String response = null;
-        while (response == null) { // follow redirects via 'refresh'
-            if (url.startsWith("https://")) {
-                registerEasyHttps();
-            }
-            if (url.contains("anonymous=true") == false) {
-                url += url.contains("?") ? "&" : "?";
-                url += "anonymous=true";
-            }
-
-            HttpMethod get = new GetMethod(url);
-            get.addRequestHeader("accept", "application/json");
-
-            try {
-                client.executeMethod(get);
-
-                String redirect = null;
-                Header h = get.getResponseHeader("Refresh");
-                if (h != null) {
-                    String s = h.getValue();
-                    int cut = s.indexOf("url=");
-                    if (cut >= 0) {
-                        redirect = s.substring(cut + 4);
-                    }
-                }
-
-                if (redirect == null) {
-                    response = get.getResponseBodyAsString();
-                    log.debug("Job " + mrJobId + " get status check result.\n");
-                } else {
-                    url = redirect;
-                    log.debug("Job " + mrJobId + " check redirect url " + url + ".\n");
-                }
-            } finally {
-                get.releaseConnection();
-            }
-        }
-
-        return response;
-    }
-
-
-
-    private static Protocol EASY_HTTPS = null;
-
-    private static void registerEasyHttps() {
-        // by pass all https issue
-        if (EASY_HTTPS == null) {
-            EASY_HTTPS = new Protocol("https", (ProtocolSocketFactory) new DefaultSslProtocolSocketFactory(), 443);
-            Protocol.registerProtocol("https", EASY_HTTPS);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java b/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
index 94eac97..f988fc2 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
@@ -18,7 +18,7 @@
 
 package org.apache.kylin.job.tools;
 
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/tools/LZOSupportnessChecker.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/LZOSupportnessChecker.java b/job/src/main/java/org/apache/kylin/job/tools/LZOSupportnessChecker.java
deleted file mode 100644
index dc37d9b..0000000
--- a/job/src/main/java/org/apache/kylin/job/tools/LZOSupportnessChecker.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.tools;
-
-import org.apache.hadoop.hbase.util.CompressionTest;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-
-/**
- */
-public class LZOSupportnessChecker {
-    private static final Logger log = LoggerFactory.getLogger(LZOSupportnessChecker.class);
-
-    public static boolean getSupportness() {
-        try {
-            File temp = File.createTempFile("test", ".tmp");
-            CompressionTest.main(new String[] { "file://" + temp.getAbsolutePath(), "lzo" });
-        } catch (Exception e) {
-            log.warn("LZO support is disabled. Fail to compress file with lzo: " + e.toString());
-            return false;
-        }
-        return true;
-    }
-
-    public static void main(String[] args) throws Exception {
-        System.out.println("LZO supported by current env? " + getSupportness());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/tools/OptionsHelper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/OptionsHelper.java b/job/src/main/java/org/apache/kylin/job/tools/OptionsHelper.java
deleted file mode 100644
index c54f8e1..0000000
--- a/job/src/main/java/org/apache/kylin/job/tools/OptionsHelper.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.tools;
-
-import java.io.File;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-public class OptionsHelper {
-    private CommandLine commandLine;
-
-    public void parseOptions(Options options, String[] args) throws ParseException {
-        CommandLineParser parser = new GnuParser();
-        commandLine = parser.parse(options, args);
-    }
-
-    public Option[] getOptions() {
-        return commandLine.getOptions();
-    }
-
-    public String getOptionsAsString() {
-        StringBuilder buf = new StringBuilder();
-        for (Option option : commandLine.getOptions()) {
-            buf.append(" ");
-            buf.append(option.getOpt());
-            if (option.hasArg()) {
-                buf.append("=");
-                buf.append(option.getValue());
-            }
-        }
-        return buf.toString();
-    }
-
-    public String getOptionValue(Option option) {
-        return commandLine.getOptionValue(option.getOpt());
-    }
-
-    public boolean hasOption(Option option) {
-        return commandLine.hasOption(option.getOpt());
-    }
-
-    public void printUsage(String programName, Options options) {
-        HelpFormatter formatter = new HelpFormatter();
-        formatter.printHelp(programName, options);
-    }
-
-    public static String convertToFileURL(String path) {
-        if (File.separatorChar != '/') {
-            path = path.replace(File.separatorChar, '/');
-        }
-
-        return path;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/job/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
deleted file mode 100644
index c40924e..0000000
--- a/job/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ /dev/null
@@ -1,174 +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.source.hive;
-
-import java.io.IOException;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hive.hcatalog.data.HCatRecord;
-import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.engine.mr.IMRInput;
-import org.apache.kylin.engine.mr.JobBuilderSupport;
-import org.apache.kylin.job.JoinedFlatTable;
-import org.apache.kylin.job.cmd.ShellCmdOutput;
-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.exception.ExecuteException;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-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;
-
-public class HiveMRInput implements IMRInput {
-
-    @Override
-    public IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg) {
-        return new BatchCubingInputSide(seg);
-    }
-    
-    @Override
-    public IMRTableInputFormat getTableInputFormat(TableDesc table) {
-        return new HiveTableInputFormat(table.getIdentity());
-    }
-    
-    public static class HiveTableInputFormat implements IMRTableInputFormat {
-        final String dbName;
-        final String tableName;
-
-        public HiveTableInputFormat(String hiveTable) {
-            String[] parts = HadoopUtil.parseHiveTableName(hiveTable);
-            dbName = parts[0];
-            tableName = parts[1];
-        }
-
-        @Override
-        public void configureJob(Job job) {
-            try {
-                HCatInputFormat.setInput(job, dbName, tableName);
-                job.setInputFormatClass(HCatInputFormat.class);
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
-        }
-
-        @Override
-        public String[] parseMapperInput(Object mapperInput) {
-            return HiveTableReader.getRowAsStringArray((HCatRecord) mapperInput);
-        }
-        
-    }
-
-    public static class BatchCubingInputSide implements IMRBatchCubingInputSide {
-        
-        final JobEngineConfig conf;
-        final CubeSegment seg;
-        final CubeJoinedFlatTableDesc flatHiveTableDesc;
-
-        public BatchCubingInputSide(CubeSegment seg) {
-            this.conf = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
-            this.seg = seg;
-            this.flatHiveTableDesc = new CubeJoinedFlatTableDesc(seg.getCubeDesc(), seg);
-        }
-
-        @Override
-        public void addStepPhase1_CreateFlatTable(DefaultChainedExecutable jobFlow) {
-            jobFlow.addTask(createFlatHiveTableStep(conf, flatHiveTableDesc, jobFlow.getId()));
-        }
-        
-        public static AbstractExecutable createFlatHiveTableStep(JobEngineConfig conf, IJoinedFlatTableDesc flatTableDesc, String jobId) {
-
-            final String dropTableHql = JoinedFlatTable.generateDropTableStatement(flatTableDesc);
-            final String createTableHql = JoinedFlatTable.generateCreateTableStatement(flatTableDesc, JobBuilderSupport.getJobWorkingDir(conf, jobId));
-            String insertDataHqls;
-            try {
-                insertDataHqls = JoinedFlatTable.generateInsertDataStatement(flatTableDesc, conf);
-            } catch (IOException e) {
-                throw new RuntimeException("Failed to generate insert data SQL for intermediate table.", e);
-            }
-
-            ShellExecutable step = new ShellExecutable();
-            StringBuffer buf = new StringBuffer();
-            buf.append("hive -e \"");
-            buf.append(dropTableHql + "\n");
-            buf.append(createTableHql + "\n");
-            buf.append(insertDataHqls + "\n");
-            buf.append("\"");
-
-            step.setCmd(buf.toString());
-            step.setName(ExecutableConstants.STEP_NAME_CREATE_FLAT_HIVE_TABLE);
-
-            return step;
-        }
-
-        @Override
-        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
-            GarbageCollectionStep step = new GarbageCollectionStep();
-            step.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
-            step.setOldHiveTable(flatHiveTableDesc.getTableName());
-            jobFlow.addTask(step);
-        }
-
-        @Override
-        public IMRTableInputFormat getFlatTableInputFormat() {
-            return new HiveTableInputFormat(flatHiveTableDesc.getTableName());
-        }
-        
-    }
-    
-    public static class GarbageCollectionStep extends AbstractExecutable {
-
-        @Override
-        protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-            StringBuffer output = new StringBuffer();
-
-            final String hiveTable = this.getOldHiveTable();
-            if (StringUtils.isNotEmpty(hiveTable)) {
-                final String dropHiveCMD = "hive -e \"DROP TABLE IF EXISTS  " + hiveTable + ";\"";
-                ShellCmdOutput shellCmdOutput = new ShellCmdOutput();
-                try {
-                    context.getConfig().getCliCommandExecutor().execute(dropHiveCMD, shellCmdOutput);
-                    output.append("Hive table " + hiveTable + " is dropped. \n");
-                } catch (IOException e) {
-                    logger.error("job:" + getId() + " execute finished with exception", e);
-                    output.append(shellCmdOutput.getOutput()).append("\n").append(e.getLocalizedMessage());
-                    return new ExecuteResult(ExecuteResult.State.ERROR, output.toString());
-                }
-            }
-
-            return new ExecuteResult(ExecuteResult.State.SUCCEED, output.toString());
-        }
-
-        public void setOldHiveTable(String hiveTable) {
-            setParam("oldHiveTable", hiveTable);
-        }
-
-        private String getOldHiveTable() {
-            return getParam("oldHiveTable");
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java b/job/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
deleted file mode 100644
index 31878a8..0000000
--- a/job/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
+++ /dev/null
@@ -1,155 +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.source.hive;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.engine.mr.HadoopUtil;
-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.TableDesc;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.*;
-
-/**
- * Management class to sync hive table metadata with command See main method for
- * how to use the class
- *
- * @author jianliu
- */
-public class HiveSourceTableLoader {
-
-    @SuppressWarnings("unused")
-    private static final Logger logger = LoggerFactory.getLogger(HiveSourceTableLoader.class);
-
-    public static final String OUTPUT_SURFIX = "json";
-    public static final String TABLE_FOLDER_NAME = "table";
-    public static final String TABLE_EXD_FOLDER_NAME = "table_exd";
-
-    public static Set<String> reloadHiveTables(String[] hiveTables, KylinConfig config) throws IOException {
-
-        Map<String, Set<String>> db2tables = Maps.newHashMap();
-        for (String table : hiveTables) {
-            String[] parts = HadoopUtil.parseHiveTableName(table);
-            Set<String> set = db2tables.get(parts[0]);
-            if (set == null) {
-                set = Sets.newHashSet();
-                db2tables.put(parts[0], set);
-            }
-            set.add(parts[1]);
-        }
-
-        // extract from hive
-        Set<String> loadedTables = Sets.newHashSet();
-        for (String database : db2tables.keySet()) {
-            List<String> loaded = extractHiveTables(database, db2tables.get(database), config);
-            loadedTables.addAll(loaded);
-        }
-
-        return loadedTables;
-    }
-
-    private static List<String> extractHiveTables(String database, Set<String> tables, KylinConfig config) throws IOException {
-
-        List<String> loadedTables = Lists.newArrayList();
-        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
-        for (String tableName : tables) {
-            Table table = null;
-            HiveClient hiveClient = new HiveClient();
-            List<FieldSchema> partitionFields = null;
-            List<FieldSchema> fields = null;
-            try {
-                table = hiveClient.getHiveTable(database, tableName);
-                partitionFields = table.getPartitionKeys();
-                fields = hiveClient.getHiveTableFields(database, tableName);
-            } catch (Exception e) {
-                e.printStackTrace();
-                throw new IOException(e);
-            }
-
-            if (fields != null && partitionFields != null && partitionFields.size() > 0) {
-                fields.addAll(partitionFields);
-            }
-
-            long tableSize = hiveClient.getFileSizeForTable(table);
-            long tableFileNum = hiveClient.getFileNumberForTable(table);
-            TableDesc tableDesc = metaMgr.getTableDesc(database + "." + tableName);
-            if (tableDesc == null) {
-                tableDesc = new TableDesc();
-                tableDesc.setDatabase(database.toUpperCase());
-                tableDesc.setName(tableName.toUpperCase());
-                tableDesc.setUuid(UUID.randomUUID().toString());
-                tableDesc.setLastModified(0);
-            }
-
-            int columnNumber = fields.size();
-            List<ColumnDesc> columns = new ArrayList<ColumnDesc>(columnNumber);
-            for (int i = 0; i < columnNumber; i++) {
-                FieldSchema field = fields.get(i);
-                ColumnDesc cdesc = new ColumnDesc();
-                cdesc.setName(field.getName().toUpperCase());
-                cdesc.setDatatype(field.getType());
-                cdesc.setId(String.valueOf(i + 1));
-                columns.add(cdesc);
-            }
-            tableDesc.setColumns(columns.toArray(new ColumnDesc[columnNumber]));
-
-            StringBuffer partitionColumnString = new StringBuffer();
-            for (int i = 0, n = partitionFields.size(); i < n; i++) {
-                if (i > 0)
-                    partitionColumnString.append(", ");
-                partitionColumnString.append(partitionFields.get(i).getName().toUpperCase());
-            }
-
-            Map<String, String> map = metaMgr.getTableDescExd(tableDesc.getIdentity());
-
-            if (map == null) {
-                map = Maps.newHashMap();
-            }
-            map.put(MetadataConstants.TABLE_EXD_TABLENAME, table.getTableName());
-            map.put(MetadataConstants.TABLE_EXD_LOCATION, table.getSd().getLocation());
-            map.put(MetadataConstants.TABLE_EXD_IF, table.getSd().getInputFormat());
-            map.put(MetadataConstants.TABLE_EXD_OF, table.getSd().getOutputFormat());
-            map.put(MetadataConstants.TABLE_EXD_OWNER, table.getOwner());
-            map.put(MetadataConstants.TABLE_EXD_LAT, String.valueOf(table.getLastAccessTime()));
-            map.put(MetadataConstants.TABLE_EXD_PC, partitionColumnString.toString());
-            map.put(MetadataConstants.TABLE_EXD_TFS, String.valueOf(tableSize));
-            map.put(MetadataConstants.TABLE_EXD_TNF, String.valueOf(tableFileNum));
-            map.put(MetadataConstants.TABLE_EXD_PARTITIONED, Boolean.valueOf(partitionFields != null && partitionFields.size() > 0).toString());
-
-            metaMgr.saveSourceTable(tableDesc);
-            metaMgr.saveTableExd(tableDesc.getIdentity(), map);
-            loadedTables.add(tableDesc.getIdentity());
-        }
-
-
-        return loadedTables;
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/source/hive/HiveTable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/source/hive/HiveTable.java b/job/src/main/java/org/apache/kylin/source/hive/HiveTable.java
deleted file mode 100644
index c09b6fc..0000000
--- a/job/src/main/java/org/apache/kylin/source/hive/HiveTable.java
+++ /dev/null
@@ -1,99 +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.source.hive;
-
-import java.io.IOException;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.engine.mr.DFSFileTable;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.source.ReadableTable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-public class HiveTable implements ReadableTable {
-
-    private static final Logger logger = LoggerFactory.getLogger(HiveTable.class);
-
-    final private String database;
-    final private String hiveTable;
-    
-    private HiveClient hiveClient;
-
-    public HiveTable(TableDesc tableDesc) {
-        this.database = tableDesc.getDatabase();
-        this.hiveTable = tableDesc.getName();
-    }
-
-    @Override
-    public TableReader getReader() throws IOException {
-        return new HiveTableReader(database, hiveTable);
-    }
-
-    @Override
-    public TableSignature getSignature() throws IOException {
-        try {
-            String path = computeHDFSLocation();
-            Pair<Long, Long> sizeAndLastModified = DFSFileTable.getSizeAndLastModified(path);
-            long size = sizeAndLastModified.getFirst();
-            long lastModified = sizeAndLastModified.getSecond();
-
-            // for non-native hive table, cannot rely on size & last modified on HDFS
-            if (getHiveClient().isNativeTable(database, hiveTable) == false) {
-                lastModified = System.currentTimeMillis(); // assume table is ever changing
-            }
-
-            return new TableSignature(path, size, lastModified);
-
-        } catch (Exception e) {
-            if (e instanceof IOException)
-                throw (IOException) e;
-            else
-                throw new IOException(e);
-        }
-    }
-
-    private String computeHDFSLocation() throws Exception {
-
-        String override = KylinConfig.getInstanceFromEnv().getOverrideHiveTableLocation(hiveTable);
-        if (override != null) {
-            logger.debug("Override hive table location " + hiveTable + " -- " + override);
-            return override;
-        }
-
-        return getHiveClient().getHiveTableLocation(database, hiveTable);
-    }
-
-    public HiveClient getHiveClient() {
-
-        if (hiveClient == null) {
-            hiveClient = new HiveClient();
-        }
-        return hiveClient;
-    }
-
-    @Override
-    public String toString() {
-        return "hive: database=[" + database + "], table=[" + hiveTable + "]";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/source/hive/HiveTableReader.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/source/hive/HiveTableReader.java b/job/src/main/java/org/apache/kylin/source/hive/HiveTableReader.java
deleted file mode 100644
index 35e24fe..0000000
--- a/job/src/main/java/org/apache/kylin/source/hive/HiveTableReader.java
+++ /dev/null
@@ -1,176 +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.source.hive;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hive.hcatalog.common.HCatException;
-import org.apache.hive.hcatalog.data.HCatRecord;
-import org.apache.hive.hcatalog.data.transfer.DataTransferFactory;
-import org.apache.hive.hcatalog.data.transfer.HCatReader;
-import org.apache.hive.hcatalog.data.transfer.ReadEntity;
-import org.apache.hive.hcatalog.data.transfer.ReaderContext;
-import org.apache.kylin.source.ReadableTable.TableReader;
-
-/**
- * An implementation of TableReader with HCatalog for Hive table.
- */
-public class HiveTableReader implements TableReader {
-
-    private String dbName;
-    private String tableName;
-    private int currentSplit = -1;
-    private ReaderContext readCntxt = null;
-    private Iterator<HCatRecord> currentHCatRecordItr = null;
-    private HCatRecord currentHCatRecord;
-    private int numberOfSplits = 0;
-    private Map<String, String> partitionKV = null;
-
-    /**
-     * Constructor for reading whole hive table
-     * @param dbName
-     * @param tableName
-     * @throws IOException
-     */
-    public HiveTableReader(String dbName, String tableName) throws IOException {
-        this(dbName, tableName, null);
-    }
-
-    /**
-     * Constructor for reading a partition of the hive table
-     * @param dbName
-     * @param tableName
-     * @param partitionKV key-value pairs condition on the partition
-     * @throws IOException
-     */
-    public HiveTableReader(String dbName, String tableName, Map<String, String> partitionKV) throws IOException {
-        this.dbName = dbName;
-        this.tableName = tableName;
-        this.partitionKV = partitionKV;
-        initialize();
-    }
-
-    private void initialize() throws IOException {
-        try {
-            this.readCntxt = getHiveReaderContext(dbName, tableName, partitionKV);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new IOException(e);
-        }
-
-        this.numberOfSplits = readCntxt.numSplits();
-
-//        HCatTableInfo tableInfo = HCatTableInfo.
-//        HCatSchema schema = HCatBaseInputFormat.getTableSchema(context.getConfiguration);
-    }
-
-    @Override
-    public boolean next() throws IOException {
-
-        while (currentHCatRecordItr == null || !currentHCatRecordItr.hasNext()) {
-            currentSplit++;
-            if (currentSplit == numberOfSplits) {
-                return false;
-            }
-
-            currentHCatRecordItr = loadHCatRecordItr(readCntxt, currentSplit);
-        }
-
-        currentHCatRecord = currentHCatRecordItr.next();
-
-        return true;
-    }
-
-    @Override
-    public String[] getRow() {
-        return getRowAsStringArray(currentHCatRecord);
-    }
-
-    public List<String> getRowAsList() {
-        return getRowAsList(currentHCatRecord);
-    }
-
-    public static List<String> getRowAsList(HCatRecord record, List<String> rowValues) {
-        List<Object> allFields = record.getAll();
-        for (Object o : allFields) {
-            rowValues.add((o == null) ? null : o.toString());
-        }
-        return rowValues;
-    }
-
-    public static List<String> getRowAsList(HCatRecord record) {
-        return Arrays.asList(getRowAsStringArray(record));
-    }
-
-    public static String[] getRowAsStringArray(HCatRecord record) {
-        String[] arr = new String[record.size()];
-        for (int i = 0; i < arr.length; i++) {
-            Object o = record.get(i);
-            arr[i] = (o == null) ? null : o.toString();
-        }
-        return arr;
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.readCntxt = null;
-        this.currentHCatRecordItr = null;
-        this.currentHCatRecord = null;
-        this.currentSplit = -1;
-    }
-
-    public String toString() {
-        return "hive table reader for: " + dbName + "." + tableName;
-    }
-
-    private static ReaderContext getHiveReaderContext(String database, String table, Map<String, String> partitionKV) throws Exception {
-        HiveConf hiveConf = new HiveConf(HiveTableReader.class);
-        Iterator<Entry<String, String>> itr = hiveConf.iterator();
-        Map<String, String> map = new HashMap<String, String>();
-        while (itr.hasNext()) {
-            Entry<String, String> kv = itr.next();
-            map.put(kv.getKey(), kv.getValue());
-        }
-
-        ReadEntity entity;
-        if (partitionKV == null || partitionKV.size() == 0) {
-            entity = new ReadEntity.Builder().withDatabase(database).withTable(table).build();
-        } else {
-            entity = new ReadEntity.Builder().withDatabase(database).withTable(table).withPartition(partitionKV).build();
-        }
-
-        HCatReader reader = DataTransferFactory.getHCatReader(entity, map);
-        ReaderContext cntxt = reader.prepareRead();
-
-        return cntxt;
-    }
-
-    private static Iterator<HCatRecord> loadHCatRecordItr(ReaderContext readCntxt, int dataSplit) throws HCatException {
-        HCatReader currentHCatReader = DataTransferFactory.getHCatReader(readCntxt, dataSplit);
-
-        return currentHCatReader.read();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java b/job/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
deleted file mode 100644
index a9e95a4..0000000
--- a/job/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.source.hive;
-
-import org.apache.kylin.engine.mr.IMRInput;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.source.ITableSource;
-import org.apache.kylin.source.ReadableTable;
-
-public class HiveTableSource implements ITableSource {
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public <I> I adaptToBuildEngine(Class<I> engineInterface) {
-        if (engineInterface == IMRInput.class) {
-            return (I) new HiveMRInput();
-        } else {
-            throw new RuntimeException("Cannot adapt to " + engineInterface);
-        }
-    }
-
-    @Override
-    public ReadableTable createReadableTable(TableDesc tableDesc) {
-        return new HiveTable(tableDesc);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/storage/StorageFactory2.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/storage/StorageFactory2.java b/job/src/main/java/org/apache/kylin/storage/StorageFactory2.java
deleted file mode 100644
index 931a316..0000000
--- a/job/src/main/java/org/apache/kylin/storage/StorageFactory2.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage;
-
-import org.apache.kylin.metadata.model.IBuildable;
-import org.apache.kylin.storage.hbase.HBaseStorage;
-
-/**
- */
-public class StorageFactory2 {
-    
-    private static final IStorage dft = new HBaseStorage();
-    
-    public static <T> T createEngineAdapter(IBuildable buildable, Class<T> engineInterface) {
-        return dft.adaptToBuildEngine(engineInterface);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput.java b/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput.java
deleted file mode 100644
index 5588d81..0000000
--- a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage.hbase;
-
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.IMROutput;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-
-public class HBaseMROutput implements IMROutput {
-
-    @Override
-    public IMRBatchCubingOutputSide getBatchCubingOutputSide(final CubeSegment seg) {
-        return new IMRBatchCubingOutputSide() {
-            HBaseMRSteps steps = new HBaseMRSteps(seg);
-
-            @Override
-            public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow, String cuboidRootPath) {
-                steps.addSaveCuboidToHTableSteps(jobFlow, cuboidRootPath);
-            }
-
-            @Override
-            public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
-                // nothing to do
-            }
-        };
-    }
-
-    @Override
-    public IMRBatchMergeOutputSide getBatchMergeOutputSide(final CubeSegment seg) {
-        return new IMRBatchMergeOutputSide() {
-            HBaseMRSteps steps = new HBaseMRSteps(seg);
-
-            @Override
-            public void addStepPhase2_BuildCube(DefaultChainedExecutable jobFlow, String cuboidRootPath) {
-                steps.addSaveCuboidToHTableSteps(jobFlow, cuboidRootPath);
-            }
-
-            @Override
-            public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow) {
-                jobFlow.addTask(steps.createMergeGCStep());
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java b/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
deleted file mode 100644
index f4be234..0000000
--- a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
+++ /dev/null
@@ -1,298 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage.hbase;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.FileSystem;
-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.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.mapreduce.TableMapper;
-import org.apache.hadoop.hbase.mapreduce.TableSplit;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.SequenceFile.Writer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Mapper.Context;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
-import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
-import org.apache.kylin.engine.mr.ByteArrayWritable;
-import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.engine.mr.IMROutput2;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-import com.google.common.collect.Lists;
-
-public class HBaseMROutput2 implements IMROutput2 {
-
-    @Override
-    public IMRBatchCubingOutputSide2 getBatchCubingOutputSide(final CubeSegment seg) {
-        return new IMRBatchCubingOutputSide2() {
-            HBaseMRSteps steps = new HBaseMRSteps(seg);
-
-            @Override
-            public IMRStorageOutputFormat getStorageOutputFormat() {
-                return new HBaseOutputFormat(seg);
-            }
-
-            @Override
-            public void addStepPhase2_BuildDictionary(DefaultChainedExecutable jobFlow) {
-                jobFlow.addTask(steps.createCreateHTableStepWithStats(jobFlow.getId()));
-            }
-
-            @Override
-            public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow) {
-                jobFlow.addTask(steps.createBulkLoadStep(jobFlow.getId()));
-            }
-
-            @Override
-            public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
-                // nothing to do
-            }
-        };
-    }
-
-    @Override
-    public IMRBatchMergeInputSide2 getBatchMergeInputSide(final CubeSegment seg) {
-        return new IMRBatchMergeInputSide2() {
-            @Override
-            public IMRStorageInputFormat getStorageInputFormat() {
-                return new HBaseInputFormat(seg);
-            }
-        };
-    }
-
-    @Override
-    public IMRBatchMergeOutputSide2 getBatchMergeOutputSide(final CubeSegment seg) {
-        return new IMRBatchMergeOutputSide2() {
-            HBaseMRSteps steps = new HBaseMRSteps(seg);
-
-            @Override
-            public IMRStorageOutputFormat getStorageOutputFormat() {
-                return new HBaseOutputFormat(seg);
-            }
-
-            @Override
-            public void addStepPhase1_MergeDictionary(DefaultChainedExecutable jobFlow) {
-                jobFlow.addTask(steps.createCreateHTableStepWithStats(jobFlow.getId()));
-            }
-
-            @Override
-            public void addStepPhase2_BuildCube(DefaultChainedExecutable jobFlow) {
-                jobFlow.addTask(steps.createBulkLoadStep(jobFlow.getId()));
-            }
-
-            @Override
-            public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow) {
-                jobFlow.addTask(steps.createMergeGCStep());
-            }
-        };
-    }
-
-    @SuppressWarnings({ "rawtypes", "unchecked" })
-    private static class HBaseInputFormat implements IMRStorageInputFormat {
-        final Iterable<String> htables;
-
-        final RowValueDecoder[] rowValueDecoders;
-        final ByteArrayWritable parsedKey;
-        final Object[] parsedValue;
-        final Pair<ByteArrayWritable, Object[]> parsedPair;
-
-        public HBaseInputFormat(CubeSegment seg) {
-            this.htables = new HBaseMRSteps(seg).getMergingHTables();
-
-            List<RowValueDecoder> valueDecoderList = Lists.newArrayList();
-            List<MeasureDesc> measuresDescs = Lists.newArrayList();
-            for (HBaseColumnFamilyDesc cfDesc : seg.getCubeDesc().getHBaseMapping().getColumnFamily()) {
-                for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
-                    valueDecoderList.add(new RowValueDecoder(colDesc));
-                    for (MeasureDesc measure : colDesc.getMeasures()) {
-                        measuresDescs.add(measure);
-                    }
-                }
-            }
-            this.rowValueDecoders = valueDecoderList.toArray(new RowValueDecoder[valueDecoderList.size()]);
-
-            this.parsedKey = new ByteArrayWritable();
-            this.parsedValue = new Object[measuresDescs.size()];
-            this.parsedPair = new Pair<ByteArrayWritable, Object[]>(parsedKey, parsedValue);
-        }
-
-        @Override
-        public void configureInput(Class<? extends Mapper> mapper, Class<? extends WritableComparable> outputKeyClass, Class<? extends Writable> outputValueClass, Job job) throws IOException {
-            List<Scan> scans = new ArrayList<Scan>();
-            for (String htable : htables) {
-                Scan scan = new Scan();
-                scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs
-                scan.setCacheBlocks(false); // don't set to true for MR jobs
-                scan.setAttribute(Scan.SCAN_ATTRIBUTES_TABLE_NAME, Bytes.toBytes(htable));
-                scans.add(scan);
-            }
-            TableMapReduceUtil.initTableMapperJob(scans, (Class<? extends TableMapper>) mapper, outputKeyClass, outputValueClass, job);
-        }
-
-        @Override
-        public CubeSegment findSourceSegment(Context context, CubeInstance cubeInstance) throws IOException {
-            TableSplit currentSplit = (TableSplit) context.getInputSplit();
-            byte[] tableName = currentSplit.getTableName();
-            String htableName = Bytes.toString(tableName);
-
-            // decide which source segment
-            for (CubeSegment segment : cubeInstance.getSegments()) {
-                String segmentHtable = segment.getStorageLocationIdentifier();
-                if (segmentHtable != null && segmentHtable.equalsIgnoreCase(htableName)) {
-                    return segment;
-                }
-            }
-            throw new IllegalStateException("No merging segment's storage location identifier equals " + htableName);
-        }
-
-        @Override
-        public Pair<ByteArrayWritable, Object[]> parseMapperInput(Object inKey, Object inValue) {
-            ImmutableBytesWritable key = (ImmutableBytesWritable) inKey;
-            parsedKey.set(key.get(), key.getOffset(), key.getLength());
-
-            Result value = (Result) inValue;
-            int position = 0;
-            for (int i = 0; i < rowValueDecoders.length; i++) {
-                rowValueDecoders[i].decode(value, false);
-                Object[] measureValues = rowValueDecoders[i].getValues();
-                for (int j = 0; j < measureValues.length; j++) {
-                    parsedValue[position++] = measureValues[j];
-                }
-            }
-
-            return parsedPair;
-        }
-    }
-
-    @SuppressWarnings({ "rawtypes", "unchecked" })
-    private static class HBaseOutputFormat implements IMRStorageOutputFormat {
-        final CubeSegment seg;
-
-        final List<InMemKeyValueCreator> keyValueCreators = Lists.newArrayList();
-        final ImmutableBytesWritable outputKey = new ImmutableBytesWritable();
-
-        public HBaseOutputFormat(CubeSegment seg) {
-            this.seg = seg;
-        }
-
-        @Override
-        public void configureOutput(Class<? extends Reducer> reducer, String jobFlowId, Job job) throws IOException {
-            Path hfilePath = new Path(new HBaseMRSteps(seg).getHFilePath(jobFlowId));
-            FileOutputFormat.setOutputPath(job, hfilePath);
-
-            String htableName = seg.getStorageLocationIdentifier();
-            Configuration conf = HBaseConfiguration.create(job.getConfiguration());
-            HTable htable = new HTable(conf, htableName);
-            HFileOutputFormat.configureIncrementalLoad(job, htable);
-
-            // set Reducer; This need be after configureIncrementalLoad, to overwrite the default reducer class
-            job.setReducerClass(reducer);
-
-            // kylin uses ByteArrayWritable instead of ImmutableBytesWritable as mapper output key
-            rewriteTotalOrderPartitionerFile(job);
-
-            HadoopUtil.deletePath(job.getConfiguration(), hfilePath);
-        }
-
-        @Override
-        public void doReducerOutput(ByteArrayWritable key, Object[] value, Reducer.Context context) throws IOException, InterruptedException {
-            if (keyValueCreators.size() == 0) {
-                int startPosition = 0;
-                for (HBaseColumnFamilyDesc cfDesc : seg.getCubeDesc().getHBaseMapping().getColumnFamily()) {
-                    for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
-                        keyValueCreators.add(new InMemKeyValueCreator(colDesc, startPosition));
-                        startPosition += colDesc.getMeasures().length;
-                    }
-                }
-            }
-
-            outputKey.set(key.array(), key.offset(), key.length());
-
-            KeyValue outputValue;
-            for (int i = 0; i < keyValueCreators.size(); i++) {
-                outputValue = keyValueCreators.get(i).create(key.array(), key.offset(), key.length(), value);
-                context.write(outputKey, outputValue);
-            }
-        }
-
-        private void rewriteTotalOrderPartitionerFile(Job job) throws IOException {
-            Configuration conf = job.getConfiguration();
-            String partitionsFile = TotalOrderPartitioner.getPartitionFile(conf);
-            if (StringUtils.isBlank(partitionsFile))
-                throw new IllegalStateException("HFileOutputFormat.configureIncrementalLoad don't configure TotalOrderPartitioner any more?");
-
-            Path partitionsPath = new Path(partitionsFile);
-
-            // read in partition file in ImmutableBytesWritable
-            List<ByteArrayWritable> keys = Lists.newArrayList();
-            Reader reader = new SequenceFile.Reader(conf, Reader.file(partitionsPath));
-            try {
-                ImmutableBytesWritable key = new ImmutableBytesWritable();
-                while (reader.next(key, NullWritable.get())) {
-                    keys.add(new ByteArrayWritable(key.copyBytes()));
-                }
-            } finally {
-                reader.close();
-            }
-
-            // write out again in ByteArrayWritable
-            Writer writer = SequenceFile.createWriter(conf, Writer.file(partitionsPath), Writer.keyClass(ByteArrayWritable.class), Writer.valueClass(NullWritable.class));
-            try {
-                for (ByteArrayWritable key : keys) {
-                    writer.append(key, NullWritable.get());
-                }
-            } finally {
-                writer.close();
-            }
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMRSteps.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMRSteps.java b/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMRSteps.java
deleted file mode 100644
index d795a9d..0000000
--- a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseMRSteps.java
+++ /dev/null
@@ -1,138 +0,0 @@
-package org.apache.kylin.storage.hbase;
-
-import java.util.List;
-
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.CubingJob;
-import org.apache.kylin.engine.mr.JobBuilderSupport;
-import org.apache.kylin.job.common.HadoopShellExecutable;
-import org.apache.kylin.job.common.MapReduceExecutable;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-import org.apache.kylin.job.hadoop.cube.CubeHFileJob;
-import org.apache.kylin.job.hadoop.cube.RangeKeyDistributionJob;
-import org.apache.kylin.job.hadoop.hbase.BulkLoadJob;
-import org.apache.kylin.job.hadoop.hbase.CreateHTableJob;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-public class HBaseMRSteps extends JobBuilderSupport {
-    
-    public HBaseMRSteps(CubeSegment seg) {
-        super(seg, null);
-    }
-
-    public void addSaveCuboidToHTableSteps(DefaultChainedExecutable jobFlow, String cuboidRootPath) {
-        String jobId = jobFlow.getId();
-        
-        // calculate key distribution
-        jobFlow.addTask(createRangeRowkeyDistributionStep(cuboidRootPath + "*", jobId));
-        // create htable step
-        jobFlow.addTask(createCreateHTableStep(jobId));
-        // generate hfiles step
-        jobFlow.addTask(createConvertCuboidToHfileStep(cuboidRootPath + "*", jobId));
-        // bulk load step
-        jobFlow.addTask(createBulkLoadStep(jobId));
-    }
-
-    public MapReduceExecutable createRangeRowkeyDistributionStep(String inputPath, String jobId) {
-        MapReduceExecutable rowkeyDistributionStep = new MapReduceExecutable();
-        rowkeyDistributionStep.setName(ExecutableConstants.STEP_NAME_GET_CUBOID_KEY_DISTRIBUTION);
-        StringBuilder cmd = new StringBuilder();
-
-        appendMapReduceParameters(cmd, seg);
-        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");
-
-        rowkeyDistributionStep.setMapReduceParams(cmd.toString());
-        rowkeyDistributionStep.setMapReduceJobClass(RangeKeyDistributionJob.class);
-        return rowkeyDistributionStep;
-    }
-
-    public HadoopShellExecutable createCreateHTableStep(String jobId) {
-        return createCreateHTableStep(jobId, false);
-    }
-    
-    public HadoopShellExecutable createCreateHTableStepWithStats(String jobId) {
-        return createCreateHTableStep(jobId, true);
-    }
-    
-    private HadoopShellExecutable createCreateHTableStep(String jobId, boolean withStats) {
-        HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
-        createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "segmentname", seg.getName());
-        appendExecCmdParameters(cmd, "input", getRowkeyDistributionOutputPath(jobId) + "/part-r-00000");
-        appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
-        appendExecCmdParameters(cmd, "statisticsenabled", String.valueOf(withStats));
-
-        createHtableStep.setJobParams(cmd.toString());
-        createHtableStep.setJobClass(CreateHTableJob.class);
-
-        return createHtableStep;
-    }
-
-    public MapReduceExecutable createConvertCuboidToHfileStep(String inputPath, String jobId) {
-        MapReduceExecutable createHFilesStep = new MapReduceExecutable();
-        createHFilesStep.setName(ExecutableConstants.STEP_NAME_CONVERT_CUBOID_TO_HFILE);
-        StringBuilder cmd = new StringBuilder();
-
-        appendMapReduceParameters(cmd, seg);
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().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");
-
-        createHFilesStep.setMapReduceParams(cmd.toString());
-        createHFilesStep.setMapReduceJobClass(CubeHFileJob.class);
-        createHFilesStep.setCounterSaveAs(",," + CubingJob.CUBE_SIZE_BYTES);
-
-        return createHFilesStep;
-    }
-
-    public HadoopShellExecutable createBulkLoadStep(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, "cubename", seg.getCubeInstance().getName());
-
-        bulkLoadStep.setJobParams(cmd.toString());
-        bulkLoadStep.setJobClass(BulkLoadJob.class);
-
-        return bulkLoadStep;
-    }
-    
-    public MergeGCStep createMergeGCStep() {
-        MergeGCStep result = new MergeGCStep();
-        result.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
-        result.setOldHTables(getMergingHTables());
-        return result;
-    }
-
-    public List<String> getMergingHTables() {
-        final List<CubeSegment> mergingSegments = seg.getCubeInstance().getMergingSegments(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) {
-            mergingHTables.add(merging.getStorageLocationIdentifier());
-        }
-        return mergingHTables;
-    }
-    
-    public String getHFilePath(String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/hfile/";
-    }
-
-    public String getRowkeyDistributionOutputPath(String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java b/job/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
deleted file mode 100644
index 3bd0300..0000000
--- a/job/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage.hbase;
-
-import org.apache.kylin.engine.mr.IMROutput;
-import org.apache.kylin.engine.mr.IMROutput2;
-import org.apache.kylin.metadata.realization.IRealization;
-import org.apache.kylin.storage.IStorage;
-import org.apache.kylin.storage.IStorageQuery;
-
-public class HBaseStorage implements IStorage {
-
-    @Override
-    public IStorageQuery createStorageQuery(IRealization realization) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public <I> I adaptToBuildEngine(Class<I> engineInterface) {
-        if (engineInterface == IMROutput.class) {
-            return (I) new HBaseMROutput();
-        } else if (engineInterface == IMROutput2.class) {
-            return (I) new HBaseMROutput2();
-        } else {
-            throw new RuntimeException("Cannot adapt to " + engineInterface);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/storage/hbase/InMemKeyValueCreator.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/storage/hbase/InMemKeyValueCreator.java b/job/src/main/java/org/apache/kylin/storage/hbase/InMemKeyValueCreator.java
deleted file mode 100644
index 5c86802..0000000
--- a/job/src/main/java/org/apache/kylin/storage/hbase/InMemKeyValueCreator.java
+++ /dev/null
@@ -1,73 +0,0 @@
-package org.apache.kylin.storage.hbase;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Text;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-import com.google.common.collect.Lists;
-
-public class InMemKeyValueCreator {
-    byte[] cfBytes;
-    byte[] qBytes;
-    long timestamp;
-
-
-    MeasureCodec codec;
-    Object[] colValues;
-    ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-
-    int startPosition = 0;
-
-    public InMemKeyValueCreator(HBaseColumnDesc colDesc, int startPosition) {
-
-        cfBytes = Bytes.toBytes(colDesc.getColumnFamilyName());
-        qBytes = Bytes.toBytes(colDesc.getQualifier());
-        timestamp = System.currentTimeMillis();
-
-        List<MeasureDesc> measures = Lists.newArrayList();
-        for (MeasureDesc measure : colDesc.getMeasures()) {
-            measures.add(measure);
-        }
-        codec = new MeasureCodec(measures);
-        colValues = new Object[measures.size()];
-
-        this.startPosition = startPosition;
-
-    }
-
-    public KeyValue create(Text key, Object[] measureValues) {
-        return create(key.getBytes(), 0, key.getLength(), measureValues);
-    }
-
-    public KeyValue create(byte[] keyBytes, int keyOffset, int keyLength, Object[] measureValues) {
-        for (int i = 0; i < colValues.length; i++) {
-            colValues[i] = measureValues[startPosition + i];
-        }
-
-        valueBuf.clear();
-        codec.encode(colValues, valueBuf);
-
-        return create(keyBytes, keyOffset, keyLength, valueBuf.array(), 0, valueBuf.position());
-    }
-
-
-    public KeyValue create(byte[] keyBytes, int keyOffset, int keyLength, byte[] value, int voffset, int vlen) {
-        return new KeyValue(keyBytes, keyOffset, keyLength, //
-                cfBytes, 0, cfBytes.length, //
-                qBytes, 0, qBytes.length, //
-                timestamp, KeyValue.Type.Put, //
-                value, voffset, vlen);
-    }
-
-    public KeyValue create(Text key, byte[] value, int voffset, int vlen) {
-        return create(key.getBytes(), 0, key.getLength(), value, voffset, vlen);
-    }
-
-}


[13/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java b/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
deleted file mode 100644
index 4862bb1..0000000
--- a/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
+++ /dev/null
@@ -1,221 +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.dao;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.io.IOUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.JsonSerializer;
-import org.apache.kylin.common.persistence.RawResource;
-import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.job.exception.PersistentException;
-import org.apache.kylin.metadata.MetadataManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- */
-public class ExecutableDao {
-
-    private static final Serializer<ExecutablePO> JOB_SERIALIZER = new JsonSerializer<ExecutablePO>(ExecutablePO.class);
-    private static final Serializer<ExecutableOutputPO> JOB_OUTPUT_SERIALIZER = new JsonSerializer<ExecutableOutputPO>(ExecutableOutputPO.class);
-    private static final Logger logger = LoggerFactory.getLogger(ExecutableDao.class);
-    private static final ConcurrentHashMap<KylinConfig, ExecutableDao> CACHE = new ConcurrentHashMap<KylinConfig, ExecutableDao>();
-    public static final String JOB_PATH_ROOT = "/execute";
-    public static final String JOB_OUTPUT_ROOT = "/execute_output";
-
-    private ResourceStore store;
-
-    public static ExecutableDao getInstance(KylinConfig config) {
-        ExecutableDao r = CACHE.get(config);
-        if (r == null) {
-            r = new ExecutableDao(config);
-            CACHE.put(config, r);
-            if (CACHE.size() > 1) {
-                logger.warn("More than one singleton exist");
-            }
-
-        }
-        return r;
-    }
-
-    private ExecutableDao(KylinConfig config) {
-        logger.info("Using metadata url: " + config);
-        this.store = MetadataManager.getInstance(config).getStore();
-    }
-
-    private String pathOfJob(ExecutablePO job) {
-        return pathOfJob(job.getUuid());
-    }
-    private String pathOfJob(String uuid) {
-        return JOB_PATH_ROOT + "/" + uuid;
-    }
-
-    private String pathOfJobOutput(String uuid) {
-        return JOB_OUTPUT_ROOT + "/" + uuid;
-    }
-
-    private ExecutablePO readJobResource(String path) throws IOException {
-        return store.getResource(path, ExecutablePO.class, JOB_SERIALIZER);
-    }
-
-    private void writeJobResource(String path, ExecutablePO job) throws IOException {
-        store.putResource(path, job, JOB_SERIALIZER);
-    }
-
-    private ExecutableOutputPO readJobOutputResource(String path) throws IOException {
-        return store.getResource(path, ExecutableOutputPO.class, JOB_OUTPUT_SERIALIZER);
-    }
-
-    private long writeJobOutputResource(String path, ExecutableOutputPO output) throws IOException {
-        return store.putResource(path, output, JOB_OUTPUT_SERIALIZER);
-    }
-
-    public List<ExecutableOutputPO> getJobOutputs() throws PersistentException {
-        try {
-            ArrayList<String> resources = store.listResources(JOB_OUTPUT_ROOT);
-            if (resources == null || resources.isEmpty()) {
-                return Collections.emptyList();
-            }
-            Collections.sort(resources);
-            String rangeStart = resources.get(0);
-            String rangeEnd = resources.get(resources.size() - 1);
-            return store.getAllResources(rangeStart, rangeEnd, ExecutableOutputPO.class, JOB_OUTPUT_SERIALIZER);
-        } catch (IOException e) {
-            logger.error("error get all Jobs:", e);
-            throw new PersistentException(e);
-        }
-    }
-
-    public List<ExecutablePO> getJobs() throws PersistentException {
-        try {
-            final List<String> jobIds = store.listResources(JOB_PATH_ROOT);
-            if (jobIds == null || jobIds.isEmpty()) {
-                return Collections.emptyList();
-            }
-            Collections.sort(jobIds);
-            String rangeStart = jobIds.get(0);
-            String rangeEnd = jobIds.get(jobIds.size() - 1);
-            return store.getAllResources(rangeStart, rangeEnd, ExecutablePO.class, JOB_SERIALIZER);
-        } catch (IOException e) {
-            logger.error("error get all Jobs:", e);
-            throw new PersistentException(e);
-        }
-    }
-
-    public List<String> getJobIds() throws PersistentException {
-        try {
-            ArrayList<String> resources = store.listResources(JOB_PATH_ROOT);
-            if (resources == null) {
-                return Collections.emptyList();
-            }
-            ArrayList<String> result = Lists.newArrayListWithExpectedSize(resources.size());
-            for (String path : resources) {
-                result.add(path.substring(path.lastIndexOf("/") + 1));
-            }
-            return result;
-        } catch (IOException e) {
-            logger.error("error get all Jobs:", e);
-            throw new PersistentException(e);
-        }
-    }
-
-    public ExecutablePO getJob(String uuid) throws PersistentException {
-        try {
-            return readJobResource(pathOfJob(uuid));
-        } catch (IOException e) {
-            logger.error("error get job:" + uuid, e);
-            throw new PersistentException(e);
-        }
-    }
-
-    public ExecutablePO addJob(ExecutablePO job) throws PersistentException {
-        try {
-            if (getJob(job.getUuid()) != null) {
-                throw new IllegalArgumentException("job id:" + job.getUuid() + " already exists");
-            }
-            writeJobResource(pathOfJob(job), job);
-            return job;
-        } catch (IOException e) {
-            logger.error("error save job:" + job.getUuid(), e);
-            throw new PersistentException(e);
-        }
-    }
-
-    public void deleteJob(String uuid) throws PersistentException {
-        try {
-            store.deleteResource(pathOfJob(uuid));
-        } catch (IOException e) {
-            logger.error("error delete job:" + uuid, e);
-            throw new PersistentException(e);
-        }
-    }
-
-    public ExecutableOutputPO getJobOutput(String uuid) throws PersistentException {
-        try {
-            ExecutableOutputPO result = readJobOutputResource(pathOfJobOutput(uuid));
-            if (result == null) {
-                result = new ExecutableOutputPO();
-                result.setUuid(uuid);
-                return result;
-            }
-            return result;
-        } catch (IOException e) {
-            logger.error("error get job output id:" + uuid, e);
-            throw new PersistentException(e);
-        }
-    }
-
-    public void addJobOutput(ExecutableOutputPO output) throws PersistentException {
-        try {
-            output.setLastModified(0);
-            writeJobOutputResource(pathOfJobOutput(output.getUuid()), output);
-        } catch (IOException e) {
-            logger.error("error update job output id:" + output.getUuid(), e);
-            throw new PersistentException(e);
-        }
-    }
-
-    public void updateJobOutput(ExecutableOutputPO output) throws PersistentException {
-        try {
-            final long ts = writeJobOutputResource(pathOfJobOutput(output.getUuid()), output);
-            output.setLastModified(ts);
-        } catch (IOException e) {
-            logger.error("error update job output id:" + output.getUuid(), e);
-            throw new PersistentException(e);
-        }
-    }
-
-    public void deleteJobOutput(String uuid) throws PersistentException {
-        try {
-            store.deleteResource(pathOfJobOutput(uuid));
-        } catch (IOException e) {
-            logger.error("error delete job:" + uuid, e);
-            throw new PersistentException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/dao/ExecutableOutputPO.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/dao/ExecutableOutputPO.java b/job/src/main/java/org/apache/kylin/job/dao/ExecutableOutputPO.java
deleted file mode 100644
index 4dacd8a..0000000
--- a/job/src/main/java/org/apache/kylin/job/dao/ExecutableOutputPO.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.dao;
-
-import java.util.Map;
-
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.Maps;
-import org.apache.kylin.common.persistence.RootPersistentEntity;
-
-/**
- */
-@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE)
-public class ExecutableOutputPO extends RootPersistentEntity {
-
-    @JsonProperty("content")
-    private String content;
-
-    @JsonProperty("status")
-    private String status = "READY";
-
-    @JsonProperty("info")
-    private Map<String, String> info = Maps.newHashMap();
-
-    public String getContent() {
-        return content;
-    }
-
-    public void setContent(String content) {
-        this.content = content;
-    }
-
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-    }
-
-    public Map<String, String> getInfo() {
-        return info;
-    }
-
-    public void setInfo(Map<String, String> info) {
-        this.info = info;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java b/job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
deleted file mode 100644
index 6a17b29..0000000
--- a/job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.dao;
-
-import java.util.List;
-import java.util.Map;
-
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.Maps;
-import org.apache.kylin.common.persistence.RootPersistentEntity;
-
-/**
- */
-@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE)
-public class ExecutablePO extends RootPersistentEntity {
-
-    @JsonProperty("name")
-    private String name;
-
-    @JsonProperty("tasks")
-    private List<ExecutablePO> tasks;
-
-    @JsonProperty("type")
-    private String type;
-
-    @JsonProperty("params")
-    private Map<String, String> params = Maps.newHashMap();
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public List<ExecutablePO> getTasks() {
-        return tasks;
-    }
-
-    public void setTasks(List<ExecutablePO> tasks) {
-        this.tasks = tasks;
-    }
-
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    public Map<String, String> getParams() {
-        return params;
-    }
-
-    public void setParams(Map<String, String> params) {
-        this.params = params;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/deployment/HbaseConfigPrinterCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/deployment/HbaseConfigPrinterCLI.java b/job/src/main/java/org/apache/kylin/job/deployment/HbaseConfigPrinterCLI.java
index 1aeb50f..b03cb5f 100644
--- a/job/src/main/java/org/apache/kylin/job/deployment/HbaseConfigPrinterCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/deployment/HbaseConfigPrinterCLI.java
@@ -27,8 +27,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-
-import org.apache.kylin.job.tools.LZOSupportnessChecker;
+import org.apache.kylin.storage.hbase.util.LZOSupportnessChecker;
 
 /**
  * <p/>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java b/job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
deleted file mode 100644
index 2eb9b31..0000000
--- a/job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.engine;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.job.tools.OptionsHelper;
-import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-
-/**
- * @author ysong1
- */
-public class JobEngineConfig {
-    private static final Logger logger = LoggerFactory.getLogger(JobEngineConfig.class);
-    public static String HADOOP_JOB_CONF_FILENAME = "kylin_job_conf";
-    public static String HIVE_CONF_FILENAME = "kylin_hive_conf";
-
-    private static File getJobConfig(String fileName) {
-        String path = System.getProperty(KylinConfig.KYLIN_CONF);
-        if (StringUtils.isNotEmpty(path)) {
-            return new File(path, fileName);
-        }
-
-        path = KylinConfig.getKylinHome();
-        if (StringUtils.isNotEmpty(path)) {
-            return new File(path + File.separator + "conf", fileName);
-        }
-        return null;
-    }
-
-    private String getHadoopJobConfFilePath(RealizationCapacity capaticy, boolean appendSuffix) throws IOException {
-        String hadoopJobConfFile;
-        if (appendSuffix) {
-            hadoopJobConfFile = (HADOOP_JOB_CONF_FILENAME + "_" + capaticy.toString().toLowerCase() + ".xml");
-        } else {
-            hadoopJobConfFile = (HADOOP_JOB_CONF_FILENAME + ".xml");
-        }
-
-        File jobConfig = getJobConfig(hadoopJobConfFile);
-        if (jobConfig == null || !jobConfig.exists()) {
-            logger.warn("fail to locate " + hadoopJobConfFile + ", trying to locate " + HADOOP_JOB_CONF_FILENAME + ".xml");
-            jobConfig = getJobConfig(HADOOP_JOB_CONF_FILENAME + ".xml");
-            if (jobConfig == null || !jobConfig.exists()) {
-                logger.error("fail to locate " + HADOOP_JOB_CONF_FILENAME + ".xml");
-                throw new RuntimeException("fail to locate " + HADOOP_JOB_CONF_FILENAME + ".xml");
-            }
-        }
-        return OptionsHelper.convertToFileURL(jobConfig.getAbsolutePath());
-    }
-
-    public String getHadoopJobConfFilePath(RealizationCapacity capaticy) throws IOException {
-        String path = getHadoopJobConfFilePath(capaticy, true);
-        if (!StringUtils.isEmpty(path)) {
-            logger.info("Chosen job conf is : " + path);
-            return path;
-        } else {
-            path = getHadoopJobConfFilePath(capaticy, false);
-            if (!StringUtils.isEmpty(path)) {
-                logger.info("Chosen job conf is : " + path);
-                return path;
-            }
-        }
-        return "";
-    }
-
-
-    public String getHiveConfFilePath() throws IOException {
-        String hiveConfFile = (HIVE_CONF_FILENAME + ".xml");
-
-        File jobConfig = getJobConfig(hiveConfFile);
-        if (jobConfig == null || !jobConfig.exists()) {
-
-            logger.error("fail to locate " + HIVE_CONF_FILENAME + ".xml");
-            throw new RuntimeException("fail to locate " + HIVE_CONF_FILENAME + ".xml");
-        }
-        return OptionsHelper.convertToFileURL(jobConfig.getAbsolutePath());
-    }
-
-    // there should be no setters
-    private final KylinConfig config;
-
-    public JobEngineConfig(KylinConfig config) {
-        this.config = config;
-    }
-
-    public KylinConfig getConfig() {
-        return config;
-    }
-
-    public String getHdfsWorkingDirectory() {
-        return config.getHdfsWorkingDirectory();
-    }
-    
-    /**
-     * @return the maxConcurrentJobLimit
-     */
-    public int getMaxConcurrentJobLimit() {
-        return config.getMaxConcurrentJobLimit();
-    }
-
-    /**
-     * @return the timeZone
-     */
-    public String getTimeZone() {
-        return config.getTimeZone();
-    }
-
-    /**
-     * @return the adminDls
-     */
-    public String getAdminDls() {
-        return config.getAdminDls();
-    }
-
-    /**
-     * @return the jobStepTimeout
-     */
-    public long getJobStepTimeout() {
-        return config.getJobStepTimeout();
-    }
-
-    /**
-     * @return the asyncJobCheckInterval
-     */
-    public int getAsyncJobCheckInterval() {
-        return config.getYarnStatusCheckIntervalSeconds();
-    }
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see java.lang.Object#hashCode()
-     */
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((config == null) ? 0 : config.hashCode());
-        return result;
-    }
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see java.lang.Object#equals(java.lang.Object)
-     */
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        JobEngineConfig other = (JobEngineConfig) obj;
-        if (config == null) {
-            if (other.config != null)
-                return false;
-        } else if (!config.equals(other.config))
-            return false;
-        return true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/exception/ExecuteException.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/exception/ExecuteException.java b/job/src/main/java/org/apache/kylin/job/exception/ExecuteException.java
deleted file mode 100644
index 8544fff..0000000
--- a/job/src/main/java/org/apache/kylin/job/exception/ExecuteException.java
+++ /dev/null
@@ -1,41 +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.exception;
-
-/**
- */
-public class ExecuteException extends Exception {
-
-    private static final long serialVersionUID = 5677121412192984281L;
-
-    public ExecuteException() {
-    }
-
-    public ExecuteException(String message) {
-        super(message);
-    }
-
-    public ExecuteException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public ExecuteException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/exception/IllegalStateTranferException.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/exception/IllegalStateTranferException.java b/job/src/main/java/org/apache/kylin/job/exception/IllegalStateTranferException.java
deleted file mode 100644
index f19b0ca..0000000
--- a/job/src/main/java/org/apache/kylin/job/exception/IllegalStateTranferException.java
+++ /dev/null
@@ -1,45 +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.exception;
-
-/**
- */
-public class IllegalStateTranferException extends RuntimeException {
-
-    private static final long serialVersionUID = 8466551519300132702L;
-
-    public IllegalStateTranferException() {
-    }
-
-    public IllegalStateTranferException(String message) {
-        super(message);
-    }
-
-    public IllegalStateTranferException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public IllegalStateTranferException(Throwable cause) {
-        super(cause);
-    }
-
-    public IllegalStateTranferException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/exception/JobException.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/exception/JobException.java b/job/src/main/java/org/apache/kylin/job/exception/JobException.java
deleted file mode 100644
index ba4c52a..0000000
--- a/job/src/main/java/org/apache/kylin/job/exception/JobException.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.exception;
-
-/**
- * @author xduo
- * 
- */
-public class JobException extends Exception {
-
-    private static final long serialVersionUID = 1L;
-
-    /**
-     * 
-     */
-    public JobException() {
-        super();
-    }
-
-    /**
-     * @param message
-     * @param cause
-     */
-    public JobException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    /**
-     * @param message
-     */
-    public JobException(String message) {
-        super(message);
-    }
-
-    /**
-     * @param cause
-     */
-    public JobException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/exception/LockException.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/exception/LockException.java b/job/src/main/java/org/apache/kylin/job/exception/LockException.java
deleted file mode 100644
index cf43ac9..0000000
--- a/job/src/main/java/org/apache/kylin/job/exception/LockException.java
+++ /dev/null
@@ -1,44 +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.exception;
-
-/**
- */
-public class LockException extends Exception {
-    private static final long serialVersionUID = 2072745879281754945L;
-
-    public LockException() {
-    }
-
-    public LockException(String message) {
-        super(message);
-    }
-
-    public LockException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public LockException(Throwable cause) {
-        super(cause);
-    }
-
-    public LockException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/exception/PersistentException.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/exception/PersistentException.java b/job/src/main/java/org/apache/kylin/job/exception/PersistentException.java
deleted file mode 100644
index 8507a53..0000000
--- a/job/src/main/java/org/apache/kylin/job/exception/PersistentException.java
+++ /dev/null
@@ -1,44 +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.exception;
-
-/**
- */
-public class PersistentException extends Exception {
-    private static final long serialVersionUID = -4239863858506718998L;
-
-    public PersistentException() {
-    }
-
-    public PersistentException(String message) {
-        super(message);
-    }
-
-    public PersistentException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public PersistentException(Throwable cause) {
-        super(cause);
-    }
-
-    public PersistentException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/exception/SchedulerException.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/exception/SchedulerException.java b/job/src/main/java/org/apache/kylin/job/exception/SchedulerException.java
deleted file mode 100644
index 057bd4a..0000000
--- a/job/src/main/java/org/apache/kylin/job/exception/SchedulerException.java
+++ /dev/null
@@ -1,44 +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.exception;
-
-/**
- */
-public class SchedulerException extends Exception {
-    private static final long serialVersionUID = 349041244824274861L;
-
-    public SchedulerException() {
-    }
-
-    public SchedulerException(String message) {
-        super(message);
-    }
-
-    public SchedulerException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public SchedulerException(Throwable cause) {
-        super(cause);
-    }
-
-    public SchedulerException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java b/job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
deleted file mode 100644
index be82b3a..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
+++ /dev/null
@@ -1,302 +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.execution;
-
-import com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.LogTitlePrinter;
-import org.apache.kylin.common.util.MailService;
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.impl.threadpool.DefaultContext;
-import org.apache.kylin.job.manager.ExecutableManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-/**
- */
-public abstract class AbstractExecutable implements Executable, Idempotent {
-
-    protected static final String SUBMITTER = "submitter";
-    protected static final String NOTIFY_LIST = "notify_list";
-    protected static final String START_TIME = "startTime";
-    protected static final String END_TIME = "endTime";
-
-    protected static final Logger logger = LoggerFactory.getLogger(AbstractExecutable.class);
-
-    private String name;
-    private String id;
-    private Map<String, String> params = Maps.newHashMap();
-
-    protected static ExecutableManager executableManager = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
-
-    public AbstractExecutable() {
-        setId(UUID.randomUUID().toString());
-    }
-
-    protected void onExecuteStart(ExecutableContext executableContext) {
-        Map<String, String> info = Maps.newHashMap();
-        info.put(START_TIME, Long.toString(System.currentTimeMillis()));
-        executableManager.updateJobOutput(getId(), ExecutableState.RUNNING, info, null);
-    }
-
-    protected void onExecuteFinished(ExecuteResult result, ExecutableContext executableContext) {
-        setEndTime(System.currentTimeMillis());
-        if (!isDiscarded()) {
-            if (result.succeed()) {
-                executableManager.updateJobOutput(getId(), ExecutableState.SUCCEED, null, result.output());
-            } else {
-                executableManager.updateJobOutput(getId(), ExecutableState.ERROR, null, result.output());
-            }
-        } else {
-        }
-    }
-
-    protected void onExecuteError(Throwable exception, ExecutableContext executableContext) {
-        if (!isDiscarded()) {
-            executableManager.addJobInfo(getId(), END_TIME, Long.toString(System.currentTimeMillis()));
-            String output = null;
-            if (exception != null) {
-                final StringWriter out = new StringWriter();
-                exception.printStackTrace(new PrintWriter(out));
-                output = out.toString();
-            }
-            executableManager.updateJobOutput(getId(), ExecutableState.ERROR, null, output);
-        } else {
-        }
-    }
-
-    @Override
-    public final ExecuteResult execute(ExecutableContext executableContext) throws ExecuteException {
-
-        //print a eye-catching title in log
-        LogTitlePrinter.printTitle(this.getName());
-
-        Preconditions.checkArgument(executableContext instanceof DefaultContext);
-        ExecuteResult result;
-        try {
-            onExecuteStart(executableContext);
-            result = doWork(executableContext);
-        } catch (Throwable e) {
-            logger.error("error running Executable", e);
-            onExecuteError(e, executableContext);
-            throw new ExecuteException(e);
-        }
-        onExecuteFinished(result, executableContext);
-        return result;
-    }
-
-    protected abstract ExecuteResult doWork(ExecutableContext context) throws ExecuteException;
-
-    @Override
-    public void cleanup() throws ExecuteException {
-
-    }
-
-    @Override
-    public boolean isRunnable() {
-        return this.getStatus() == ExecutableState.READY;
-    }
-
-    @Override
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    @Override
-    public final String getId() {
-        return this.id;
-    }
-
-    public final void setId(String id) {
-        this.id = id;
-    }
-
-    @Override
-    public final ExecutableState getStatus() {
-        return executableManager.getOutput(this.getId()).getState();
-    }
-
-    @Override
-    public final Map<String, String> getParams() {
-        return Collections.unmodifiableMap(this.params);
-    }
-
-    public final String getParam(String key) {
-        return this.params.get(key);
-    }
-
-    public final void setParam(String key, String value) {
-        this.params.put(key, value);
-    }
-
-    public final void setParams(Map<String, String> params) {
-        this.params.putAll(params);
-    }
-
-    public final long getLastModified() {
-        return executableManager.getOutput(getId()).getLastModified();
-    }
-
-    public final void setSubmitter(String submitter) {
-        setParam(SUBMITTER, submitter);
-    }
-
-    public final List<String> getNotifyList() {
-        final String str = getParam(NOTIFY_LIST);
-        if (str != null) {
-            return Lists.newArrayList(StringUtils.split(str, ","));
-        } else {
-            return Collections.emptyList();
-        }
-    }
-
-    public final void setNotifyList(String notifications) {
-        setParam(NOTIFY_LIST, notifications);
-    }
-
-    public final void setNotifyList(List<String> notifications) {
-        setNotifyList(StringUtils.join(notifications, ","));
-    }
-
-    protected Pair<String, String> formatNotifications(ExecutableContext executableContext, ExecutableState state) {
-        return null;
-    }
-
-    protected final void notifyUserStatusChange(ExecutableContext context, ExecutableState state) {
-        try {
-            List<String> users = Lists.newArrayList();
-            users.addAll(getNotifyList());
-            final String adminDls = KylinConfig.getInstanceFromEnv().getAdminDls();
-            if (null != adminDls) {
-                for (String adminDl : adminDls.split(",")) {
-                    users.add(adminDl);
-                }
-            }
-            if (users.isEmpty()) {
-                return;
-            }
-            final Pair<String, String> email = formatNotifications(context, state);
-            if (email == null) {
-                return;
-            }
-            logger.info("prepare to send email to:" + users);
-            logger.info("job name:" + getName());
-            logger.info("submitter:" + getSubmitter());
-            logger.info("notify list:" + users);
-            new MailService().sendMail(users, email.getLeft(), email.getRight());
-        } catch (Exception e) {
-            logger.error(e.getLocalizedMessage(), e);
-        }
-    }
-
-    public final String getSubmitter() {
-        return getParam(SUBMITTER);
-    }
-
-    @Override
-    public final Output getOutput() {
-        return executableManager.getOutput(getId());
-    }
-
-    protected long getExtraInfoAsLong(String key, long defaultValue) {
-        return getExtraInfoAsLong(executableManager.getOutput(getId()), key, defaultValue);
-    }
-
-    public static long getStartTime(Output output) {
-        return getExtraInfoAsLong(output, START_TIME, 0L);
-    }
-
-    public static long getEndTime(Output output) {
-        return getExtraInfoAsLong(output, END_TIME, 0L);
-    }
-
-    public static long getDuration(long startTime, long endTime) {
-        if (startTime == 0) {
-            return 0;
-        }
-        if (endTime == 0) {
-            return System.currentTimeMillis() - startTime;
-        } else {
-            return endTime - startTime;
-        }
-    }
-
-    public static long getExtraInfoAsLong(Output output, String key, long defaultValue) {
-        final String str = output.getExtra().get(key);
-        if (str != null) {
-            return Long.parseLong(str);
-        } else {
-            return defaultValue;
-        }
-    }
-
-    protected final void addExtraInfo(String key, String value) {
-        executableManager.addJobInfo(getId(), key, value);
-    }
-
-    public final void setStartTime(long time) {
-        addExtraInfo(START_TIME, time + "");
-    }
-
-    public final void setEndTime(long time) {
-        addExtraInfo(END_TIME, time + "");
-    }
-
-    public final long getStartTime() {
-        return getExtraInfoAsLong(START_TIME, 0L);
-    }
-
-    public final long getEndTime() {
-        return getExtraInfoAsLong(END_TIME, 0L);
-    }
-
-    public final long getDuration() {
-        return getDuration(getStartTime(), getEndTime());
-    }
-
-    /*
-    * discarded is triggered by JobService, the Scheduler is not awake of that
-    *
-    * */
-    protected final boolean isDiscarded() {
-        final ExecutableState status = executableManager.getOutput(getId()).getState();
-        return status == ExecutableState.DISCARDED;
-    }
-
-    @Override
-    public String toString() {
-        return Objects.toStringHelper(this).add("id", getId()).add("name", getName()).add("state", getStatus()).toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/ChainedExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/ChainedExecutable.java b/job/src/main/java/org/apache/kylin/job/execution/ChainedExecutable.java
deleted file mode 100644
index d5a7aae..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/ChainedExecutable.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.execution;
-
-import java.util.List;
-
-/**
- */
-public interface ChainedExecutable extends Executable {
-
-    List<? extends AbstractExecutable> getTasks();
-    
-    void addTask(AbstractExecutable executable);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java b/job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
deleted file mode 100644
index 6443762..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
+++ /dev/null
@@ -1,122 +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.execution;
-
-import java.util.List;
-import java.util.Map;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.manager.ExecutableManager;
-
-/**
- */
-public class DefaultChainedExecutable extends AbstractExecutable implements ChainedExecutable {
-
-    private final List<AbstractExecutable> subTasks = Lists.newArrayList();
-
-    protected final ExecutableManager jobService = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
-
-    public DefaultChainedExecutable(){
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        List<? extends Executable> executables = getTasks();
-        final int size = executables.size();
-        for (int i = 0; i < size; ++i) {
-            Executable subTask = executables.get(i);
-            if (subTask.isRunnable()) {
-                return subTask.execute(context);
-            }
-        }
-        return new ExecuteResult(ExecuteResult.State.SUCCEED, null);
-    }
-
-    @Override
-    protected void onExecuteStart(ExecutableContext executableContext) {
-        Map<String, String> info = Maps.newHashMap();
-        info.put(START_TIME, Long.toString(System.currentTimeMillis()));
-        final long startTime = getStartTime();
-        if (startTime > 0) {
-            jobService.updateJobOutput(getId(), ExecutableState.RUNNING, null, null);
-        } else {
-            jobService.updateJobOutput(getId(), ExecutableState.RUNNING, info, null);
-        }
-    }
-
-    @Override
-    protected void onExecuteFinished(ExecuteResult result, ExecutableContext executableContext) {
-        if (isDiscarded()) {
-            setEndTime(System.currentTimeMillis());
-            notifyUserStatusChange(executableContext, ExecutableState.DISCARDED);
-        } else if (result.succeed()) {
-            List<? extends Executable> jobs = getTasks();
-            boolean allSucceed = true;
-            boolean hasError = false;
-            for (Executable task: jobs) {
-                final ExecutableState status = task.getStatus();
-                if (status == ExecutableState.ERROR) {
-                    hasError = true;
-                }
-                if (status != ExecutableState.SUCCEED) {
-                    allSucceed = false;
-                }
-            }
-            if (allSucceed) {
-                setEndTime(System.currentTimeMillis());
-                jobService.updateJobOutput(getId(), ExecutableState.SUCCEED, null, null);
-                notifyUserStatusChange(executableContext, ExecutableState.SUCCEED);
-            } else if (hasError) {
-                setEndTime(System.currentTimeMillis());
-                jobService.updateJobOutput(getId(), ExecutableState.ERROR, null, null);
-                notifyUserStatusChange(executableContext, ExecutableState.ERROR);
-            } else {
-                jobService.updateJobOutput(getId(), ExecutableState.READY, null, null);
-            }
-        } else {
-            setEndTime(System.currentTimeMillis());
-            jobService.updateJobOutput(getId(), ExecutableState.ERROR, null, null);
-            notifyUserStatusChange(executableContext, ExecutableState.ERROR);
-        }
-    }
-
-    @Override
-    public List<AbstractExecutable> getTasks() {
-        return subTasks;
-    }
-
-    public final AbstractExecutable getTaskByName(String name) {
-        for (AbstractExecutable task : subTasks) {
-            if (task.getName() != null && task.getName().equalsIgnoreCase(name)) {
-                return task;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public void addTask(AbstractExecutable executable) {
-        executable.setId(getId() + "-" + String.format("%02d", subTasks.size()));
-        this.subTasks.add(executable);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/DefaultOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/DefaultOutput.java b/job/src/main/java/org/apache/kylin/job/execution/DefaultOutput.java
deleted file mode 100644
index 6bc3281..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/DefaultOutput.java
+++ /dev/null
@@ -1,97 +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.execution;
-
-import org.apache.commons.lang3.StringUtils;
-
-import java.util.Map;
-
-/**
- */
-public class DefaultOutput implements Output {
-
-    private ExecutableState state;
-    private Map<String, String> extra;
-    private String verboseMsg;
-    private long lastModified;
-
-    @Override
-    public Map<String, String> getExtra() {
-        return extra;
-    }
-
-    @Override
-    public String getVerboseMsg() {
-        return verboseMsg;
-    }
-
-    @Override
-    public ExecutableState getState() {
-        return state;
-    }
-
-    @Override
-    public long getLastModified() {
-        return lastModified;
-    }
-
-    public void setState(ExecutableState state) {
-        this.state = state;
-    }
-
-    public void setExtra(Map<String, String> extra) {
-        this.extra = extra;
-    }
-
-    public void setVerboseMsg(String verboseMsg) {
-        this.verboseMsg = verboseMsg;
-    }
-
-    public void setLastModified(long lastModified) {
-        this.lastModified = lastModified;
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int hashCode = state.hashCode();
-        hashCode = hashCode * prime + extra.hashCode();
-        hashCode = hashCode * prime + verboseMsg.hashCode();
-        hashCode = hashCode * prime + Long.valueOf(lastModified).hashCode();
-        return hashCode;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (!(obj instanceof DefaultOutput)) {
-            return false;
-        }
-        DefaultOutput another = ((DefaultOutput) obj);
-        if (this.state != another.state) {
-            return false;
-        }
-        if (!extra.equals(another.extra)) {
-            return false;
-        }
-        if (this.lastModified != another.lastModified) {
-            return false;
-        }
-        return StringUtils.equals(verboseMsg, another.verboseMsg);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/Executable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/Executable.java b/job/src/main/java/org/apache/kylin/job/execution/Executable.java
deleted file mode 100644
index a7f1358..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/Executable.java
+++ /dev/null
@@ -1,42 +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.execution;
-
-import org.apache.kylin.job.exception.ExecuteException;
-
-import java.util.Map;
-
-/**
- */
-public interface Executable {
-
-    String getId();
-
-    String getName();
-
-    ExecuteResult execute(ExecutableContext executableContext) throws ExecuteException;
-
-    ExecutableState getStatus();
-
-    Output getOutput();
-
-    boolean isRunnable();
-
-    Map<String, String> getParams();
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/ExecutableContext.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/ExecutableContext.java b/job/src/main/java/org/apache/kylin/job/execution/ExecutableContext.java
deleted file mode 100644
index e3f99ca..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/ExecutableContext.java
+++ /dev/null
@@ -1,30 +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.execution;
-
-import org.apache.kylin.common.KylinConfig;
-
-/**
- */
-public interface ExecutableContext {
-
-    Object getSchedulerContext();
-
-    KylinConfig getConfig();
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/ExecutableState.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/ExecutableState.java b/job/src/main/java/org/apache/kylin/job/execution/ExecutableState.java
deleted file mode 100644
index 5dad4b3..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/ExecutableState.java
+++ /dev/null
@@ -1,82 +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.execution;
-
-import java.util.Collection;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-
-import com.google.common.base.Supplier;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Multimaps;
-
-/**
- */
-public enum ExecutableState {
-
-    READY,
-    RUNNING,
-    ERROR,
-    STOPPED,
-    DISCARDED,
-    SUCCEED;
-
-    private static Multimap<ExecutableState, ExecutableState> VALID_STATE_TRANSFER;
-
-    static {
-        VALID_STATE_TRANSFER = Multimaps.newSetMultimap(Maps.<ExecutableState, Collection<ExecutableState>>newEnumMap(ExecutableState.class), new Supplier<Set<ExecutableState>>() {
-            @Override
-            public Set<ExecutableState> get() {
-                return new CopyOnWriteArraySet<ExecutableState>();
-            }
-        });
-
-        //scheduler
-        VALID_STATE_TRANSFER.put(ExecutableState.READY, ExecutableState.RUNNING);
-        VALID_STATE_TRANSFER.put(ExecutableState.READY, ExecutableState.ERROR);
-        //user
-        VALID_STATE_TRANSFER.put(ExecutableState.READY, ExecutableState.DISCARDED);
-
-        //job
-        VALID_STATE_TRANSFER.put(ExecutableState.RUNNING, ExecutableState.READY);
-        //job
-        VALID_STATE_TRANSFER.put(ExecutableState.RUNNING, ExecutableState.SUCCEED);
-        //user
-        VALID_STATE_TRANSFER.put(ExecutableState.RUNNING, ExecutableState.DISCARDED);
-        //scheduler,job
-        VALID_STATE_TRANSFER.put(ExecutableState.RUNNING, ExecutableState.ERROR);
-
-
-        VALID_STATE_TRANSFER.put(ExecutableState.STOPPED, ExecutableState.DISCARDED);
-        VALID_STATE_TRANSFER.put(ExecutableState.STOPPED, ExecutableState.READY);
-
-        VALID_STATE_TRANSFER.put(ExecutableState.ERROR, ExecutableState.DISCARDED);
-        VALID_STATE_TRANSFER.put(ExecutableState.ERROR, ExecutableState.READY);
-    }
-
-    public boolean isFinalState() {
-        return this == SUCCEED || this == DISCARDED;
-    }
-
-    public static boolean isValidStateTransfer(ExecutableState from, ExecutableState to) {
-        return VALID_STATE_TRANSFER.containsEntry(from, to);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java b/job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
deleted file mode 100644
index cddc0f7..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
+++ /dev/null
@@ -1,54 +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.execution;
-
-import com.google.common.base.Preconditions;
-
-/**
- */
-public final class ExecuteResult {
-
-    public static enum State {SUCCEED, FAILED, ERROR, DISCARDED, STOPPED}
-
-    private final State state;
-    private final String output;
-
-    public ExecuteResult(State state) {
-        this(state, "");
-    }
-
-    public ExecuteResult(State state, String output) {
-        Preconditions.checkArgument(state != null, "state cannot be null");
-        this.state = state;
-        this.output = output;
-    }
-
-    public State state() {
-        return state;
-    }
-
-    public boolean succeed() {
-        return state == State.SUCCEED;
-    }
-
-
-    public String output() {
-        return output;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/Idempotent.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/Idempotent.java b/job/src/main/java/org/apache/kylin/job/execution/Idempotent.java
deleted file mode 100644
index 98c950e..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/Idempotent.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.execution;
-
-import org.apache.kylin.job.exception.ExecuteException;
-
-/**
- */
-public interface Idempotent {
-
-    void cleanup() throws ExecuteException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/execution/Output.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/Output.java b/job/src/main/java/org/apache/kylin/job/execution/Output.java
deleted file mode 100644
index 4d93132..0000000
--- a/job/src/main/java/org/apache/kylin/job/execution/Output.java
+++ /dev/null
@@ -1,34 +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.execution;
-
-import java.util.Map;
-
-/**
- */
-public interface Output {
-
-    Map<String, String> getExtra();
-
-    String getVerboseMsg();
-
-    ExecutableState getState();
-
-    long getLastModified();
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
deleted file mode 100644
index c10d4e0..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
+++ /dev/null
@@ -1,401 +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;
-
-/**
- * @author George Song (ysong1)
- *
- */
-
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.common.util.CliCommandExecutor;
-import org.apache.kylin.common.util.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.job.cmd.ShellCmdOutput;
-import org.apache.kylin.job.exception.JobException;
-import org.apache.kylin.job.tools.OptionsHelper;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.hadoop.util.StringUtils.formatTime;
-
-@SuppressWarnings("static-access")
-public abstract class AbstractHadoopJob extends Configured implements Tool {
-    protected static final Logger logger = LoggerFactory.getLogger(AbstractHadoopJob.class);
-
-    protected static final Option OPTION_JOB_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Job name. For exmaple, Kylin_Cuboid_Builder-clsfd_v2_Step_22-D)").create("jobname");
-    protected static final Option OPTION_JOB_FLOW_ID = OptionBuilder.withArgName("job flow ID").hasArg().isRequired(true).withDescription("job flow ID").create("jobflowid");
-    protected static final Option OPTION_CUBE_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Cube name. For exmaple, flat_item_cube").create("cubename");
-    protected static final Option OPTION_II_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("II name. For exmaple, some_ii").create("iiname");
-    protected static final Option OPTION_SEGMENT_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Cube segment name)").create("segmentname");
-    protected static final Option OPTION_TABLE_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Hive table name.").create("tablename");
-    protected static final Option OPTION_INPUT_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Input path").create("input");
-    protected static final Option OPTION_INPUT_FORMAT = OptionBuilder.withArgName("inputformat").hasArg().isRequired(false).withDescription("Input format").create("inputformat");
-    protected static final Option OPTION_INPUT_DELIM = OptionBuilder.withArgName("inputdelim").hasArg().isRequired(false).withDescription("Input delimeter").create("inputdelim");
-    protected static final Option OPTION_OUTPUT_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Output path").create("output");
-    protected static final Option OPTION_NCUBOID_LEVEL = OptionBuilder.withArgName("level").hasArg().isRequired(true).withDescription("N-Cuboid build level, e.g. 1, 2, 3...").create("level");
-    protected static final Option OPTION_PARTITION_FILE_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Partition file path.").create("input");
-    protected static final Option OPTION_HTABLE_NAME = OptionBuilder.withArgName("htable name").hasArg().isRequired(true).withDescription("HTable name").create("htablename");
-    protected static final Option OPTION_KEY_COLUMN_PERCENTAGE = OptionBuilder.withArgName("rowkey column percentage").hasArg().isRequired(true).withDescription("Percentage of row key columns").create("columnpercentage");
-    protected static final Option OPTION_KEY_SPLIT_NUMBER = OptionBuilder.withArgName("key split number").hasArg().isRequired(true).withDescription("Number of key split range").create("splitnumber");
-
-    protected static final Option OPTION_STATISTICS_ENABLED = OptionBuilder.withArgName("statisticsenabled").hasArg().isRequired(false).withDescription("Statistics enabled").create("statisticsenabled");
-    protected static final Option OPTION_STATISTICS_OUTPUT = OptionBuilder.withArgName("statisticsoutput").hasArg().isRequired(false).withDescription("Statistics output").create("statisticsoutput");
-    protected static final Option OPTION_STATISTICS_SAMPLING_PERCENT = OptionBuilder.withArgName("statisticssamplingpercent").hasArg().isRequired(false).withDescription("Statistics sampling percentage").create("statisticssamplingpercent");
-
-    protected String name;
-    protected String description;
-    protected boolean isAsync = false;
-    protected OptionsHelper optionsHelper = new OptionsHelper();
-
-    protected Job job;
-
-    public AbstractHadoopJob() {
-        super(HadoopUtil.newHadoopJobConfiguration());
-    }
-
-    protected void parseOptions(Options options, String[] args) throws ParseException {
-        optionsHelper.parseOptions(options, args);
-    }
-
-    public void printUsage(Options options) {
-        optionsHelper.printUsage(getClass().getSimpleName(), options);
-    }
-
-    public Option[] getOptions() {
-        return optionsHelper.getOptions();
-    }
-
-    public String getOptionsAsString() {
-        return optionsHelper.getOptionsAsString();
-    }
-
-    protected String getOptionValue(Option option) {
-        return optionsHelper.getOptionValue(option);
-    }
-
-    protected boolean hasOption(Option option) {
-        return optionsHelper.hasOption(option);
-    }
-
-    protected int waitForCompletion(Job job) throws IOException, InterruptedException, ClassNotFoundException {
-        int retVal = 0;
-        long start = System.nanoTime();
-        if (isAsync) {
-            job.submit();
-        } else {
-            job.waitForCompletion(true);
-            retVal = job.isSuccessful() ? 0 : 1;
-            logger.debug("Job '" + job.getJobName() + "' finished " + (job.isSuccessful() ? "successfully in " : "with failures.  Time taken ") + formatTime((System.nanoTime() - start) / 1000000L));
-        }
-        return retVal;
-    }
-
-    protected static void runJob(Tool job, String[] args) {
-        try {
-            int exitCode = ToolRunner.run(job, args);
-            System.exit(exitCode);
-        } catch (Exception e) {
-            e.printStackTrace(System.err);
-            System.exit(5);
-        }
-    }
-
-    private static final String MAP_REDUCE_CLASSPATH = "mapreduce.application.classpath";
-
-    protected void setJobClasspath(Job job) {
-        String jarPath = KylinConfig.getInstanceFromEnv().getKylinJobJarPath();
-        File jarFile = new File(jarPath);
-        if (jarFile.exists()) {
-            job.setJar(jarPath);
-            logger.info("append job jar: " + jarPath);
-        } else {
-            job.setJarByClass(this.getClass());
-        }
-
-        String kylinHiveDependency = System.getProperty("kylin.hive.dependency");
-        String kylinHBaseDependency = System.getProperty("kylin.hbase.dependency");
-        logger.info("append kylin.hive.dependency: " + kylinHiveDependency + " and kylin.hive.dependency: " + kylinHBaseDependency + " to " + MAP_REDUCE_CLASSPATH);
-
-        Configuration jobConf = job.getConfiguration();
-        String classpath = jobConf.get(MAP_REDUCE_CLASSPATH);
-        if (classpath == null || classpath.length() == 0) {
-            logger.info("Didn't find " + MAP_REDUCE_CLASSPATH + " in job configuration, will run 'mapred classpath' to get the default value.");
-            classpath = getDefaultMapRedClasspath();
-            logger.info("The default mapred classpath is: " + classpath);
-        }
-
-
-        if (kylinHBaseDependency != null) {
-            // yarn classpath is comma separated
-            kylinHBaseDependency = kylinHBaseDependency.replace(":", ",");
-            classpath = classpath + "," + kylinHBaseDependency;
-        }
-
-        if (kylinHiveDependency != null) {
-            // yarn classpath is comma separated
-            kylinHiveDependency = kylinHiveDependency.replace(":", ",");
-            classpath = classpath + "," + kylinHiveDependency;
-        }
-
-        jobConf.set(MAP_REDUCE_CLASSPATH, classpath + "," + kylinHiveDependency);
-        logger.info("Hadoop job classpath is: " + job.getConfiguration().get(MAP_REDUCE_CLASSPATH));
-    }
-
-
-    private String getDefaultMapRedClasspath() {
-
-        String classpath = "";
-        try {
-            CliCommandExecutor executor = KylinConfig.getInstanceFromEnv().getCliCommandExecutor();
-            ShellCmdOutput output = new ShellCmdOutput();
-            executor.execute("mapred classpath", output);
-
-            classpath = output.getOutput().trim().replace(':', ',');
-        } catch (IOException e) {
-            logger.error("Failed to run: 'mapred classpath'.", e);
-        }
-
-        return classpath;
-    }
-
-
-    public void addInputDirs(String input, Job job) throws IOException {
-        for (String inp : StringSplitter.split(input, ",")) {
-            inp = inp.trim();
-            if (inp.endsWith("/*")) {
-                inp = inp.substring(0, inp.length() - 2);
-                FileSystem fs = FileSystem.get(job.getConfiguration());
-                Path path = new Path(inp);
-                FileStatus[] fileStatuses = fs.listStatus(path);
-                boolean hasDir = false;
-                for (FileStatus stat : fileStatuses) {
-                    if (stat.isDirectory() && !stat.getPath().getName().startsWith("_")) {
-                        hasDir = true;
-                        addInputDirs(stat.getPath().toString(), job);
-                    }
-                }
-                if (fileStatuses.length > 0 && !hasDir) {
-                    addInputDirs(path.toString(), job);
-                }
-            } else {
-                logger.debug("Add input " + inp);
-                FileInputFormat.addInputPath(job, new Path(inp));
-            }
-        }
-    }
-
-    public static KylinConfig loadKylinPropsAndMetadata() throws IOException {
-        File metaDir = new File("meta");
-        System.setProperty(KylinConfig.KYLIN_CONF, metaDir.getAbsolutePath());
-        logger.info("The absolute path for meta dir is " + metaDir.getAbsolutePath());
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        kylinConfig.setMetadataUrl(metaDir.getCanonicalPath());
-        return kylinConfig;
-    }
-
-    protected void attachKylinPropsAndMetadata(TableDesc table, Configuration conf) throws IOException {
-        ArrayList<String> dumpList = new ArrayList<String>();
-        dumpList.add(table.getResourcePath());
-        attachKylinPropsAndMetadata(dumpList, conf);
-    }
-    
-    protected void attachKylinPropsAndMetadata(CubeInstance cube, Configuration conf) throws IOException {
-        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
-        
-        // write cube / model_desc / cube_desc / dict / table
-        ArrayList<String> dumpList = new ArrayList<String>();
-        dumpList.add(cube.getResourcePath());
-        dumpList.add(cube.getDescriptor().getModel().getResourcePath());
-        dumpList.add(cube.getDescriptor().getResourcePath());
-        
-        for (String tableName : cube.getDescriptor().getModel().getAllTables()) {
-            TableDesc table = metaMgr.getTableDesc(tableName);
-            dumpList.add(table.getResourcePath());
-        }
-        for (CubeSegment segment : cube.getSegments()) {
-            dumpList.addAll(segment.getDictionaryPaths());
-        }
-        
-        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);
-    }
-
-    private void attachKylinPropsAndMetadata(ArrayList<String> dumpList, Configuration conf) throws IOException {
-        File tmp = File.createTempFile("kylin_job_meta", "");
-        tmp.delete(); // we need a directory, so delete the file first
-
-        File metaDir = new File(tmp, "meta");
-        metaDir.mkdirs();
-        metaDir.getParentFile().deleteOnExit();
-
-        // write kylin.properties
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        File kylinPropsFile = new File(metaDir, "kylin.properties");
-        kylinConfig.writeProperties(kylinPropsFile);
-
-        // write resources
-        dumpResources(kylinConfig, metaDir, dumpList);
-
-        // hadoop distributed cache
-        conf.set("tmpfiles", "file:///" + OptionsHelper.convertToFileURL(metaDir.getAbsolutePath()));
-    }
-
-    private void dumpResources(KylinConfig kylinConfig, File metaDir, ArrayList<String> dumpList) throws IOException {
-        ResourceStore from = ResourceStore.getStore(kylinConfig);
-        KylinConfig localConfig = KylinConfig.createInstanceFromUri(metaDir.getAbsolutePath());
-        ResourceStore to = ResourceStore.getStore(localConfig);
-        for (String path : dumpList) {
-            InputStream in = from.getResource(path);
-            if (in == null)
-                throw new IllegalStateException("No resource found at -- " + path);
-            long ts = from.getResourceTimestamp(path);
-            to.putResource(path, in, ts);
-            //The following log is duplicate with in ResourceStore
-            //log.info("Dumped resource " + path + " to " + metaDir.getAbsolutePath());
-        }
-    }
-
-    protected void deletePath(Configuration conf, Path path) throws IOException {
-        HadoopUtil.deletePath(conf, path);
-    }
-
-    protected double getTotalMapInputMB() throws ClassNotFoundException, IOException, InterruptedException, JobException {
-        if (job == null) {
-            throw new JobException("Job is null");
-        }
-
-        long mapInputBytes = 0;
-        InputFormat<?, ?> input = ReflectionUtils.newInstance(job.getInputFormatClass(), job.getConfiguration());
-        for (InputSplit split : input.getSplits(job)) {
-            mapInputBytes += split.getLength();
-        }
-        if (mapInputBytes == 0) {
-            throw new IllegalArgumentException("Map input splits are 0 bytes, something is wrong!");
-        }
-        double totalMapInputMB = (double) mapInputBytes / 1024 / 1024;
-        return totalMapInputMB;
-    }
-
-    protected int getMapInputSplitCount() throws ClassNotFoundException, JobException, IOException, InterruptedException {
-        if (job == null) {
-            throw new JobException("Job is null");
-        }
-        InputFormat<?, ?> input = ReflectionUtils.newInstance(job.getInputFormatClass(), job.getConfiguration());
-        return input.getSplits(job).size();
-    }
-
-    public void kill() throws JobException {
-        if (job != null) {
-            try {
-                job.killJob();
-            } catch (IOException e) {
-                throw new JobException(e);
-            }
-        }
-    }
-
-    public Map<String, String> getInfo() throws JobException {
-        if (job != null) {
-            Map<String, String> status = new HashMap<String, String>();
-            if (null != job.getJobID()) {
-                status.put(JobInstance.MR_JOB_ID, job.getJobID().toString());
-            }
-            if (null != job.getTrackingURL()) {
-                status.put(JobInstance.YARN_APP_URL, job.getTrackingURL().toString());
-            }
-
-            return status;
-        } else {
-            throw new JobException("Job is null");
-        }
-    }
-
-    public Counters getCounters() throws JobException {
-        if (job != null) {
-            try {
-                return job.getCounters();
-            } catch (IOException e) {
-                throw new JobException(e);
-            }
-        } else {
-            throw new JobException("Job is null");
-        }
-    }
-
-    public void setAsync(boolean isAsync) {
-        this.isAsync = isAsync;
-    }
-
-    public Job getJob() {
-        return this.job;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java
index 7f549d0..2e3cdb0 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/ColumnCardinalityMapper.java
@@ -32,10 +32,10 @@ import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
 import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java
index 0670178..de10e65 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.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.TextOutputFormat;
 import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
 import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 
 /**
  * This hadoop job will scan all rows of the hive table and then calculate the cardinality on each column.



[21/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
new file mode 100644
index 0000000..be82b3a
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.execution;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.LogTitlePrinter;
+import org.apache.kylin.common.util.MailService;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.impl.threadpool.DefaultContext;
+import org.apache.kylin.job.manager.ExecutableManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ */
+public abstract class AbstractExecutable implements Executable, Idempotent {
+
+    protected static final String SUBMITTER = "submitter";
+    protected static final String NOTIFY_LIST = "notify_list";
+    protected static final String START_TIME = "startTime";
+    protected static final String END_TIME = "endTime";
+
+    protected static final Logger logger = LoggerFactory.getLogger(AbstractExecutable.class);
+
+    private String name;
+    private String id;
+    private Map<String, String> params = Maps.newHashMap();
+
+    protected static ExecutableManager executableManager = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+    public AbstractExecutable() {
+        setId(UUID.randomUUID().toString());
+    }
+
+    protected void onExecuteStart(ExecutableContext executableContext) {
+        Map<String, String> info = Maps.newHashMap();
+        info.put(START_TIME, Long.toString(System.currentTimeMillis()));
+        executableManager.updateJobOutput(getId(), ExecutableState.RUNNING, info, null);
+    }
+
+    protected void onExecuteFinished(ExecuteResult result, ExecutableContext executableContext) {
+        setEndTime(System.currentTimeMillis());
+        if (!isDiscarded()) {
+            if (result.succeed()) {
+                executableManager.updateJobOutput(getId(), ExecutableState.SUCCEED, null, result.output());
+            } else {
+                executableManager.updateJobOutput(getId(), ExecutableState.ERROR, null, result.output());
+            }
+        } else {
+        }
+    }
+
+    protected void onExecuteError(Throwable exception, ExecutableContext executableContext) {
+        if (!isDiscarded()) {
+            executableManager.addJobInfo(getId(), END_TIME, Long.toString(System.currentTimeMillis()));
+            String output = null;
+            if (exception != null) {
+                final StringWriter out = new StringWriter();
+                exception.printStackTrace(new PrintWriter(out));
+                output = out.toString();
+            }
+            executableManager.updateJobOutput(getId(), ExecutableState.ERROR, null, output);
+        } else {
+        }
+    }
+
+    @Override
+    public final ExecuteResult execute(ExecutableContext executableContext) throws ExecuteException {
+
+        //print a eye-catching title in log
+        LogTitlePrinter.printTitle(this.getName());
+
+        Preconditions.checkArgument(executableContext instanceof DefaultContext);
+        ExecuteResult result;
+        try {
+            onExecuteStart(executableContext);
+            result = doWork(executableContext);
+        } catch (Throwable e) {
+            logger.error("error running Executable", e);
+            onExecuteError(e, executableContext);
+            throw new ExecuteException(e);
+        }
+        onExecuteFinished(result, executableContext);
+        return result;
+    }
+
+    protected abstract ExecuteResult doWork(ExecutableContext context) throws ExecuteException;
+
+    @Override
+    public void cleanup() throws ExecuteException {
+
+    }
+
+    @Override
+    public boolean isRunnable() {
+        return this.getStatus() == ExecutableState.READY;
+    }
+
+    @Override
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @Override
+    public final String getId() {
+        return this.id;
+    }
+
+    public final void setId(String id) {
+        this.id = id;
+    }
+
+    @Override
+    public final ExecutableState getStatus() {
+        return executableManager.getOutput(this.getId()).getState();
+    }
+
+    @Override
+    public final Map<String, String> getParams() {
+        return Collections.unmodifiableMap(this.params);
+    }
+
+    public final String getParam(String key) {
+        return this.params.get(key);
+    }
+
+    public final void setParam(String key, String value) {
+        this.params.put(key, value);
+    }
+
+    public final void setParams(Map<String, String> params) {
+        this.params.putAll(params);
+    }
+
+    public final long getLastModified() {
+        return executableManager.getOutput(getId()).getLastModified();
+    }
+
+    public final void setSubmitter(String submitter) {
+        setParam(SUBMITTER, submitter);
+    }
+
+    public final List<String> getNotifyList() {
+        final String str = getParam(NOTIFY_LIST);
+        if (str != null) {
+            return Lists.newArrayList(StringUtils.split(str, ","));
+        } else {
+            return Collections.emptyList();
+        }
+    }
+
+    public final void setNotifyList(String notifications) {
+        setParam(NOTIFY_LIST, notifications);
+    }
+
+    public final void setNotifyList(List<String> notifications) {
+        setNotifyList(StringUtils.join(notifications, ","));
+    }
+
+    protected Pair<String, String> formatNotifications(ExecutableContext executableContext, ExecutableState state) {
+        return null;
+    }
+
+    protected final void notifyUserStatusChange(ExecutableContext context, ExecutableState state) {
+        try {
+            List<String> users = Lists.newArrayList();
+            users.addAll(getNotifyList());
+            final String adminDls = KylinConfig.getInstanceFromEnv().getAdminDls();
+            if (null != adminDls) {
+                for (String adminDl : adminDls.split(",")) {
+                    users.add(adminDl);
+                }
+            }
+            if (users.isEmpty()) {
+                return;
+            }
+            final Pair<String, String> email = formatNotifications(context, state);
+            if (email == null) {
+                return;
+            }
+            logger.info("prepare to send email to:" + users);
+            logger.info("job name:" + getName());
+            logger.info("submitter:" + getSubmitter());
+            logger.info("notify list:" + users);
+            new MailService().sendMail(users, email.getLeft(), email.getRight());
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+        }
+    }
+
+    public final String getSubmitter() {
+        return getParam(SUBMITTER);
+    }
+
+    @Override
+    public final Output getOutput() {
+        return executableManager.getOutput(getId());
+    }
+
+    protected long getExtraInfoAsLong(String key, long defaultValue) {
+        return getExtraInfoAsLong(executableManager.getOutput(getId()), key, defaultValue);
+    }
+
+    public static long getStartTime(Output output) {
+        return getExtraInfoAsLong(output, START_TIME, 0L);
+    }
+
+    public static long getEndTime(Output output) {
+        return getExtraInfoAsLong(output, END_TIME, 0L);
+    }
+
+    public static long getDuration(long startTime, long endTime) {
+        if (startTime == 0) {
+            return 0;
+        }
+        if (endTime == 0) {
+            return System.currentTimeMillis() - startTime;
+        } else {
+            return endTime - startTime;
+        }
+    }
+
+    public static long getExtraInfoAsLong(Output output, String key, long defaultValue) {
+        final String str = output.getExtra().get(key);
+        if (str != null) {
+            return Long.parseLong(str);
+        } else {
+            return defaultValue;
+        }
+    }
+
+    protected final void addExtraInfo(String key, String value) {
+        executableManager.addJobInfo(getId(), key, value);
+    }
+
+    public final void setStartTime(long time) {
+        addExtraInfo(START_TIME, time + "");
+    }
+
+    public final void setEndTime(long time) {
+        addExtraInfo(END_TIME, time + "");
+    }
+
+    public final long getStartTime() {
+        return getExtraInfoAsLong(START_TIME, 0L);
+    }
+
+    public final long getEndTime() {
+        return getExtraInfoAsLong(END_TIME, 0L);
+    }
+
+    public final long getDuration() {
+        return getDuration(getStartTime(), getEndTime());
+    }
+
+    /*
+    * discarded is triggered by JobService, the Scheduler is not awake of that
+    *
+    * */
+    protected final boolean isDiscarded() {
+        final ExecutableState status = executableManager.getOutput(getId()).getState();
+        return status == ExecutableState.DISCARDED;
+    }
+
+    @Override
+    public String toString() {
+        return Objects.toStringHelper(this).add("id", getId()).add("name", getName()).add("state", getStatus()).toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/ChainedExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/ChainedExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/ChainedExecutable.java
new file mode 100644
index 0000000..d5a7aae
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/ChainedExecutable.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.execution;
+
+import java.util.List;
+
+/**
+ */
+public interface ChainedExecutable extends Executable {
+
+    List<? extends AbstractExecutable> getTasks();
+    
+    void addTask(AbstractExecutable executable);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
new file mode 100644
index 0000000..6443762
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.execution;
+
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.manager.ExecutableManager;
+
+/**
+ */
+public class DefaultChainedExecutable extends AbstractExecutable implements ChainedExecutable {
+
+    private final List<AbstractExecutable> subTasks = Lists.newArrayList();
+
+    protected final ExecutableManager jobService = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+    public DefaultChainedExecutable(){
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        List<? extends Executable> executables = getTasks();
+        final int size = executables.size();
+        for (int i = 0; i < size; ++i) {
+            Executable subTask = executables.get(i);
+            if (subTask.isRunnable()) {
+                return subTask.execute(context);
+            }
+        }
+        return new ExecuteResult(ExecuteResult.State.SUCCEED, null);
+    }
+
+    @Override
+    protected void onExecuteStart(ExecutableContext executableContext) {
+        Map<String, String> info = Maps.newHashMap();
+        info.put(START_TIME, Long.toString(System.currentTimeMillis()));
+        final long startTime = getStartTime();
+        if (startTime > 0) {
+            jobService.updateJobOutput(getId(), ExecutableState.RUNNING, null, null);
+        } else {
+            jobService.updateJobOutput(getId(), ExecutableState.RUNNING, info, null);
+        }
+    }
+
+    @Override
+    protected void onExecuteFinished(ExecuteResult result, ExecutableContext executableContext) {
+        if (isDiscarded()) {
+            setEndTime(System.currentTimeMillis());
+            notifyUserStatusChange(executableContext, ExecutableState.DISCARDED);
+        } else if (result.succeed()) {
+            List<? extends Executable> jobs = getTasks();
+            boolean allSucceed = true;
+            boolean hasError = false;
+            for (Executable task: jobs) {
+                final ExecutableState status = task.getStatus();
+                if (status == ExecutableState.ERROR) {
+                    hasError = true;
+                }
+                if (status != ExecutableState.SUCCEED) {
+                    allSucceed = false;
+                }
+            }
+            if (allSucceed) {
+                setEndTime(System.currentTimeMillis());
+                jobService.updateJobOutput(getId(), ExecutableState.SUCCEED, null, null);
+                notifyUserStatusChange(executableContext, ExecutableState.SUCCEED);
+            } else if (hasError) {
+                setEndTime(System.currentTimeMillis());
+                jobService.updateJobOutput(getId(), ExecutableState.ERROR, null, null);
+                notifyUserStatusChange(executableContext, ExecutableState.ERROR);
+            } else {
+                jobService.updateJobOutput(getId(), ExecutableState.READY, null, null);
+            }
+        } else {
+            setEndTime(System.currentTimeMillis());
+            jobService.updateJobOutput(getId(), ExecutableState.ERROR, null, null);
+            notifyUserStatusChange(executableContext, ExecutableState.ERROR);
+        }
+    }
+
+    @Override
+    public List<AbstractExecutable> getTasks() {
+        return subTasks;
+    }
+
+    public final AbstractExecutable getTaskByName(String name) {
+        for (AbstractExecutable task : subTasks) {
+            if (task.getName() != null && task.getName().equalsIgnoreCase(name)) {
+                return task;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public void addTask(AbstractExecutable executable) {
+        executable.setId(getId() + "-" + String.format("%02d", subTasks.size()));
+        this.subTasks.add(executable);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/DefaultOutput.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultOutput.java b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultOutput.java
new file mode 100644
index 0000000..6bc3281
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultOutput.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.execution;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.Map;
+
+/**
+ */
+public class DefaultOutput implements Output {
+
+    private ExecutableState state;
+    private Map<String, String> extra;
+    private String verboseMsg;
+    private long lastModified;
+
+    @Override
+    public Map<String, String> getExtra() {
+        return extra;
+    }
+
+    @Override
+    public String getVerboseMsg() {
+        return verboseMsg;
+    }
+
+    @Override
+    public ExecutableState getState() {
+        return state;
+    }
+
+    @Override
+    public long getLastModified() {
+        return lastModified;
+    }
+
+    public void setState(ExecutableState state) {
+        this.state = state;
+    }
+
+    public void setExtra(Map<String, String> extra) {
+        this.extra = extra;
+    }
+
+    public void setVerboseMsg(String verboseMsg) {
+        this.verboseMsg = verboseMsg;
+    }
+
+    public void setLastModified(long lastModified) {
+        this.lastModified = lastModified;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int hashCode = state.hashCode();
+        hashCode = hashCode * prime + extra.hashCode();
+        hashCode = hashCode * prime + verboseMsg.hashCode();
+        hashCode = hashCode * prime + Long.valueOf(lastModified).hashCode();
+        return hashCode;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof DefaultOutput)) {
+            return false;
+        }
+        DefaultOutput another = ((DefaultOutput) obj);
+        if (this.state != another.state) {
+            return false;
+        }
+        if (!extra.equals(another.extra)) {
+            return false;
+        }
+        if (this.lastModified != another.lastModified) {
+            return false;
+        }
+        return StringUtils.equals(verboseMsg, another.verboseMsg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/Executable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/Executable.java b/core-job/src/main/java/org/apache/kylin/job/execution/Executable.java
new file mode 100644
index 0000000..a7f1358
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/Executable.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.execution;
+
+import org.apache.kylin.job.exception.ExecuteException;
+
+import java.util.Map;
+
+/**
+ */
+public interface Executable {
+
+    String getId();
+
+    String getName();
+
+    ExecuteResult execute(ExecutableContext executableContext) throws ExecuteException;
+
+    ExecutableState getStatus();
+
+    Output getOutput();
+
+    boolean isRunnable();
+
+    Map<String, String> getParams();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableContext.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableContext.java b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableContext.java
new file mode 100644
index 0000000..e3f99ca
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableContext.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job.execution;
+
+import org.apache.kylin.common.KylinConfig;
+
+/**
+ */
+public interface ExecutableContext {
+
+    Object getSchedulerContext();
+
+    KylinConfig getConfig();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableState.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableState.java b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableState.java
new file mode 100644
index 0000000..5dad4b3
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableState.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.execution;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+
+import com.google.common.base.Supplier;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+
+/**
+ */
+public enum ExecutableState {
+
+    READY,
+    RUNNING,
+    ERROR,
+    STOPPED,
+    DISCARDED,
+    SUCCEED;
+
+    private static Multimap<ExecutableState, ExecutableState> VALID_STATE_TRANSFER;
+
+    static {
+        VALID_STATE_TRANSFER = Multimaps.newSetMultimap(Maps.<ExecutableState, Collection<ExecutableState>>newEnumMap(ExecutableState.class), new Supplier<Set<ExecutableState>>() {
+            @Override
+            public Set<ExecutableState> get() {
+                return new CopyOnWriteArraySet<ExecutableState>();
+            }
+        });
+
+        //scheduler
+        VALID_STATE_TRANSFER.put(ExecutableState.READY, ExecutableState.RUNNING);
+        VALID_STATE_TRANSFER.put(ExecutableState.READY, ExecutableState.ERROR);
+        //user
+        VALID_STATE_TRANSFER.put(ExecutableState.READY, ExecutableState.DISCARDED);
+
+        //job
+        VALID_STATE_TRANSFER.put(ExecutableState.RUNNING, ExecutableState.READY);
+        //job
+        VALID_STATE_TRANSFER.put(ExecutableState.RUNNING, ExecutableState.SUCCEED);
+        //user
+        VALID_STATE_TRANSFER.put(ExecutableState.RUNNING, ExecutableState.DISCARDED);
+        //scheduler,job
+        VALID_STATE_TRANSFER.put(ExecutableState.RUNNING, ExecutableState.ERROR);
+
+
+        VALID_STATE_TRANSFER.put(ExecutableState.STOPPED, ExecutableState.DISCARDED);
+        VALID_STATE_TRANSFER.put(ExecutableState.STOPPED, ExecutableState.READY);
+
+        VALID_STATE_TRANSFER.put(ExecutableState.ERROR, ExecutableState.DISCARDED);
+        VALID_STATE_TRANSFER.put(ExecutableState.ERROR, ExecutableState.READY);
+    }
+
+    public boolean isFinalState() {
+        return this == SUCCEED || this == DISCARDED;
+    }
+
+    public static boolean isValidStateTransfer(ExecutableState from, ExecutableState to) {
+        return VALID_STATE_TRANSFER.containsEntry(from, to);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java b/core-job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
new file mode 100644
index 0000000..cddc0f7
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.execution;
+
+import com.google.common.base.Preconditions;
+
+/**
+ */
+public final class ExecuteResult {
+
+    public static enum State {SUCCEED, FAILED, ERROR, DISCARDED, STOPPED}
+
+    private final State state;
+    private final String output;
+
+    public ExecuteResult(State state) {
+        this(state, "");
+    }
+
+    public ExecuteResult(State state, String output) {
+        Preconditions.checkArgument(state != null, "state cannot be null");
+        this.state = state;
+        this.output = output;
+    }
+
+    public State state() {
+        return state;
+    }
+
+    public boolean succeed() {
+        return state == State.SUCCEED;
+    }
+
+
+    public String output() {
+        return output;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/Idempotent.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/Idempotent.java b/core-job/src/main/java/org/apache/kylin/job/execution/Idempotent.java
new file mode 100644
index 0000000..98c950e
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/Idempotent.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.execution;
+
+import org.apache.kylin.job.exception.ExecuteException;
+
+/**
+ */
+public interface Idempotent {
+
+    void cleanup() throws ExecuteException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/execution/Output.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/Output.java b/core-job/src/main/java/org/apache/kylin/job/execution/Output.java
new file mode 100644
index 0000000..4d93132
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/Output.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.execution;
+
+import java.util.Map;
+
+/**
+ */
+public interface Output {
+
+    Map<String, String> getExtra();
+
+    String getVerboseMsg();
+
+    ExecutableState getState();
+
+    long getLastModified();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultContext.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultContext.java b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultContext.java
new file mode 100644
index 0000000..e959ae2
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultContext.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.impl.threadpool;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.job.execution.Executable;
+import org.apache.kylin.job.execution.ExecutableContext;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ */
+public class DefaultContext implements ExecutableContext {
+
+    private final ConcurrentMap<String, Executable> runningJobs;
+    private final KylinConfig kylinConfig;
+
+    public DefaultContext(ConcurrentMap<String, Executable> runningJobs, KylinConfig kylinConfig) {
+        this.runningJobs = runningJobs;
+        this.kylinConfig = kylinConfig;
+    }
+    @Override
+    public Object getSchedulerContext() {
+        return null;
+    }
+
+    @Override
+    public KylinConfig getConfig() {
+        return kylinConfig;
+    }
+
+    void addRunningJob(Executable executable) {
+        runningJobs.put(executable.getId(), executable);
+    }
+
+    void removeRunningJob(Executable executable) {
+        runningJobs.remove(executable.getId());
+    }
+
+    public Map<String, Executable> getRunningJobs() {
+        return Collections.unmodifiableMap(runningJobs);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
new file mode 100644
index 0000000..8a83870
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.impl.threadpool;
+
+import com.google.common.collect.Maps;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.kylin.job.Scheduler;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.exception.SchedulerException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.Executable;
+import org.apache.kylin.job.execution.ExecutableState;
+import org.apache.kylin.job.execution.Output;
+import org.apache.kylin.job.lock.JobLock;
+import org.apache.kylin.job.manager.ExecutableManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.*;
+
+/**
+ */
+public class DefaultScheduler implements Scheduler<AbstractExecutable>, ConnectionStateListener {
+
+
+    private ExecutableManager executableManager;
+    private FetcherRunner fetcher;
+    private ScheduledExecutorService fetcherPool;
+    private ExecutorService jobPool;
+    private DefaultContext context;
+
+    private Logger logger = LoggerFactory.getLogger(DefaultScheduler.class);
+    private volatile boolean initialized = false;
+    private volatile boolean hasStarted = false;
+    private JobEngineConfig jobEngineConfig;
+
+    private static final DefaultScheduler INSTANCE = new DefaultScheduler();
+
+    private DefaultScheduler() {
+    }
+
+    private class FetcherRunner implements Runnable {
+
+        @Override
+        synchronized public void run() {
+            // logger.debug("Job Fetcher is running...");
+            Map<String, Executable> runningJobs = context.getRunningJobs();
+            if (runningJobs.size() >= jobEngineConfig.getMaxConcurrentJobLimit()) {
+                logger.warn("There are too many jobs running, Job Fetch will wait until next schedule time");
+                return;
+            }
+
+            int nRunning = 0, nReady = 0, nOthers = 0;
+            for (final String id : executableManager.getAllJobIds()) {
+                if (runningJobs.containsKey(id)) {
+                    // logger.debug("Job id:" + id + " is already running");
+                    nRunning++;
+                    continue;
+                }
+                final Output output = executableManager.getOutput(id);
+                if ((output.getState() != ExecutableState.READY)) {
+                    // logger.debug("Job id:" + id + " not runnable");
+                    nOthers++;
+                    continue;
+                }
+                nReady++;
+                AbstractExecutable executable = executableManager.getJob(id);
+                String jobDesc = executable.toString();
+                logger.info(jobDesc + " prepare to schedule");
+                try {
+                    context.addRunningJob(executable);
+                    jobPool.execute(new JobRunner(executable));
+                    logger.info(jobDesc + " scheduled");
+                } catch (Exception ex) {
+                    context.removeRunningJob(executable);
+                    logger.warn(jobDesc + " fail to schedule", ex);
+                }
+            }
+            logger.info("Job Fetcher: " + nRunning + " running, " + runningJobs.size() + " actual running, " + nReady + " ready, " + nOthers + " others");
+        }
+    }
+
+    private class JobRunner implements Runnable {
+
+        private final AbstractExecutable executable;
+
+        public JobRunner(AbstractExecutable executable) {
+            this.executable = executable;
+        }
+
+        @Override
+        public void run() {
+            try {
+                executable.execute(context);
+                // trigger the next step asap
+                fetcherPool.schedule(fetcher, 0, TimeUnit.SECONDS);
+            } catch (ExecuteException e) {
+                logger.error("ExecuteException job:" + executable.getId(), e);
+            } catch (Exception e) {
+                logger.error("unknown error execute job:" + executable.getId(), e);
+            } finally {
+                context.removeRunningJob(executable);
+            }
+        }
+    }
+
+    public static DefaultScheduler getInstance() {
+        return INSTANCE;
+    }
+
+    @Override
+    public void stateChanged(CuratorFramework client, ConnectionState newState) {
+        if ((newState == ConnectionState.SUSPENDED) || (newState == ConnectionState.LOST)) {
+            try {
+                shutdown();
+            } catch (SchedulerException e) {
+                throw new RuntimeException("failed to shutdown scheduler", e);
+            }
+        }
+    }
+
+    @Override
+    public synchronized void init(JobEngineConfig jobEngineConfig, final JobLock jobLock) throws SchedulerException {
+        if (!initialized) {
+            initialized = true;
+        } else {
+            return;
+        }
+
+        this.jobEngineConfig = jobEngineConfig;
+
+        if (jobLock.lock() == false) {
+            throw new IllegalStateException("Cannot start job scheduler due to lack of job lock");
+        }
+
+        executableManager = ExecutableManager.getInstance(jobEngineConfig.getConfig());
+        //load all executable, set them to a consistent status
+        fetcherPool = Executors.newScheduledThreadPool(1);
+        int corePoolSize = jobEngineConfig.getMaxConcurrentJobLimit();
+        jobPool = new ThreadPoolExecutor(corePoolSize, corePoolSize, Long.MAX_VALUE, TimeUnit.DAYS, new SynchronousQueue<Runnable>());
+        context = new DefaultContext(Maps.<String, Executable> newConcurrentMap(), jobEngineConfig.getConfig());
+
+        for (AbstractExecutable executable : executableManager.getAllExecutables()) {
+            if (executable.getStatus() == ExecutableState.READY) {
+                executableManager.updateJobOutput(executable.getId(), ExecutableState.ERROR, null, "scheduler initializing work to reset job to ERROR status");
+            }
+        }
+        executableManager.updateAllRunningJobsToError();
+
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            public void run() {
+                logger.debug("Closing zk connection");
+                try {
+                    shutdown();
+                    jobLock.unlock();
+                } catch (SchedulerException e) {
+                    logger.error("error shutdown scheduler", e);
+                }
+            }
+        });
+
+        fetcher = new FetcherRunner();
+        fetcherPool.scheduleAtFixedRate(fetcher, 10, ExecutableConstants.DEFAULT_SCHEDULER_INTERVAL_SECONDS, TimeUnit.SECONDS);
+        hasStarted = true;
+    }
+
+    @Override
+    public void shutdown() throws SchedulerException {
+        fetcherPool.shutdown();
+        jobPool.shutdown();
+    }
+
+    @Override
+    public boolean stop(AbstractExecutable executable) throws SchedulerException {
+        if (hasStarted) {
+            return true;
+        } else {
+            //TODO should try to stop this executable
+            return true;
+        }
+    }
+
+    public boolean hasStarted() {
+        return this.hasStarted;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java b/core-job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
new file mode 100644
index 0000000..05f8c8e
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
@@ -0,0 +1,318 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.manager;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.job.dao.ExecutableDao;
+import org.apache.kylin.job.dao.ExecutableOutputPO;
+import org.apache.kylin.job.dao.ExecutablePO;
+import org.apache.kylin.job.exception.IllegalStateTranferException;
+import org.apache.kylin.job.exception.PersistentException;
+import org.apache.kylin.job.execution.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.lang.reflect.Constructor;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ */
+public class ExecutableManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(ExecutableManager.class);
+    private static final ConcurrentHashMap<KylinConfig, ExecutableManager> CACHE = new ConcurrentHashMap<KylinConfig, ExecutableManager>();
+    @SuppressWarnings("unused")
+    private final KylinConfig config;
+
+    private ExecutableDao executableDao;
+
+    public static ExecutableManager getInstance(KylinConfig config) {
+        ExecutableManager r = CACHE.get(config);
+        if (r == null) {
+            r = new ExecutableManager(config);
+            CACHE.put(config, r);
+            if (CACHE.size() > 1) {
+                logger.warn("More than one singleton exist");
+            }
+
+        }
+        return r;
+    }
+
+    private ExecutableManager(KylinConfig config) {
+        logger.info("Using metadata url: " + config);
+        this.config = config;
+        this.executableDao = ExecutableDao.getInstance(config);
+    }
+
+    public void addJob(AbstractExecutable executable) {
+        try {
+            executableDao.addJob(parse(executable));
+            addJobOutput(executable);
+        } catch (PersistentException e) {
+            logger.error("fail to submit job:" + executable.getId(), e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    private void addJobOutput(AbstractExecutable executable) throws PersistentException {
+        ExecutableOutputPO executableOutputPO = new ExecutableOutputPO();
+        executableOutputPO.setUuid(executable.getId());
+        executableDao.addJobOutput(executableOutputPO);
+        if (executable instanceof DefaultChainedExecutable) {
+            for (AbstractExecutable subTask: ((DefaultChainedExecutable) executable).getTasks()) {
+                addJobOutput(subTask);
+            }
+        }
+    }
+
+    //for ut
+    public void deleteJob(String jobId) {
+        try {
+            executableDao.deleteJob(jobId);
+        } catch (PersistentException e) {
+            logger.error("fail to delete job:" + jobId, e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public AbstractExecutable getJob(String uuid) {
+        try {
+            return parseTo(executableDao.getJob(uuid));
+        } catch (PersistentException e) {
+            logger.error("fail to get job:" + uuid, e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public Output getOutput(String uuid) {
+        try {
+            final ExecutableOutputPO jobOutput = executableDao.getJobOutput(uuid);
+            Preconditions.checkArgument(jobOutput != null, "there is no related output for job id:" + uuid);
+            return parseOutput(jobOutput);
+        } catch (PersistentException e) {
+            logger.error("fail to get job output:" + uuid, e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    private DefaultOutput parseOutput(ExecutableOutputPO jobOutput) {
+        final DefaultOutput result = new DefaultOutput();
+        result.setExtra(jobOutput.getInfo());
+        result.setState(ExecutableState.valueOf(jobOutput.getStatus()));
+        result.setVerboseMsg(jobOutput.getContent());
+        result.setLastModified(jobOutput.getLastModified());
+        return result;
+    }
+
+    public Map<String, Output> getAllOutputs() {
+        try {
+            final List<ExecutableOutputPO> jobOutputs = executableDao.getJobOutputs();
+            HashMap<String, Output> result = Maps.newHashMap();
+            for (ExecutableOutputPO jobOutput : jobOutputs) {
+                result.put(jobOutput.getId(), parseOutput(jobOutput));
+            }
+            return result;
+        } catch (PersistentException e) {
+            logger.error("fail to get all job output:", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public List<AbstractExecutable> getAllExecutables() {
+        try {
+            return Lists.transform(executableDao.getJobs(), new Function<ExecutablePO, AbstractExecutable>() {
+                @Nullable
+                @Override
+                public AbstractExecutable apply(ExecutablePO input) {
+                        return parseTo(input);
+                }
+            });
+        } catch (PersistentException e) {
+            logger.error("error get All Jobs", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public List<String> getAllJobIds() {
+        try {
+            return executableDao.getJobIds();
+        } catch (PersistentException e) {
+            logger.error("error get All Job Ids", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void updateAllRunningJobsToError() {
+        try {
+            final List<ExecutableOutputPO> jobOutputs = executableDao.getJobOutputs();
+            for (ExecutableOutputPO executableOutputPO : jobOutputs) {
+                if (executableOutputPO.getStatus().equalsIgnoreCase(ExecutableState.RUNNING.toString())) {
+                    executableOutputPO.setStatus(ExecutableState.ERROR.toString());
+                    executableDao.updateJobOutput(executableOutputPO);
+                }
+            }
+        } catch (PersistentException e) {
+            logger.error("error reset job status from RUNNING to ERROR", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void resumeJob(String jobId) {
+        AbstractExecutable job = getJob(jobId);
+        if (job == null) {
+            return;
+        }
+        updateJobOutput(jobId, ExecutableState.READY, null, null);
+        if (job instanceof DefaultChainedExecutable) {
+            List<AbstractExecutable> tasks = ((DefaultChainedExecutable) job).getTasks();
+            for (AbstractExecutable task : tasks) {
+                if (task.getStatus() == ExecutableState.ERROR) {
+                    updateJobOutput(task.getId(), ExecutableState.READY, null, null);
+                    break;
+                }
+            }
+        }
+    }
+
+    public void discardJob(String jobId) {
+        AbstractExecutable job = getJob(jobId);
+        if (job instanceof DefaultChainedExecutable) {
+            List<AbstractExecutable> tasks = ((DefaultChainedExecutable) job).getTasks();
+            for (AbstractExecutable task : tasks) {
+                if (!task.getStatus().isFinalState()) {
+                    updateJobOutput(task.getId(), ExecutableState.DISCARDED, null, null);
+                }
+            }
+        }
+        updateJobOutput(jobId, ExecutableState.DISCARDED, null, null);
+    }
+
+    public void updateJobOutput(String jobId, ExecutableState newStatus, Map<String, String> info, String output) {
+        try {
+            final ExecutableOutputPO jobOutput = executableDao.getJobOutput(jobId);
+            Preconditions.checkArgument(jobOutput != null, "there is no related output for job id:" + jobId);
+            ExecutableState oldStatus = ExecutableState.valueOf(jobOutput.getStatus());
+            if (newStatus != null && oldStatus != newStatus) {
+                if (!ExecutableState.isValidStateTransfer(oldStatus, newStatus)) {
+                    throw new IllegalStateTranferException("there is no valid state transfer from:" + oldStatus + " to:" + newStatus);
+                }
+                jobOutput.setStatus(newStatus.toString());
+            }
+            if (info != null) {
+                jobOutput.setInfo(info);
+            }
+            if (output != null) {
+                jobOutput.setContent(output);
+            }
+            executableDao.updateJobOutput(jobOutput);
+            logger.info("job id:" + jobId + " from " + oldStatus + " to " + newStatus);
+        } catch (PersistentException e) {
+            logger.error("error change job:" + jobId + " to " + newStatus.toString());
+            throw new RuntimeException(e);
+        }
+    }
+
+    //for migration only
+    //TODO delete when migration finished
+    public void resetJobOutput(String jobId, ExecutableState state, String output) {
+        try {
+            final ExecutableOutputPO jobOutput = executableDao.getJobOutput(jobId);
+            jobOutput.setStatus(state.toString());
+            if (output != null) {
+                jobOutput.setContent(output);
+            }
+            executableDao.updateJobOutput(jobOutput);
+        } catch (PersistentException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void addJobInfo(String id, Map<String, String> info) {
+        if (info == null) {
+            return;
+        }
+        try {
+            ExecutableOutputPO output = executableDao.getJobOutput(id);
+            Preconditions.checkArgument(output != null, "there is no related output for job id:" + id);
+            output.getInfo().putAll(info);
+            executableDao.updateJobOutput(output);
+        } catch (PersistentException e) {
+            logger.error("error update job info, id:" + id + "  info:" + info.toString());
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void addJobInfo(String id, String key, String value) {
+        Map<String, String> info = Maps.newHashMap();
+        info.put(key, value);
+        addJobInfo(id, info);
+    }
+
+    private static ExecutablePO parse(AbstractExecutable executable) {
+        ExecutablePO result = new ExecutablePO();
+        result.setName(executable.getName());
+        result.setUuid(executable.getId());
+        result.setType(executable.getClass().getName());
+        result.setParams(executable.getParams());
+        if (executable instanceof ChainedExecutable) {
+            List<ExecutablePO> tasks = Lists.newArrayList();
+            for (AbstractExecutable task : ((ChainedExecutable) executable).getTasks()) {
+                tasks.add(parse(task));
+            }
+            result.setTasks(tasks);
+        }
+        return result;
+    }
+
+    private static AbstractExecutable parseTo(ExecutablePO executablePO) {
+        if (executablePO == null) {
+            return null;
+        }
+        String type = executablePO.getType();
+        try {
+            Class<? extends AbstractExecutable> clazz = ClassUtil.forName(type, AbstractExecutable.class);
+            Constructor<? extends AbstractExecutable> constructor = clazz.getConstructor();
+            AbstractExecutable result = constructor.newInstance();
+            result.setId(executablePO.getUuid());
+            result.setName(executablePO.getName());
+            result.setParams(executablePO.getParams());
+            List<ExecutablePO> tasks = executablePO.getTasks();
+            if (tasks != null && !tasks.isEmpty()) {
+                Preconditions.checkArgument(result instanceof ChainedExecutable);
+                for (ExecutablePO subTask: tasks) {
+                    ((ChainedExecutable) result).addTask(parseTo(subTask));
+                }
+            }
+            return result;
+        } catch (ReflectiveOperationException e) {
+            throw new IllegalArgumentException("cannot parse this job:" + executablePO.getId(), e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/test/java/org/apache/kylin/job/BaseTestExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/BaseTestExecutable.java b/core-job/src/test/java/org/apache/kylin/job/BaseTestExecutable.java
new file mode 100644
index 0000000..5ab97c9
--- /dev/null
+++ b/core-job/src/test/java/org/apache/kylin/job/BaseTestExecutable.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import org.apache.kylin.job.execution.AbstractExecutable;
+
+/**
+ */
+public abstract class BaseTestExecutable extends AbstractExecutable {
+
+
+    public BaseTestExecutable() {
+        super();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java b/core-job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java
new file mode 100644
index 0000000..d034a73
--- /dev/null
+++ b/core-job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java
@@ -0,0 +1,24 @@
+package org.apache.kylin.job;
+
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ */
+public class BasicLocalMetaTest extends LocalFileMetadataTestCase {
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void basicTest() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/test/java/org/apache/kylin/job/ErrorTestExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/ErrorTestExecutable.java b/core-job/src/test/java/org/apache/kylin/job/ErrorTestExecutable.java
new file mode 100644
index 0000000..ac4828e
--- /dev/null
+++ b/core-job/src/test/java/org/apache/kylin/job/ErrorTestExecutable.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+
+/**
+ */
+public class ErrorTestExecutable extends BaseTestExecutable {
+
+    public ErrorTestExecutable() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        try {
+            Thread.sleep(1000);
+        } catch (InterruptedException e) {
+        }
+        throw new RuntimeException("test error");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/test/java/org/apache/kylin/job/ExecutableManagerTest.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/ExecutableManagerTest.java b/core-job/src/test/java/org/apache/kylin/job/ExecutableManagerTest.java
new file mode 100644
index 0000000..4bca55c
--- /dev/null
+++ b/core-job/src/test/java/org/apache/kylin/job/ExecutableManagerTest.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.job.BaseTestExecutable;
+import org.apache.kylin.job.SucceedTestExecutable;
+import org.apache.kylin.job.exception.IllegalStateTranferException;
+import org.apache.kylin.job.execution.ChainedExecutable;
+import org.apache.kylin.job.execution.Executable;
+import org.apache.kylin.job.execution.ExecutableState;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.job.manager.ExecutableManager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+/**
+ */
+public class ExecutableManagerTest extends LocalFileMetadataTestCase {
+
+    private ExecutableManager service;
+
+    @Before
+    public void setup() throws Exception {
+        createTestMetadata();
+        service = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+        for (String jobId: service.getAllJobIds()) {
+            System.out.println("deleting " + jobId);
+            service.deleteJob(jobId);
+        }
+
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void test() throws Exception {
+        assertNotNull(service);
+        BaseTestExecutable executable = new SucceedTestExecutable();
+        executable.setParam("test1", "test1");
+        executable.setParam("test2", "test2");
+        executable.setParam("test3", "test3");
+        service.addJob(executable);
+        List<AbstractExecutable> result = service.getAllExecutables();
+        assertEquals(1, result.size());
+        AbstractExecutable another = service.getJob(executable.getId());
+        assertJobEqual(executable, another);
+
+        service.updateJobOutput(executable.getId(), ExecutableState.RUNNING, null, "test output");
+        assertJobEqual(executable, service.getJob(executable.getId()));
+    }
+
+    @Test
+    public void testDefaultChainedExecutable() throws Exception {
+        DefaultChainedExecutable job = new DefaultChainedExecutable();
+        job.addTask(new SucceedTestExecutable());
+        job.addTask(new SucceedTestExecutable());
+
+        service.addJob(job);
+        assertEquals(2, job.getTasks().size());
+        AbstractExecutable anotherJob = service.getJob(job.getId());
+        assertEquals(DefaultChainedExecutable.class, anotherJob.getClass());
+        assertEquals(2, ((DefaultChainedExecutable) anotherJob).getTasks().size());
+        assertJobEqual(job, anotherJob);
+    }
+
+    @Test
+    public void testValidStateTransfer() throws Exception {
+        SucceedTestExecutable job = new SucceedTestExecutable();
+        String id = job.getId();
+        service.addJob(job);
+        service.updateJobOutput(id, ExecutableState.RUNNING, null, null);
+        service.updateJobOutput(id, ExecutableState.ERROR, null, null);
+        service.updateJobOutput(id, ExecutableState.READY, null, null);
+        service.updateJobOutput(id, ExecutableState.RUNNING, null, null);
+        service.updateJobOutput(id, ExecutableState.READY, null, null);
+        service.updateJobOutput(id, ExecutableState.RUNNING, null, null);
+        service.updateJobOutput(id, ExecutableState.SUCCEED, null, null);
+    }
+
+    @Test(expected = IllegalStateTranferException.class)
+    public void testInvalidStateTransfer(){
+        SucceedTestExecutable job = new SucceedTestExecutable();
+        service.addJob(job);
+        service.updateJobOutput(job.getId(), ExecutableState.RUNNING, null, null);
+        service.updateJobOutput(job.getId(), ExecutableState.STOPPED, null, null);
+    }
+
+
+
+    private static void assertJobEqual(Executable one, Executable another) {
+        assertEquals(one.getClass(), another.getClass());
+        assertEquals(one.getId(), another.getId());
+        assertEquals(one.getStatus(), another.getStatus());
+        assertEquals(one.isRunnable(), another.isRunnable());
+        assertEquals(one.getOutput(), another.getOutput());
+        assertTrue((one.getParams() == null && another.getParams() == null) || (one.getParams() != null && another.getParams() != null));
+        if (one.getParams() != null) {
+            assertEquals(one.getParams().size(), another.getParams().size());
+            for (String key : one.getParams().keySet()) {
+                assertEquals(one.getParams().get(key), another.getParams().get(key));
+            }
+        }
+        if (one instanceof ChainedExecutable) {
+            assertTrue(another instanceof ChainedExecutable);
+            List<? extends Executable> onesSubs = ((ChainedExecutable) one).getTasks();
+            List<? extends Executable> anotherSubs = ((ChainedExecutable) another).getTasks();
+            assertTrue((onesSubs == null && anotherSubs == null) || (onesSubs != null && anotherSubs != null));
+            if (onesSubs != null) {
+                assertEquals(onesSubs.size(), anotherSubs.size());
+                for (int i = 0; i < onesSubs.size(); ++i) {
+                    assertJobEqual(onesSubs.get(i), anotherSubs.get(i));
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/test/java/org/apache/kylin/job/FailedTestExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/FailedTestExecutable.java b/core-job/src/test/java/org/apache/kylin/job/FailedTestExecutable.java
new file mode 100644
index 0000000..e18cc0d
--- /dev/null
+++ b/core-job/src/test/java/org/apache/kylin/job/FailedTestExecutable.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+
+/**
+ */
+public class FailedTestExecutable extends BaseTestExecutable {
+
+    public FailedTestExecutable() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        try {
+            Thread.sleep(1000);
+        } catch (InterruptedException e) {
+        }
+        return new ExecuteResult(ExecuteResult.State.FAILED, "failed");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/test/java/org/apache/kylin/job/JoinedFlatTableTest.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/JoinedFlatTableTest.java b/core-job/src/test/java/org/apache/kylin/job/JoinedFlatTableTest.java
new file mode 100644
index 0000000..48f566e
--- /dev/null
+++ b/core-job/src/test/java/org/apache/kylin/job/JoinedFlatTableTest.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.job.JoinedFlatTable;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+@Ignore("This test case doesn't have much value, ignore it.")
+public class JoinedFlatTableTest extends LocalFileMetadataTestCase {
+
+    CubeInstance cube = null;
+    CubeJoinedFlatTableDesc intermediateTableDesc = null;
+    String fakeJobUUID = "abc-def";
+    CubeSegment cubeSegment = null;
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        cube = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_ready");
+        cubeSegment = cube.getSegments().get(0);
+        intermediateTableDesc = new CubeJoinedFlatTableDesc(cube.getDescriptor(), cubeSegment);
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testGenCreateTableDDL() {
+        String ddl = JoinedFlatTable.generateCreateTableStatement(intermediateTableDesc, "/tmp");
+        System.out.println(ddl);
+
+        System.out.println("The length for the ddl is " + ddl.length());
+    }
+
+    @Test
+    public void testGenDropTableDDL() {
+        String ddl = JoinedFlatTable.generateDropTableStatement(intermediateTableDesc);
+        System.out.println(ddl);
+        assertEquals(107, ddl.length());
+    }
+
+    @Test
+    public void testGenerateInsertSql() throws IOException {
+        String sqls = JoinedFlatTable.generateInsertDataStatement(intermediateTableDesc, new JobEngineConfig(KylinConfig.getInstanceFromEnv()));
+        System.out.println(sqls);
+
+        int length = sqls.length();
+        assertEquals(1155, length);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/test/java/org/apache/kylin/job/SelfStopExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/SelfStopExecutable.java b/core-job/src/test/java/org/apache/kylin/job/SelfStopExecutable.java
new file mode 100644
index 0000000..b4f6a98
--- /dev/null
+++ b/core-job/src/test/java/org/apache/kylin/job/SelfStopExecutable.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+
+/**
+ */
+public class SelfStopExecutable extends BaseTestExecutable {
+
+    public SelfStopExecutable() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        try {
+            Thread.sleep(5000);
+        } catch (InterruptedException e) {
+        }
+        if (isDiscarded()) {
+            return new ExecuteResult(ExecuteResult.State.STOPPED, "stopped");
+        } else {
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/test/java/org/apache/kylin/job/SucceedTestExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/SucceedTestExecutable.java b/core-job/src/test/java/org/apache/kylin/job/SucceedTestExecutable.java
new file mode 100644
index 0000000..1421f10
--- /dev/null
+++ b/core-job/src/test/java/org/apache/kylin/job/SucceedTestExecutable.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+
+/**
+ */
+public class SucceedTestExecutable extends BaseTestExecutable {
+
+    public SucceedTestExecutable() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        try {
+            Thread.sleep(1000);
+        } catch (InterruptedException e) {
+        }
+        return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-storage/pom.xml
----------------------------------------------------------------------
diff --git a/core-storage/pom.xml b/core-storage/pom.xml
index 1219d4f..da1006d 100644
--- a/core-storage/pom.xml
+++ b/core-storage/pom.xml
@@ -41,6 +41,12 @@
             <version>${project.parent.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>net.sf.ehcache</groupId>
+            <artifactId>ehcache</artifactId>
+            <version>2.8.1</version>
+        </dependency>
+
         <!-- Env & Test -->
         <dependency>
             <groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-storage/src/main/java/org/apache/kylin/storage/ICachableStorageQuery.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/ICachableStorageQuery.java b/core-storage/src/main/java/org/apache/kylin/storage/ICachableStorageQuery.java
new file mode 100644
index 0000000..179202e
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/ICachableStorageQuery.java
@@ -0,0 +1,33 @@
+package org.apache.kylin.storage;
+
+import com.google.common.collect.Range;
+
+/**
+ */
+public interface ICachableStorageQuery extends IStorageQuery {
+    /**
+     *
+     * being dynamic => getVolatilePeriod() return not null
+     * being dynamic => partition column of its realization not null
+     *
+     * @return true for static storage like cubes
+     *          false for dynamic storage like II
+     */
+    boolean isDynamic();
+
+    /**
+     * volatile period is the period of time in which the returned data is not stable
+     * e.g. inverted index's last several minutes' data is dynamic as time goes by.
+     * data in this period cannot be cached
+     *
+     * This method should not be called before ITupleIterator.close() is called
+     *
+     * @return null if the underlying storage guarantees the data is static
+     */
+    Range<Long> getVolatilePeriod();
+
+    /**
+     * get the uuid for the realization that serves this query
+     */
+    String getStorageUUID();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-storage/src/main/java/org/apache/kylin/storage/IStorage.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/IStorage.java b/core-storage/src/main/java/org/apache/kylin/storage/IStorage.java
new file mode 100644
index 0000000..89b96e9
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/IStorage.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+import org.apache.kylin.metadata.realization.IRealization;
+
+public interface IStorage {
+    
+    public IStorageQuery createStorageQuery(IRealization realization);
+
+    public <I> I adaptToBuildEngine(Class<I> engineInterface);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java b/core-storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java
new file mode 100644
index 0000000..f090ebb
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.storage.tuple.TupleInfo;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public interface IStorageQuery {
+
+    ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo);
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
new file mode 100644
index 0000000..caa2439
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+
+/**
+ * @author xjiang
+ */
+public class StorageContext {
+
+    public static final int HARD_THRESHOLD = 4000000;
+    public static final int DEFAULT_THRESHOLD = 1000000;
+
+    public enum OrderEnum {
+        ASCENDING, DESCENDING
+    }
+
+    private String connUrl;
+    private int threshold;
+    private int limit;
+    private boolean hasSort;
+    private List<MeasureDesc> sortMeasures;
+    private List<OrderEnum> sortOrders;
+    private boolean acceptPartialResult;
+
+    private boolean exactAggregation;
+    private boolean enableLimit;
+    private boolean enableCoprocessor;
+
+    private long totalScanCount;
+    private Cuboid cuboid;
+    private boolean partialResultReturned;
+
+    public StorageContext() {
+        this.threshold = DEFAULT_THRESHOLD;
+        this.limit = DEFAULT_THRESHOLD;
+        this.totalScanCount = 0;
+        this.cuboid = null;
+        this.hasSort = false;
+        this.sortOrders = new ArrayList<OrderEnum>();
+        this.sortMeasures = new ArrayList<MeasureDesc>();
+
+        this.exactAggregation = false;
+        this.enableLimit = false;
+        this.enableCoprocessor = false;
+
+        this.acceptPartialResult = false;
+        this.partialResultReturned = false;
+    }
+
+    public String getConnUrl() {
+        return connUrl;
+    }
+
+    public void setConnUrl(String connUrl) {
+        this.connUrl = connUrl;
+    }
+
+    public int getThreshold() {
+        return threshold;
+    }
+
+    public void setThreshold(int t) {
+        threshold = Math.min(t, HARD_THRESHOLD);
+    }
+
+    public int getLimit() {
+        return limit;
+    }
+
+    public void setLimit(int l) {
+        this.limit = l;
+    }
+
+    public void enableLimit() {
+        this.enableLimit = true;
+    }
+
+    public boolean isLimitEnabled() {
+        return this.enableLimit;
+    }
+
+    public void addSort(MeasureDesc measure, OrderEnum order) {
+        if (measure != null) {
+            sortMeasures.add(measure);
+            sortOrders.add(order);
+        }
+    }
+
+    public void markSort() {
+        this.hasSort = true;
+    }
+
+    public boolean hasSort() {
+        return this.hasSort;
+    }
+
+    public void setCuboid(Cuboid c) {
+        cuboid = c;
+    }
+
+    public Cuboid getCuboid() {
+        return cuboid;
+    }
+
+    public long getTotalScanCount() {
+        return totalScanCount;
+    }
+
+    public void setTotalScanCount(long totalScanCount) {
+        this.totalScanCount = totalScanCount;
+    }
+
+    public boolean isAcceptPartialResult() {
+        return acceptPartialResult;
+    }
+
+    public void setAcceptPartialResult(boolean acceptPartialResult) {
+        this.acceptPartialResult = acceptPartialResult;
+    }
+
+    public boolean isPartialResultReturned() {
+        return partialResultReturned;
+    }
+
+    public void setPartialResultReturned(boolean partialResultReturned) {
+        this.partialResultReturned = partialResultReturned;
+    }
+
+    public void setExactAggregation(boolean isExactAggregation) {
+        this.exactAggregation = isExactAggregation;
+    }
+
+    public boolean isExactAggregation() {
+        return this.exactAggregation;
+    }
+
+    public void enableCoprocessor() {
+        this.enableCoprocessor = true;
+    }
+
+    public boolean isCoprocessorEnabled() {
+        return this.enableCoprocessor;
+    }
+
+}



[23/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilderTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilderTest.java b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilderTest.java
new file mode 100644
index 0000000..0b812a6
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilderTest.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.cube.inmemcubing;
+
+import static org.junit.Assert.*;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.inmemcubing.DoggedCubeBuilder;
+import org.apache.kylin.cube.inmemcubing.ICuboidWriter;
+import org.apache.kylin.cube.inmemcubing.InMemCubeBuilder;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class DoggedCubeBuilderTest extends LocalFileMetadataTestCase {
+
+    @SuppressWarnings("unused")
+    private static final Logger logger = LoggerFactory.getLogger(DoggedCubeBuilderTest.class);
+
+    private static final int INPUT_ROWS = 10000;
+    private static final int SPLIT_ROWS = 5000;
+    private static final int THREADS = 4;
+
+    private static CubeInstance cube;
+    private static String flatTable;
+    private static Map<TblColRef, Dictionary<?>> dictionaryMap;
+
+    @BeforeClass
+    public static void before() throws IOException {
+        staticCreateTestMetadata();
+
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        CubeManager cubeManager = CubeManager.getInstance(kylinConfig);
+
+        cube = cubeManager.getCube("test_kylin_cube_without_slr_left_join_empty");
+        flatTable = "../examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv";
+        dictionaryMap = InMemCubeBuilderTest.getDictionaryMap(cube, flatTable);
+    }
+
+    @AfterClass
+    public static void after() throws Exception {
+        staticCleanupTestMetadata();
+    }
+
+    @Test
+    public void test() throws Exception {
+
+        ArrayBlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(1000);
+        ExecutorService executorService = Executors.newSingleThreadExecutor();
+        long randSeed = System.currentTimeMillis();
+
+        DoggedCubeBuilder doggedBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap);
+        doggedBuilder.setConcurrentThreads(THREADS);
+        doggedBuilder.setSplitRowThreshold(SPLIT_ROWS);
+        FileRecordWriter doggedResult = new FileRecordWriter();
+
+        {
+            Future<?> future = executorService.submit(doggedBuilder.buildAsRunnable(queue, doggedResult));
+            InMemCubeBuilderTest.feedData(cube, flatTable, queue, INPUT_ROWS, randSeed);
+            future.get();
+            doggedResult.close();
+        }
+
+        InMemCubeBuilder inmemBuilder = new InMemCubeBuilder(cube.getDescriptor(), dictionaryMap);
+        inmemBuilder.setConcurrentThreads(THREADS);
+        FileRecordWriter inmemResult = new FileRecordWriter();
+
+        {
+            Future<?> future = executorService.submit(inmemBuilder.buildAsRunnable(queue, inmemResult));
+            InMemCubeBuilderTest.feedData(cube, flatTable, queue, INPUT_ROWS, randSeed);
+            future.get();
+            inmemResult.close();
+        }
+
+        fileCompare(doggedResult.file, inmemResult.file);
+        doggedResult.file.delete();
+        inmemResult.file.delete();
+    }
+
+    private void fileCompare(File file, File file2) throws IOException {
+        BufferedReader r1 = new BufferedReader(new InputStreamReader(new FileInputStream(file), "UTF-8"));
+        BufferedReader r2 = new BufferedReader(new InputStreamReader(new FileInputStream(file2), "UTF-8"));
+
+        String line1, line2;
+        do {
+            line1 = r1.readLine();
+            line2 = r2.readLine();
+            
+            assertEquals(line1, line2);
+            
+        } while (line1 != null || line2 != null);
+
+        r1.close();
+        r2.close();
+    }
+
+    class FileRecordWriter implements ICuboidWriter {
+
+        File file;
+        PrintWriter writer;
+
+        FileRecordWriter() throws IOException {
+            file = File.createTempFile("DoggedCubeBuilderTest_", ".data");
+            writer = new PrintWriter(file, "UTF-8");
+        }
+
+        @Override
+        public void write(long cuboidId, GTRecord record) throws IOException {
+            writer.print(cuboidId);
+            writer.print(", ");
+            writer.print(record.toString());
+            writer.println();
+        }
+
+        public void close() {
+            writer.close();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderTest.java b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderTest.java
new file mode 100644
index 0000000..1487dff
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderTest.java
@@ -0,0 +1,208 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.dict.DictionaryGenerator;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ */
+public class InMemCubeBuilderTest extends LocalFileMetadataTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(InMemCubeBuilderTest.class);
+
+    private static final int INPUT_ROWS = 70000;
+    private static final int THREADS = 4;
+    
+    private static CubeInstance cube;
+    private static String flatTable;
+    private static Map<TblColRef, Dictionary<?>> dictionaryMap;
+    
+    @BeforeClass
+    public static void before() throws IOException {
+        staticCreateTestMetadata();
+        
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        CubeManager cubeManager = CubeManager.getInstance(kylinConfig);
+        
+        cube = cubeManager.getCube("test_kylin_cube_without_slr_left_join_empty");
+        flatTable = "../examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv";
+        dictionaryMap = getDictionaryMap(cube, flatTable);
+    }
+
+    @AfterClass
+    public static void after() throws Exception {
+        staticCleanupTestMetadata();
+    }
+
+    @Test
+    public void test() throws Exception {
+
+        InMemCubeBuilder cubeBuilder = new InMemCubeBuilder(cube.getDescriptor(), dictionaryMap);
+        //DoggedCubeBuilder cubeBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap);
+        cubeBuilder.setConcurrentThreads(THREADS);
+        
+        ArrayBlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(1000);
+        ExecutorService executorService = Executors.newSingleThreadExecutor();
+
+        try {
+            // round 1
+            {
+                Future<?> future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new ConsoleGTRecordWriter()));
+                feedData(cube, flatTable, queue, INPUT_ROWS);
+                future.get();
+            }
+            
+            // round 2, zero input
+            {
+                Future<?> future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new ConsoleGTRecordWriter()));
+                feedData(cube, flatTable, queue, 0);
+                future.get();
+            }
+            
+            // round 3
+            {
+                Future<?> future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new ConsoleGTRecordWriter()));
+                feedData(cube, flatTable, queue, INPUT_ROWS);
+                future.get();
+            }
+            
+        } catch (Exception e) {
+            logger.error("stream build failed", e);
+            throw new IOException("Failed to build cube ", e);
+        }
+    }
+
+    static void feedData(final CubeInstance cube, final String flatTable, ArrayBlockingQueue<List<String>> queue, int count) throws IOException, InterruptedException {
+        feedData(cube, flatTable, queue, count, 0);
+    }
+    
+    static void feedData(final CubeInstance cube, final String flatTable, ArrayBlockingQueue<List<String>> queue, int count, long randSeed) throws IOException, InterruptedException {
+        CubeJoinedFlatTableDesc flatTableDesc = new CubeJoinedFlatTableDesc(cube.getDescriptor(), null);
+        int nColumns = flatTableDesc.getColumnList().size();
+
+        @SuppressWarnings("unchecked")
+        Set<String>[] distinctSets = new Set[nColumns];
+        for (int i = 0; i < nColumns; i++)
+            distinctSets[i] = new TreeSet<String>();
+
+        // get distinct values on each column
+        List<String> lines = FileUtils.readLines(new File(flatTable), "UTF-8");
+        for (String line : lines) {
+            String[] row = line.trim().split(",");
+            assert row.length == nColumns;
+            for (int i = 0; i < nColumns; i++)
+                distinctSets[i].add(row[i]);
+        }
+
+        List<String[]> distincts = new ArrayList<String[]>();
+        for (int i = 0; i < nColumns; i++) {
+            distincts.add((String[]) distinctSets[i].toArray(new String[distinctSets[i].size()]));
+        }
+
+        Random rand = new Random();
+        if (randSeed != 0)
+            rand.setSeed(randSeed);
+        
+        // output with random data
+        for (; count > 0; count--) {
+            ArrayList<String> row = new ArrayList<String>(nColumns);
+            for (int i = 0; i < nColumns; i++) {
+                String[] candidates = distincts.get(i);
+                row.add(candidates[rand.nextInt(candidates.length)]);
+            }
+            queue.put(row);
+        }
+        queue.put(new ArrayList<String>(0));
+    }
+
+    static Map<TblColRef, Dictionary<?>> getDictionaryMap(CubeInstance cube, String flatTable) throws IOException {
+        Map<TblColRef, Dictionary<?>> result = Maps.newHashMap();
+        CubeDesc desc = cube.getDescriptor();
+        CubeJoinedFlatTableDesc flatTableDesc = new CubeJoinedFlatTableDesc(desc, null);
+        int nColumns = flatTableDesc.getColumnList().size();
+
+        List<TblColRef> columns = Cuboid.getBaseCuboid(desc).getColumns();
+        for (int c = 0; c < columns.size(); c++) {
+            TblColRef col = columns.get(c);
+            if (desc.getRowkey().isUseDictionary(col)) {
+                logger.info("Building dictionary for " + col);
+                List<byte[]> valueList = readValueList(flatTable, nColumns, flatTableDesc.getRowKeyColumnIndexes()[c]);
+                Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueList(col.getType(), valueList);
+                result.put(col, dict);
+            }
+        }
+        return result;
+    }
+
+    private static List<byte[]> readValueList(String flatTable, int nColumns, int c) throws IOException {
+        List<byte[]> result = Lists.newArrayList();
+        List<String> lines = FileUtils.readLines(new File(flatTable), "UTF-8");
+        for (String line : lines) {
+            String[] row = line.trim().split(",");
+            assert row.length == nColumns;
+            if (row[c] != null) {
+                result.add(Bytes.toBytes(row[c]));
+            }
+        }
+        return result;
+    }
+
+    class ConsoleGTRecordWriter implements ICuboidWriter {
+
+        boolean verbose = false;
+
+        @Override
+        public void write(long cuboidId, GTRecord record) throws IOException {
+            if (verbose)
+                System.out.println(record.toString());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/MemDiskStoreTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/MemDiskStoreTest.java b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/MemDiskStoreTest.java
new file mode 100644
index 0000000..645908d
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/MemDiskStoreTest.java
@@ -0,0 +1,98 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.kylin.common.util.MemoryBudgetController;
+import org.apache.kylin.cube.inmemcubing.MemDiskStore;
+import org.apache.kylin.gridtable.GTBuilder;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.GridTable;
+import org.apache.kylin.gridtable.IGTScanner;
+import org.apache.kylin.gridtable.UnitTestSupport;
+import org.junit.Test;
+
+public class MemDiskStoreTest {
+
+    final MemoryBudgetController budgetCtrl = new MemoryBudgetController(20);
+    final GTInfo info = UnitTestSupport.advancedInfo();
+    final List<GTRecord> data = UnitTestSupport.mockupData(info, 1000000); // converts to about 34 MB data
+
+    @Test
+    public void testSingleThreadWriteRead() throws IOException {
+        long start = System.currentTimeMillis();
+        verifyOneTableWriteAndRead();
+        long end = System.currentTimeMillis();
+        System.out.println("Cost " + (end - start) + " millis");
+    }
+
+    @Test
+    public void testMultiThreadWriteRead() throws IOException, InterruptedException {
+        long start = System.currentTimeMillis();
+
+        int nThreads = 5;
+        Thread[] t = new Thread[nThreads];
+        for (int i = 0; i < nThreads; i++) {
+            t[i] = new Thread() {
+                public void run() {
+                    try {
+                        verifyOneTableWriteAndRead();
+                    } catch (Exception ex) {
+                        ex.printStackTrace();
+                    }
+                }
+            };
+            t[i].start();
+        }
+        for (int i = 0; i < nThreads; i++) {
+            t[i].join();
+        }
+
+        long end = System.currentTimeMillis();
+        System.out.println("Cost " + (end - start) + " millis");
+    }
+
+    private void verifyOneTableWriteAndRead() throws IOException {
+        MemDiskStore store = new MemDiskStore(info, budgetCtrl);
+        GridTable table = new GridTable(info, store);
+        verifyWriteAndRead(table);
+    }
+
+    private void verifyWriteAndRead(GridTable table) throws IOException {
+        GTInfo info = table.getInfo();
+
+        GTBuilder builder = table.rebuild();
+        for (GTRecord r : data) {
+            builder.write(r);
+        }
+        builder.close();
+
+        IGTScanner scanner = table.scan(new GTScanRequest(info));
+        int i = 0;
+        for (GTRecord r : scanner) {
+            assertEquals(data.get(i++), r);
+        }
+        scanner.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/MemoryBudgetControllerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/MemoryBudgetControllerTest.java b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/MemoryBudgetControllerTest.java
new file mode 100644
index 0000000..f7bf432
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/cube/inmemcubing/MemoryBudgetControllerTest.java
@@ -0,0 +1,98 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.cube.inmemcubing;
+
+import org.apache.kylin.common.util.MemoryBudgetController;
+import org.apache.kylin.common.util.MemoryBudgetController.NotEnoughBudgetException;
+import org.junit.Test;
+
+import java.util.ArrayList;
+
+import static org.junit.Assert.*;
+
+public class MemoryBudgetControllerTest {
+
+    @Test
+    public void test() {
+        final int n = MemoryBudgetController.getSystemAvailMB() / 2;
+        final MemoryBudgetController mbc = new MemoryBudgetController(n);
+
+        ArrayList<Consumer> mbList = new ArrayList<Consumer>();
+        for (int i = 0; i < n; i++) {
+            mbList.add(new Consumer(mbc));
+            assertEquals(mbList.size(), mbc.getTotalReservedMB());
+        }
+
+        // a's reservation will free up all the previous
+        final Consumer a = new Consumer();
+        mbc.reserve(a, n);
+        for (int i = 0; i < n; i++) {
+            assertEquals(null, mbList.get(i).data);
+        }
+        
+        // cancel a in 2 seconds
+        new Thread() {
+            @Override
+            public void run() {
+                try {
+                    Thread.sleep(2000);
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+                mbc.reserve(a, 0);
+            }
+        }.start();
+        
+        // b will success after some wait
+        long bWaitStart = System.currentTimeMillis();
+        final Consumer b = new Consumer();
+        mbc.reserveInsist(b, n);
+        assertTrue(System.currentTimeMillis() - bWaitStart > 1000);
+
+        try {
+            mbc.reserve(a, 1);
+            fail();
+        } catch (NotEnoughBudgetException ex) {
+            // expected
+        }
+    }
+
+    class Consumer implements MemoryBudgetController.MemoryConsumer {
+
+        byte[] data;
+
+        Consumer() {
+        }
+
+        Consumer(MemoryBudgetController mbc) {
+            mbc.reserve(this, 1);
+            data = new byte[MemoryBudgetController.ONE_MB - 24]; // 24 is object shell of this + object shell of data + reference of data 
+        }
+
+        @Override
+        public int freeUp(int mb) {
+            if (data != null) {
+                data = null;
+                return 1;
+            } else {
+                return 0;
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
new file mode 100644
index 0000000..f53f11f
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
@@ -0,0 +1,214 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.gridtable;
+
+import java.math.BigDecimal;
+import java.util.Comparator;
+import java.util.Random;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.gridtable.GTAggregateScanner;
+import org.apache.kylin.metadata.measure.BigDecimalSumAggregator;
+import org.apache.kylin.metadata.measure.DoubleSumAggregator;
+import org.apache.kylin.metadata.measure.DoubleMutable;
+import org.apache.kylin.metadata.measure.HLLCAggregator;
+import org.apache.kylin.metadata.measure.LongSumAggregator;
+import org.apache.kylin.metadata.measure.LongMutable;
+import org.apache.kylin.metadata.measure.MeasureAggregator;
+import org.junit.Test;
+
+public class AggregationCacheMemSizeTest {
+
+    public static final int NUM_OF_OBJS = 1000000 / 2;
+
+    interface CreateAnObject {
+        Object create();
+    }
+
+    @Test
+    public void testHLLCAggregatorSize() throws InterruptedException {
+        int est = estimateObjectSize(new CreateAnObject() {
+            @Override
+            public Object create() {
+                HLLCAggregator aggr = new HLLCAggregator(10);
+                aggr.aggregate(new HyperLogLogPlusCounter(10));
+                return aggr;
+            }
+        });
+        System.out.println("HLLC: " + est);
+    }
+
+    @Test
+    public void testBigDecimalAggregatorSize() throws InterruptedException {
+        int est = estimateObjectSize(new CreateAnObject() {
+            @Override
+            public Object create() {
+                return newBigDecimalAggr();
+            }
+
+        });
+        System.out.println("BigDecimal: " + est);
+    }
+
+    private BigDecimalSumAggregator newBigDecimalAggr() {
+        BigDecimalSumAggregator aggr = new BigDecimalSumAggregator();
+        aggr.aggregate(new BigDecimal("12345678901234567890.123456789"));
+        return aggr;
+    }
+
+    @Test
+    public void testLongAggregatorSize() throws InterruptedException {
+        int est = estimateObjectSize(new CreateAnObject() {
+            @Override
+            public Object create() {
+                return newLongAggr();
+            }
+        });
+        System.out.println("Long: " + est);
+    }
+
+    private LongSumAggregator newLongAggr() {
+        LongSumAggregator aggr = new LongSumAggregator();
+        aggr.aggregate(new LongMutable(10));
+        return aggr;
+    }
+
+    @Test
+    public void testDoubleAggregatorSize() throws InterruptedException {
+        int est = estimateObjectSize(new CreateAnObject() {
+            @Override
+            public Object create() {
+                return newDoubleAggr();
+            }
+        });
+        System.out.println("Double: " + est);
+    }
+
+    private DoubleSumAggregator newDoubleAggr() {
+        DoubleSumAggregator aggr = new DoubleSumAggregator();
+        aggr.aggregate(new DoubleMutable(10));
+        return aggr;
+    }
+
+    @Test
+    public void testByteArraySize() throws InterruptedException {
+        int est = estimateObjectSize(new CreateAnObject() {
+            @Override
+            public Object create() {
+                return new byte[10];
+            }
+        });
+        System.out.println("byte[10]: " + est);
+    }
+
+    @Test
+    public void testAggregatorArraySize() throws InterruptedException {
+        int est = estimateObjectSize(new CreateAnObject() {
+            @Override
+            public Object create() {
+                return new MeasureAggregator[7];
+            }
+        });
+        System.out.println("MeasureAggregator[7]: " + est);
+    }
+
+    @Test
+    public void testTreeMapSize() throws InterruptedException {
+        final SortedMap<byte[], Object> map = new TreeMap<byte[], Object>(new Comparator<byte[]>() {
+            @Override
+            public int compare(byte[] o1, byte[] o2) {
+                return Bytes.compareTo(o1, o2);
+            }
+        });
+        final Random rand = new Random();
+        int est = estimateObjectSize(new CreateAnObject() {
+            @Override
+            public Object create() {
+                byte[] key = new byte[10];
+                rand.nextBytes(key);
+                map.put(key, null);
+                return null;
+            }
+        });
+        System.out.println("TreeMap entry: " + (est - 20)); // -20 is to exclude byte[10]
+    }
+
+    @Test
+    public void testAggregationCacheSize() throws InterruptedException {
+        final SortedMap<byte[], Object> map = new TreeMap<byte[], Object>(new Comparator<byte[]>() {
+            @Override
+            public int compare(byte[] o1, byte[] o2) {
+                return Bytes.compareTo(o1, o2);
+            }
+        });
+        final Random rand = new Random();
+
+        long bytesBefore = memLeft();
+        byte[] key = null;
+        MeasureAggregator<?>[] aggrs = null;
+        for (int i = 0; i < NUM_OF_OBJS; i++) {
+            key = new byte[10];
+            rand.nextBytes(key);
+            aggrs = new MeasureAggregator[4];
+            aggrs[0] = newBigDecimalAggr();
+            aggrs[1] = newLongAggr();
+            aggrs[2] = newDoubleAggr();
+            aggrs[3] = newDoubleAggr();
+            map.put(key, aggrs);
+        }
+
+        long bytesAfter = memLeft();
+        
+        long mapActualSize = bytesBefore - bytesAfter;
+        long mapExpectSize = GTAggregateScanner.estimateSizeOfAggrCache(key, aggrs, map.size());
+        System.out.println("Actual cache size: " + mapActualSize);
+        System.out.println("Expect cache size: " + mapExpectSize);
+    }
+
+    private int estimateObjectSize(CreateAnObject factory) throws InterruptedException {
+        Object[] hold = new Object[NUM_OF_OBJS];
+        long bytesBefore = memLeft();
+
+        for (int i = 0; i < hold.length; i++) {
+            hold[i] = factory.create();
+        }
+
+        long bytesAfter = memLeft();
+        return (int) ((bytesBefore - bytesAfter) / hold.length);
+    }
+
+    private long memLeft() throws InterruptedException {
+        Runtime.getRuntime().gc();
+        Thread.sleep(500);
+        return getSystemAvailBytes();
+    }
+
+    private long getSystemAvailBytes() {
+        Runtime runtime = Runtime.getRuntime();
+        long totalMemory = runtime.totalMemory(); // current heap allocated to the VM process
+        long freeMemory = runtime.freeMemory(); // out of the current heap, how much is free
+        long maxMemory = runtime.maxMemory(); // Max heap VM can use e.g. Xmx setting
+        long usedMemory = totalMemory - freeMemory; // how much of the current heap the VM is using
+        long availableMemory = maxMemory - usedMemory; // available memory i.e. Maximum heap size minus the current amount used
+        return availableMemory;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java
new file mode 100644
index 0000000..c90c5d3
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java
@@ -0,0 +1,381 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.gridtable;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.cube.gridtable.CubeCodeSystem;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.dict.NumberDictionaryBuilder;
+import org.apache.kylin.dict.StringBytesConverter;
+import org.apache.kylin.dict.TrieDictionaryBuilder;
+import org.apache.kylin.gridtable.GTInfo.Builder;
+import org.apache.kylin.gridtable.memstore.GTSimpleMemStore;
+import org.apache.kylin.metadata.filter.ColumnTupleFilter;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.ConstantTupleFilter;
+import org.apache.kylin.metadata.filter.ExtractTupleFilter;
+import org.apache.kylin.metadata.filter.LogicalTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.measure.LongMutable;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class DictGridTableTest {
+
+    @Test
+    public void test() throws IOException {
+        GridTable table = newTestTable();
+        verifyScanRangePlanner(table);
+        verifyFirstRow(table);
+        verifyScanWithUnevaluatableFilter(table);
+        verifyScanWithEvaluatableFilter(table);
+        verifyConvertFilterConstants1(table);
+        verifyConvertFilterConstants2(table);
+        verifyConvertFilterConstants3(table);
+        verifyConvertFilterConstants4(table);
+    }
+
+    private void verifyScanRangePlanner(GridTable table) {
+        GTInfo info = table.getInfo();
+        GTScanRangePlanner planner = new GTScanRangePlanner(info);
+        
+        CompareTupleFilter timeComp1 = compare(info.colRef(0), FilterOperatorEnum.GT, enc(info, 0, "2015-01-14"));
+        CompareTupleFilter timeComp2 = compare(info.colRef(0), FilterOperatorEnum.LT, enc(info, 0, "2015-01-13"));
+        CompareTupleFilter timeComp3 = compare(info.colRef(0), FilterOperatorEnum.LT, enc(info, 0, "2015-01-15"));
+        CompareTupleFilter timeComp4 = compare(info.colRef(0), FilterOperatorEnum.EQ, enc(info, 0, "2015-01-15"));
+        CompareTupleFilter ageComp1 = compare(info.colRef(1), FilterOperatorEnum.EQ, enc(info, 1, "10"));
+        CompareTupleFilter ageComp2 = compare(info.colRef(1), FilterOperatorEnum.EQ, enc(info, 1, "20"));
+        CompareTupleFilter ageComp3 = compare(info.colRef(1), FilterOperatorEnum.EQ, enc(info, 1, "30"));
+        CompareTupleFilter ageComp4 = compare(info.colRef(1), FilterOperatorEnum.NEQ, enc(info, 1, "30"));
+        
+        // flatten or-and & hbase fuzzy value
+        {
+            LogicalTupleFilter filter = and(timeComp1, or(ageComp1, ageComp2));
+            List<GTScanRange> r = planner.planScanRanges(filter);
+            assertEquals(1, r.size());
+            assertEquals("[1421193600000, 10]-[null, null]", r.get(0).toString());
+            assertEquals("[[10], [20]]", r.get(0).hbaseFuzzyKeys.toString());
+        }
+        
+        // pre-evaluate ever false
+        {
+            LogicalTupleFilter filter = and(timeComp1, timeComp2);
+            List<GTScanRange> r = planner.planScanRanges(filter);
+            assertEquals(0, r.size());
+        }
+        
+        // pre-evaluate ever true
+        {
+            LogicalTupleFilter filter = or(timeComp1, ageComp4);
+            List<GTScanRange> r = planner.planScanRanges(filter);
+            assertEquals("[[null, null]-[null, null]]", r.toString());
+        }
+        
+        // merge overlap range
+        {
+            LogicalTupleFilter filter = or(timeComp1, timeComp3);
+            List<GTScanRange> r = planner.planScanRanges(filter);
+            assertEquals("[[null, null]-[null, null]]", r.toString());
+        }
+        
+        // merge too many ranges
+        {
+            LogicalTupleFilter filter = or(and(timeComp4, ageComp1), and(timeComp4, ageComp2), and(timeComp4, ageComp3));
+            List<GTScanRange> r = planner.planScanRanges(filter);
+            assertEquals(3, r.size());
+            assertEquals("[1421280000000, 10]-[1421280000000, 10]", r.get(0).toString());
+            assertEquals("[1421280000000, 20]-[1421280000000, 20]", r.get(1).toString());
+            assertEquals("[1421280000000, 30]-[1421280000000, 30]", r.get(2).toString());
+            List<GTScanRange> r2 = planner.planScanRanges(filter, 2);
+            assertEquals("[[1421280000000, 10]-[1421280000000, 30]]", r2.toString());
+        }
+    }
+
+    private void verifyFirstRow(GridTable table) throws IOException {
+        doScanAndVerify(table, new GTScanRequest(table.getInfo()), "[1421193600000, 30, Yang, 10, 10.5]");
+    }
+
+    private void verifyScanWithUnevaluatableFilter(GridTable table) throws IOException {
+        GTInfo info = table.getInfo();
+
+        CompareTupleFilter fComp = compare(info.colRef(0), FilterOperatorEnum.GT, enc(info, 0, "2015-01-14"));
+        ExtractTupleFilter fUnevaluatable = unevaluatable(info.colRef(1));
+        LogicalTupleFilter fNotPlusUnevaluatable = not(unevaluatable(info.colRef(1)));
+        LogicalTupleFilter filter = and(fComp, fUnevaluatable, fNotPlusUnevaluatable);
+
+        GTScanRequest req = new GTScanRequest(info, null, setOf(0), setOf(3), new String[] { "sum" }, filter);
+
+        // note the unEvaluatable column 1 in filter is added to group by
+        assertEquals("GTScanRequest [range=[null, null]-[null, null], columns={0, 1, 3}, filterPushDown=AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], [null], [null]], aggrGroupBy={0, 1}, aggrMetrics={3}, aggrMetricsFuncs=[sum]]", req.toString());
+        
+        doScanAndVerify(table, req, "[1421280000000, 20, null, 20, null]");
+    }
+    
+    private void verifyScanWithEvaluatableFilter(GridTable table) throws IOException {
+        GTInfo info = table.getInfo();
+
+        CompareTupleFilter fComp1 = compare(info.colRef(0), FilterOperatorEnum.GT, enc(info, 0, "2015-01-14"));
+        CompareTupleFilter fComp2 = compare(info.colRef(1), FilterOperatorEnum.GT, enc(info, 1, "10"));
+        LogicalTupleFilter filter = and(fComp1, fComp2);
+
+        GTScanRequest req = new GTScanRequest(info, null, setOf(0), setOf(3), new String[] { "sum" }, filter);
+        
+        // note the evaluatable column 1 in filter is added to returned columns but not in group by
+        assertEquals("GTScanRequest [range=[null, null]-[null, null], columns={0, 1, 3}, filterPushDown=AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], NULL.GT_MOCKUP_TABLE.1 GT [\\x00]], aggrGroupBy={0}, aggrMetrics={3}, aggrMetricsFuncs=[sum]]", req.toString());
+        
+        doScanAndVerify(table, req, "[1421280000000, 20, null, 30, null]", "[1421366400000, 20, null, 40, null]");
+    }
+
+    private void verifyConvertFilterConstants1(GridTable table) {
+        GTInfo info = table.getInfo();
+        
+        TableDesc extTable = TableDesc.mockup("ext");
+        TblColRef extColA = new TblColRef(ColumnDesc.mockup(extTable, 1, "A", "timestamp"));
+        TblColRef extColB = new TblColRef(ColumnDesc.mockup(extTable, 2, "B", "integer"));
+
+        CompareTupleFilter fComp1 = compare(extColA, FilterOperatorEnum.GT, "2015-01-14");
+        CompareTupleFilter fComp2 = compare(extColB, FilterOperatorEnum.EQ, "10");
+        LogicalTupleFilter filter = and(fComp1, fComp2);
+        
+        List<TblColRef> colMapping = Lists.newArrayList();
+        colMapping.add(extColA);
+        colMapping.add(extColB);
+        
+        TupleFilter newFilter = GTUtil.convertFilterColumnsAndConstants(filter, info, colMapping, null);
+        assertEquals("AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], NULL.GT_MOCKUP_TABLE.1 EQ [\\x00]]", newFilter.toString());
+    }
+
+    private void verifyConvertFilterConstants2(GridTable table) {
+        GTInfo info = table.getInfo();
+        
+        TableDesc extTable = TableDesc.mockup("ext");
+        TblColRef extColA = new TblColRef(ColumnDesc.mockup(extTable, 1, "A", "timestamp"));
+        TblColRef extColB = new TblColRef(ColumnDesc.mockup(extTable, 2, "B", "integer"));
+        
+        CompareTupleFilter fComp1 = compare(extColA, FilterOperatorEnum.GT, "2015-01-14");
+        CompareTupleFilter fComp2 = compare(extColB, FilterOperatorEnum.LT, "9");
+        LogicalTupleFilter filter = and(fComp1, fComp2);
+        
+        List<TblColRef> colMapping = Lists.newArrayList();
+        colMapping.add(extColA);
+        colMapping.add(extColB);
+        
+        // $1<"9" round up to $1<"10"
+        TupleFilter newFilter = GTUtil.convertFilterColumnsAndConstants(filter, info, colMapping, null);
+        assertEquals("AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], NULL.GT_MOCKUP_TABLE.1 LT [\\x00]]", newFilter.toString());
+    }
+    
+    private void verifyConvertFilterConstants3(GridTable table) {
+        GTInfo info = table.getInfo();
+        
+        TableDesc extTable = TableDesc.mockup("ext");
+        TblColRef extColA = new TblColRef(ColumnDesc.mockup(extTable, 1, "A", "timestamp"));
+        TblColRef extColB = new TblColRef(ColumnDesc.mockup(extTable, 2, "B", "integer"));
+        
+        CompareTupleFilter fComp1 = compare(extColA, FilterOperatorEnum.GT, "2015-01-14");
+        CompareTupleFilter fComp2 = compare(extColB, FilterOperatorEnum.LTE, "9");
+        LogicalTupleFilter filter = and(fComp1, fComp2);
+        
+        List<TblColRef> colMapping = Lists.newArrayList();
+        colMapping.add(extColA);
+        colMapping.add(extColB);
+        
+        // $1<="9" round down to FALSE
+        TupleFilter newFilter = GTUtil.convertFilterColumnsAndConstants(filter, info, colMapping, null);
+        assertEquals("AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], []]", newFilter.toString());
+    }
+    
+    private void verifyConvertFilterConstants4(GridTable table) {
+        GTInfo info = table.getInfo();
+        
+        TableDesc extTable = TableDesc.mockup("ext");
+        TblColRef extColA = new TblColRef(ColumnDesc.mockup(extTable, 1, "A", "timestamp"));
+        TblColRef extColB = new TblColRef(ColumnDesc.mockup(extTable, 2, "B", "integer"));
+        
+        CompareTupleFilter fComp1 = compare(extColA, FilterOperatorEnum.GT, "2015-01-14");
+        CompareTupleFilter fComp2 = compare(extColB, FilterOperatorEnum.IN, "9", "10", "15");
+        LogicalTupleFilter filter = and(fComp1, fComp2);
+        
+        List<TblColRef> colMapping = Lists.newArrayList();
+        colMapping.add(extColA);
+        colMapping.add(extColB);
+        
+        // $1 in ("9", "10", "15") has only "10" left
+        TupleFilter newFilter = GTUtil.convertFilterColumnsAndConstants(filter, info, colMapping, null);
+        assertEquals("AND [NULL.GT_MOCKUP_TABLE.0 GT [\\x00\\x00\\x01J\\xE5\\xBD\\x5C\\x00], NULL.GT_MOCKUP_TABLE.1 IN [\\x00]]", newFilter.toString());
+    }
+    
+    private void doScanAndVerify(GridTable table, GTScanRequest req, String... verifyRows) throws IOException {
+        System.out.println(req);
+        IGTScanner scanner = table.scan(req);
+        int i = 0;
+        for (GTRecord r : scanner) {
+            System.out.println(r);
+            if (verifyRows != null && i < verifyRows.length) {
+                assertEquals(verifyRows[i], r.toString());
+            }
+            i++;
+        }
+        scanner.close();
+    }
+
+    private Object enc(GTInfo info, int col, String value) {
+        ByteBuffer buf = ByteBuffer.allocate(info.getMaxColumnLength());
+        info.codeSystem.encodeColumnValue(col, value, buf);
+        return ByteArray.copyOf(buf.array(), buf.arrayOffset(), buf.position());
+    }
+
+    private ExtractTupleFilter unevaluatable(TblColRef col) {
+        ExtractTupleFilter r = new ExtractTupleFilter(FilterOperatorEnum.EXTRACT);
+        r.addChild(new ColumnTupleFilter(col));
+        return r;
+    }
+
+    private CompareTupleFilter compare(TblColRef col, FilterOperatorEnum op, Object... value) {
+        CompareTupleFilter result = new CompareTupleFilter(op);
+        result.addChild(new ColumnTupleFilter(col));
+        result.addChild(new ConstantTupleFilter(Arrays.asList(value)));
+        return result;
+    }
+
+    private LogicalTupleFilter and(TupleFilter... children) {
+        return logic(FilterOperatorEnum.AND, children);
+    }
+
+    private LogicalTupleFilter or(TupleFilter... children) {
+        return logic(FilterOperatorEnum.OR, children);
+    }
+
+    private LogicalTupleFilter not(TupleFilter child) {
+        return logic(FilterOperatorEnum.NOT, child);
+    }
+
+    private LogicalTupleFilter logic(FilterOperatorEnum op, TupleFilter... children) {
+        LogicalTupleFilter result = new LogicalTupleFilter(op);
+        for (TupleFilter c : children) {
+            result.addChild(c);
+        }
+        return result;
+    }
+
+    static GridTable newTestTable() throws IOException {
+        GTInfo info = newInfo();
+        GTSimpleMemStore store = new GTSimpleMemStore(info);
+        GridTable table = new GridTable(info, store);
+
+        GTRecord r = new GTRecord(table.getInfo());
+        GTBuilder builder = table.rebuild();
+
+        builder.write(r.setValues("2015-01-14", "30", "Yang", new LongMutable(10), new BigDecimal("10.5")));
+        builder.write(r.setValues("2015-01-14", "30", "Luke", new LongMutable(10), new BigDecimal("10.5")));
+        builder.write(r.setValues("2015-01-15", "20", "Dong", new LongMutable(10), new BigDecimal("10.5")));
+        builder.write(r.setValues("2015-01-15", "20", "Jason", new LongMutable(10), new BigDecimal("10.5")));
+        builder.write(r.setValues("2015-01-15", "30", "Xu", new LongMutable(10), new BigDecimal("10.5")));
+        builder.write(r.setValues("2015-01-16", "20", "Mahone", new LongMutable(10), new BigDecimal("10.5")));
+        builder.write(r.setValues("2015-01-16", "20", "Qianhao", new LongMutable(10), new BigDecimal("10.5")));
+        builder.write(r.setValues("2015-01-16", "30", "George", new LongMutable(10), new BigDecimal("10.5")));
+        builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new LongMutable(10), new BigDecimal("10.5")));
+        builder.write(r.setValues("2015-01-17", "10", "Kejia", new LongMutable(10), new BigDecimal("10.5")));
+        builder.close();
+
+        return table;
+    }
+
+    static GTInfo newInfo() {
+        Builder builder = GTInfo.builder();
+        builder.setCodeSystem(newDictCodeSystem());
+        builder.setColumns( //
+                DataType.getInstance("timestamp"), //
+                DataType.getInstance("integer"), //
+                DataType.getInstance("varchar(10)"), //
+                DataType.getInstance("bigint"), //
+                DataType.getInstance("decimal") //
+        );
+        builder.setPrimaryKey(setOf(0, 1));
+        builder.setColumnPreferIndex(setOf(0));
+        builder.enableColumnBlock(new ImmutableBitSet[] { setOf(0, 1), setOf(2), setOf(3, 4) });
+        builder.enableRowBlock(4);
+        GTInfo info = builder.build();
+        return info;
+    }
+
+    @SuppressWarnings("rawtypes")
+    private static CubeCodeSystem newDictCodeSystem() {
+        Map<Integer, Dictionary> dictionaryMap = Maps.newHashMap();
+        dictionaryMap.put(1, newDictionaryOfInteger());
+        dictionaryMap.put(2, newDictionaryOfString());
+        return new CubeCodeSystem(dictionaryMap);
+    }
+
+    @SuppressWarnings("rawtypes")
+    private static Dictionary newDictionaryOfString() {
+        TrieDictionaryBuilder<String> builder = new TrieDictionaryBuilder<>(new StringBytesConverter());
+        builder.addValue("Dong");
+        builder.addValue("George");
+        builder.addValue("Jason");
+        builder.addValue("Kejia");
+        builder.addValue("Luke");
+        builder.addValue("Mahone");
+        builder.addValue("Qianhao");
+        builder.addValue("Shaofeng");
+        builder.addValue("Xu");
+        builder.addValue("Yang");
+        return builder.build(0);
+    }
+
+    @SuppressWarnings("rawtypes")
+    private static Dictionary newDictionaryOfInteger() {
+        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<>(new StringBytesConverter());
+        builder.addValue("10");
+        builder.addValue("20");
+        builder.addValue("30");
+        builder.addValue("40");
+        builder.addValue("50");
+        builder.addValue("60");
+        builder.addValue("70");
+        builder.addValue("80");
+        builder.addValue("90");
+        builder.addValue("100");
+        return builder.build(0);
+    }
+
+    private static ImmutableBitSet setOf(int... values) {
+        BitSet set = new BitSet();
+        for (int i : values)
+            set.set(i);
+        return new ImmutableBitSet(set);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
new file mode 100644
index 0000000..c87e970
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
@@ -0,0 +1,195 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.gridtable;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.gridtable.GTBuilder;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.GridTable;
+import org.apache.kylin.gridtable.IGTScanner;
+import org.apache.kylin.gridtable.UnitTestSupport;
+import org.apache.kylin.gridtable.memstore.GTSimpleMemStore;
+import org.apache.kylin.metadata.measure.LongMutable;
+import org.junit.Test;
+
+public class SimpleGridTableTest {
+
+    @Test
+    public void testBasics() throws IOException {
+        GTInfo info = UnitTestSupport.basicInfo();
+        GTSimpleMemStore store = new GTSimpleMemStore(info);
+        GridTable table = new GridTable(info, store);
+
+        GTBuilder builder = rebuild(table);
+        IGTScanner scanner = scan(table);
+        assertEquals(builder.getWrittenRowBlockCount(), scanner.getScannedRowBlockCount());
+        assertEquals(builder.getWrittenRowCount(), scanner.getScannedRowCount());
+    }
+
+    @Test
+    public void testAdvanced() throws IOException {
+        GTInfo info = UnitTestSupport.advancedInfo();
+        GTSimpleMemStore store = new GTSimpleMemStore(info);
+        GridTable table = new GridTable(info, store);
+
+        GTBuilder builder = rebuild(table);
+        IGTScanner scanner = scan(table);
+        assertEquals(builder.getWrittenRowBlockCount(), scanner.getScannedRowBlockCount());
+        assertEquals(builder.getWrittenRowCount(), scanner.getScannedRowCount());
+    }
+
+    @Test
+    public void testAggregate() throws IOException {
+        GTInfo info = UnitTestSupport.advancedInfo();
+        GTSimpleMemStore store = new GTSimpleMemStore(info);
+        GridTable table = new GridTable(info, store);
+
+        GTBuilder builder = rebuild(table);
+        IGTScanner scanner = scanAndAggregate(table);
+        assertEquals(builder.getWrittenRowBlockCount(), scanner.getScannedRowBlockCount());
+        assertEquals(builder.getWrittenRowCount(), scanner.getScannedRowCount());
+    }
+
+    @Test
+    public void testAppend() throws IOException {
+        GTInfo info = UnitTestSupport.advancedInfo();
+        GTSimpleMemStore store = new GTSimpleMemStore(info);
+        GridTable table = new GridTable(info, store);
+
+        rebuildViaAppend(table);
+        IGTScanner scanner = scan(table);
+        assertEquals(3, scanner.getScannedRowBlockCount());
+        assertEquals(10, scanner.getScannedRowCount());
+    }
+
+    private IGTScanner scan(GridTable table) throws IOException {
+        GTScanRequest req = new GTScanRequest(table.getInfo());
+        IGTScanner scanner = table.scan(req);
+        for (GTRecord r : scanner) {
+            Object[] v = r.getValues();
+            assertTrue(((String) v[0]).startsWith("2015-"));
+            assertTrue(((String) v[2]).equals("Food"));
+            assertTrue(((LongMutable) v[3]).get() == 10);
+            assertTrue(((BigDecimal) v[4]).doubleValue() == 10.5);
+            System.out.println(r);
+        }
+        scanner.close();
+        System.out.println("Scanned Row Block Count: " + scanner.getScannedRowBlockCount());
+        System.out.println("Scanned Row Count: " + scanner.getScannedRowCount());
+        return scanner;
+    }
+
+    private IGTScanner scanAndAggregate(GridTable table) throws IOException {
+        GTScanRequest req = new GTScanRequest(table.getInfo(), null, setOf(0, 2), setOf(3, 4), new String[] { "count", "sum" }, null);
+        IGTScanner scanner = table.scan(req);
+        int i = 0;
+        for (GTRecord r : scanner) {
+            Object[] v = r.getValues();
+            switch (i) {
+            case 0:
+                assertTrue(((LongMutable) v[3]).get() == 20);
+                assertTrue(((BigDecimal) v[4]).doubleValue() == 21.0);
+                break;
+            case 1:
+                assertTrue(((LongMutable) v[3]).get() == 30);
+                assertTrue(((BigDecimal) v[4]).doubleValue() == 31.5);
+                break;
+            case 2:
+                assertTrue(((LongMutable) v[3]).get() == 40);
+                assertTrue(((BigDecimal) v[4]).doubleValue() == 42.0);
+                break;
+            case 3:
+                assertTrue(((LongMutable) v[3]).get() == 10);
+                assertTrue(((BigDecimal) v[4]).doubleValue() == 10.5);
+                break;
+            default:
+                fail();
+            }
+            i++;
+            System.out.println(r);
+        }
+        scanner.close();
+        System.out.println("Scanned Row Block Count: " + scanner.getScannedRowBlockCount());
+        System.out.println("Scanned Row Count: " + scanner.getScannedRowCount());
+        return scanner;
+    }
+
+    static GTBuilder rebuild(GridTable table) throws IOException {
+        GTBuilder builder = table.rebuild();
+        for (GTRecord rec : UnitTestSupport.mockupData(table.getInfo(), 10)) {
+            builder.write(rec);
+        }
+        builder.close();
+
+        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
+        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
+        return builder;
+    }
+    
+    static void rebuildViaAppend(GridTable table) throws IOException {
+        List<GTRecord> data = UnitTestSupport.mockupData(table.getInfo(), 10);
+        GTBuilder builder;
+        int i = 0;
+
+        builder = table.append();
+        builder.write(data.get(i++));
+        builder.write(data.get(i++));
+        builder.write(data.get(i++));
+        builder.write(data.get(i++));
+        builder.close();
+        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
+        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
+
+        builder = table.append();
+        builder.write(data.get(i++));
+        builder.write(data.get(i++));
+        builder.write(data.get(i++));
+        builder.close();
+        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
+        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
+
+        builder = table.append();
+        builder.write(data.get(i++));
+        builder.write(data.get(i++));
+        builder.close();
+        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
+        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
+
+        builder = table.append();
+        builder.write(data.get(i++));
+        builder.close();
+        System.out.println("Written Row Block Count: " + builder.getWrittenRowBlockCount());
+        System.out.println("Written Row Count: " + builder.getWrittenRowCount());
+    }
+
+    private static ImmutableBitSet setOf(int... values) {
+        BitSet set = new BitSet();
+        for (int i : values)
+            set.set(i);
+        return new ImmutableBitSet(set);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
new file mode 100644
index 0000000..f5c9645
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
@@ -0,0 +1,188 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements. See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.gridtable;
+
+import static org.junit.Assert.*;
+import it.uniroma3.mat.extendedset.intset.ConciseSet;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTInvertedIndex;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTRowBlock;
+import org.apache.kylin.gridtable.UnitTestSupport;
+import org.apache.kylin.metadata.filter.ColumnTupleFilter;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.ConstantTupleFilter;
+import org.apache.kylin.metadata.filter.LogicalTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.measure.LongMutable;
+import org.apache.kylin.metadata.measure.serializer.StringSerializer;
+import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class SimpleInvertedIndexTest {
+
+    GTInfo info;
+    GTInvertedIndex index;
+    ArrayList<CompareTupleFilter> basicFilters = Lists.newArrayList();
+    ArrayList<ConciseSet> basicResults = Lists.newArrayList();
+
+    public SimpleInvertedIndexTest() {
+        
+        info = UnitTestSupport.advancedInfo();
+        TblColRef colA = info.colRef(0);
+        
+        // block i contains value "i", the last is NULL
+        index = new GTInvertedIndex(info);
+        GTRowBlock mockBlock = GTRowBlock.allocate(info);
+        GTRowBlock.Writer writer = mockBlock.getWriter();
+        GTRecord record = new GTRecord(info);
+        for (int i = 0; i < 10; i++) {
+            record.setValues(i < 9 ? "" + i : null, "", "", new LongMutable(0), new BigDecimal(0));
+            for (int j = 0; j < info.getRowBlockSize(); j++) {
+                writer.append(record);
+            }
+            writer.readyForFlush();
+            index.add(mockBlock);
+            
+            writer.clearForNext();
+        }
+        
+        basicFilters.add(compare(colA, FilterOperatorEnum.ISNULL));
+        basicResults.add(set(9));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.ISNOTNULL));
+        basicResults.add(set(0, 1, 2, 3, 4, 5, 6, 7, 8, 9));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.EQ, 0));
+        basicResults.add(set(0));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.NEQ, 0));
+        basicResults.add(set(0, 1, 2, 3, 4, 5, 6, 7, 8, 9));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.IN, 0, 5));
+        basicResults.add(set(0, 5));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.NOTIN, 0, 5));
+        basicResults.add(set(0, 1, 2, 3, 4, 5, 6, 7, 8, 9));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.LT, 3));
+        basicResults.add(set(0, 1, 2));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.LTE, 3));
+        basicResults.add(set(0, 1, 2, 3));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.GT, 3));
+        basicResults.add(set(4, 5, 6, 7, 8));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.GTE, 3));
+        basicResults.add(set(3, 4, 5, 6, 7, 8));
+    }
+
+    @Test
+    public void testBasics() {
+        for (int i = 0; i < basicFilters.size(); i++) {
+            assertEquals(basicResults.get(i), index.filter(basicFilters.get(i)));
+        }
+    }
+
+    @Test
+    public void testLogicalAnd() {
+        for (int i = 0; i < basicFilters.size(); i++) {
+            for (int j = 0; j < basicFilters.size(); j++) {
+                LogicalTupleFilter f = logical(FilterOperatorEnum.AND, basicFilters.get(i), basicFilters.get(j));
+                ConciseSet r = basicResults.get(i).clone();
+                r.retainAll(basicResults.get(j));
+                assertEquals(r, index.filter(f));
+            }
+        }
+    }
+
+    @Test
+    public void testLogicalOr() {
+        for (int i = 0; i < basicFilters.size(); i++) {
+            for (int j = 0; j < basicFilters.size(); j++) {
+                LogicalTupleFilter f = logical(FilterOperatorEnum.OR, basicFilters.get(i), basicFilters.get(j));
+                ConciseSet r = basicResults.get(i).clone();
+                r.addAll(basicResults.get(j));
+                assertEquals(r, index.filter(f));
+            }
+        }
+    }
+
+    @Test
+    public void testNotEvaluable() {
+        ConciseSet all = set(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
+        
+        CompareTupleFilter notEvaluable = compare(info.colRef(1), FilterOperatorEnum.EQ, 0);
+        assertEquals(all, index.filter(notEvaluable));
+
+        LogicalTupleFilter or = logical(FilterOperatorEnum.OR, basicFilters.get(0), notEvaluable);
+        assertEquals(all, index.filter(or));
+
+        LogicalTupleFilter and = logical(FilterOperatorEnum.AND, basicFilters.get(0), notEvaluable);
+        assertEquals(basicResults.get(0), index.filter(and));
+    }
+
+    public static CompareTupleFilter compare(TblColRef col, TupleFilter.FilterOperatorEnum op, int... ids) {
+        CompareTupleFilter filter = new CompareTupleFilter(op);
+        filter.addChild(columnFilter(col));
+        for (int i : ids) {
+            filter.addChild(constFilter(i));
+        }
+        return filter;
+    }
+
+    public static LogicalTupleFilter logical(TupleFilter.FilterOperatorEnum op, TupleFilter... filters) {
+        LogicalTupleFilter filter = new LogicalTupleFilter(op);
+        for (TupleFilter f : filters)
+            filter.addChild(f);
+        return filter;
+    }
+
+    public static ColumnTupleFilter columnFilter(TblColRef col) {
+        return new ColumnTupleFilter(col);
+    }
+
+    public static ConstantTupleFilter constFilter(int id) {
+        byte[] space = new byte[10];
+        ByteBuffer buf = ByteBuffer.wrap(space);
+        StringSerializer stringSerializer = new StringSerializer(DataType.getInstance("string"));
+        stringSerializer.serialize("" + id, buf);
+        ByteArray data = new ByteArray(buf.array(), buf.arrayOffset(), buf.position());
+        return new ConstantTupleFilter(data);
+    }
+
+    public static ConciseSet set(int... ints) {
+        ConciseSet set = new ConciseSet();
+        for (int i : ints)
+            set.add(i);
+        return set;
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/pom.xml
----------------------------------------------------------------------
diff --git a/core-job/pom.xml b/core-job/pom.xml
index b51c1cf..b619f43 100644
--- a/core-job/pom.xml
+++ b/core-job/pom.xml
@@ -41,8 +41,20 @@
             <version>${project.parent.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+        </dependency>
+
         <!-- Env & Test -->
         <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java b/core-job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java
new file mode 100644
index 0000000..6b814ef
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+
+public class BuildEngineFactory {
+    
+    private static IBatchCubingEngine defaultBatchEngine;
+    
+    public static IBatchCubingEngine defaultBatchEngine() {
+        if (defaultBatchEngine == null) {
+            KylinConfig conf = KylinConfig.getInstanceFromEnv();
+            if (conf.isCubingInMem()) {
+                defaultBatchEngine = (IBatchCubingEngine) ClassUtil.newInstance("org.apache.kylin.engine.mr.MRBatchCubingEngine2");
+            } else {
+                defaultBatchEngine = (IBatchCubingEngine) ClassUtil.newInstance("org.apache.kylin.engine.mr.MRBatchCubingEngine");
+            }
+        }
+        return defaultBatchEngine;
+    }
+    
+    /** Build a new cube segment, typically its time range appends to the end of current cube. */
+    public static DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter) {
+        return defaultBatchEngine().createBatchCubingJob(newSegment, submitter);
+    }
+    
+    /** Merge multiple small segments into a big one. */
+    public static DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter) {
+        return defaultBatchEngine().createBatchMergeJob(mergeSegment, submitter);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java b/core-job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
new file mode 100644
index 0000000..904f557
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+
+public interface IBatchCubingEngine {
+
+    /** Build a new cube segment, typically its time range appends to the end of current cube. */
+    public DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter);
+    
+    /** Merge multiple small segments into a big one. */
+    public DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter);
+    
+    public Class<?> getSourceInterface();
+    
+    public Class<?> getStorageInterface();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/engine/IStreamingCubingEngine.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/engine/IStreamingCubingEngine.java b/core-job/src/main/java/org/apache/kylin/engine/IStreamingCubingEngine.java
new file mode 100644
index 0000000..0359ce9
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/engine/IStreamingCubingEngine.java
@@ -0,0 +1,8 @@
+package org.apache.kylin.engine;
+
+import org.apache.kylin.cube.CubeSegment;
+
+public interface IStreamingCubingEngine {
+
+    public Runnable createStreamingCubingBuilder(CubeSegment seg);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/JobInstance.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/JobInstance.java b/core-job/src/main/java/org/apache/kylin/job/JobInstance.java
new file mode 100644
index 0000000..e7f5772
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/JobInstance.java
@@ -0,0 +1,504 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonBackReference;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonManagedReference;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Lists;
+import org.apache.kylin.common.persistence.RootPersistentEntity;
+import org.apache.kylin.cube.model.CubeBuildTypeEnum;
+import org.apache.kylin.job.constant.JobStatusEnum;
+import org.apache.kylin.job.constant.JobStepCmdTypeEnum;
+import org.apache.kylin.job.constant.JobStepStatusEnum;
+import org.apache.kylin.job.engine.JobEngineConfig;
+
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class JobInstance extends RootPersistentEntity implements Comparable<JobInstance> {
+
+    public static final String JOB_WORKING_DIR_PREFIX = "kylin-";
+
+    public static final String YARN_APP_ID = "yarn_application_id";
+    public static final String YARN_APP_URL = "yarn_application_tracking_url";
+    public static final String MR_JOB_ID = "mr_job_id";
+    public static final String HDFS_BYTES_WRITTEN = "hdfs_bytes_written";
+    public static final String SOURCE_RECORDS_COUNT = "source_records_count";
+    public static final String SOURCE_RECORDS_SIZE = "source_records_size";
+
+    public static String getStepIdentity(JobInstance jobInstance, JobStep jobStep) {
+        return jobInstance.getRelatedCube() + "." + jobInstance.getUuid() + "." + jobStep.getSequenceID();
+    }
+
+    public static String getJobIdentity(JobInstance jobInstance) {
+        return jobInstance.getRelatedCube() + "." + jobInstance.getUuid();
+    }
+
+    public static String getJobWorkingDir(JobInstance jobInstance, JobEngineConfig engineConfig) {
+        return getJobWorkingDir(jobInstance.getUuid(), engineConfig.getHdfsWorkingDirectory());
+    }
+
+    public static String getJobWorkingDir(String jobUuid, String hdfsWorkdingDir) {
+        if (jobUuid == null || jobUuid.equals("")) {
+            throw new IllegalArgumentException("jobUuid can't be null or empty");
+        }
+        return hdfsWorkdingDir + "/" + JOB_WORKING_DIR_PREFIX + jobUuid;
+    }
+
+    @JsonProperty("name")
+    private String name;
+
+    @JsonProperty("type")
+    private CubeBuildTypeEnum type; // java implementation
+    @JsonProperty("duration")
+    private long duration;
+    @JsonProperty("related_cube")
+    private String relatedCube;
+    @JsonProperty("related_segment")
+    private String relatedSegment;
+    @JsonProperty("exec_start_time")
+    private long execStartTime;
+    @JsonProperty("exec_end_time")
+    private long execEndTime;
+    @JsonProperty("mr_waiting")
+    private long mrWaiting = 0;
+    @JsonManagedReference
+    @JsonProperty("steps")
+    private List<JobStep> steps;
+    @JsonProperty("submitter")
+    private String submitter;
+    @JsonProperty("job_status")
+    private JobStatusEnum status;
+
+    public JobStep getRunningStep() {
+        for (JobStep step : this.getSteps()) {
+            if (step.getStatus().equals(JobStepStatusEnum.RUNNING) || step.getStatus().equals(JobStepStatusEnum.WAITING)) {
+                return step;
+            }
+        }
+
+        return null;
+    }
+
+    @JsonProperty("progress")
+    public double getProgress() {
+        int completedStepCount = 0;
+        for (JobStep step : this.getSteps()) {
+            if (step.getStatus().equals(JobStepStatusEnum.FINISHED)) {
+                completedStepCount++;
+            }
+        }
+
+        return 100.0 * completedStepCount / steps.size();
+    }
+
+    public JobStatusEnum getStatus() {
+        return this.status;
+    }
+
+    public void setStatus(JobStatusEnum status) {
+        this.status = status;
+    }
+
+//    @JsonProperty("job_status")
+//    public JobStatusEnum getStatus() {
+//
+//        // JobStatusEnum finalJobStatus;
+//        int compositResult = 0;
+//
+//        // if steps status are all NEW, then job status is NEW
+//        // if steps status are all FINISHED, then job status is FINISHED
+//        // if steps status are all PENDING, then job status is PENDING
+//        // if steps status are FINISHED and PENDING, the job status is PENDING
+//        // if one of steps status is RUNNING, then job status is RUNNING
+//        // if one of steps status is ERROR, then job status is ERROR
+//        // if one of steps status is KILLED, then job status is KILLED
+//        // default status is RUNNING
+//
+//        System.out.println(this.getName());
+//
+//        for (JobStep step : this.getSteps()) {
+//            //System.out.println("step: " + step.getSequenceID() + "'s status:" + step.getStatus());
+//            compositResult = compositResult | step.getStatus().getCode();
+//        }
+//
+//        System.out.println();
+//
+//        if (compositResult == JobStatusEnum.FINISHED.getCode()) {
+//            return JobStatusEnum.FINISHED;
+//        } else if (compositResult == JobStatusEnum.NEW.getCode()) {
+//            return JobStatusEnum.NEW;
+//        } else if (compositResult == JobStatusEnum.PENDING.getCode()) {
+//            return JobStatusEnum.PENDING;
+//        } else if (compositResult == (JobStatusEnum.FINISHED.getCode() | JobStatusEnum.PENDING.getCode())) {
+//            return JobStatusEnum.PENDING;
+//        } else if ((compositResult & JobStatusEnum.ERROR.getCode()) == JobStatusEnum.ERROR.getCode()) {
+//            return JobStatusEnum.ERROR;
+//        } else if ((compositResult & JobStatusEnum.DISCARDED.getCode()) == JobStatusEnum.DISCARDED.getCode()) {
+//            return JobStatusEnum.DISCARDED;
+//        } else if ((compositResult & JobStatusEnum.RUNNING.getCode()) == JobStatusEnum.RUNNING.getCode()) {
+//            return JobStatusEnum.RUNNING;
+//        }
+//
+//        return JobStatusEnum.RUNNING;
+//    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public CubeBuildTypeEnum getType() {
+        return type;
+    }
+
+    public void setType(CubeBuildTypeEnum type) {
+        this.type = type;
+    }
+
+    public long getDuration() {
+        return duration;
+    }
+
+    public void setDuration(long duration) {
+        this.duration = duration;
+    }
+
+    public String getRelatedCube() {
+        return relatedCube;
+    }
+
+    public void setRelatedCube(String relatedCube) {
+        this.relatedCube = relatedCube;
+    }
+
+    public String getRelatedSegment() {
+        return relatedSegment;
+    }
+
+    public void setRelatedSegment(String relatedSegment) {
+        this.relatedSegment = relatedSegment;
+    }
+
+    /**
+     * @return the execStartTime
+     */
+    public long getExecStartTime() {
+        return execStartTime;
+    }
+
+    /**
+     * @param execStartTime the execStartTime to set
+     */
+    public void setExecStartTime(long execStartTime) {
+        this.execStartTime = execStartTime;
+    }
+
+    /**
+     * @return the execEndTime
+     */
+    public long getExecEndTime() {
+        return execEndTime;
+    }
+
+    /**
+     * @param execEndTime the execEndTime to set
+     */
+    public void setExecEndTime(long execEndTime) {
+        this.execEndTime = execEndTime;
+    }
+
+    public long getMrWaiting() {
+        return this.mrWaiting;
+    }
+
+    public void setMrWaiting(long mrWaiting) {
+        this.mrWaiting = mrWaiting;
+    }
+
+    public List<JobStep> getSteps() {
+        if (steps == null) {
+            steps = Lists.newArrayList();
+        }
+        return steps;
+    }
+
+    public void clearSteps() {
+        getSteps().clear();
+    }
+
+    public void addSteps(Collection<JobStep> steps) {
+        this.getSteps().addAll(steps);
+    }
+
+    public void addStep(JobStep step) {
+        getSteps().add(step);
+    }
+
+    public void addStep(int index, JobStep step) {
+        getSteps().add(index, step);
+    }
+
+    public JobStep findStep(String stepName) {
+        for (JobStep step : getSteps()) {
+            if (stepName.equals(step.getName())) {
+                return step;
+            }
+        }
+        return null;
+    }
+
+        
+    public String getSubmitter() {
+        return submitter;
+    }
+
+    public void setSubmitter(String submitter) {
+        this.submitter = submitter;
+    }
+
+
+
+
+    @JsonIgnoreProperties(ignoreUnknown = true)
+    public static class JobStep implements Comparable<JobStep> {
+
+        @JsonBackReference
+        private JobInstance jobInstance;
+        
+        @JsonProperty("id")
+        private String id;
+
+        @JsonProperty("name")
+        private String name;
+
+        @JsonProperty("sequence_id")
+        private int sequenceID;
+
+        @JsonProperty("exec_cmd")
+        private String execCmd;
+
+        @JsonProperty("interrupt_cmd")
+        private String InterruptCmd;
+
+        @JsonProperty("exec_start_time")
+        private long execStartTime;
+        @JsonProperty("exec_end_time")
+        private long execEndTime;
+        @JsonProperty("exec_wait_time")
+        private long execWaitTime;
+
+        @JsonProperty("step_status")
+        private JobStepStatusEnum status;
+
+        @JsonProperty("cmd_type")
+        private JobStepCmdTypeEnum cmdType = JobStepCmdTypeEnum.SHELL_CMD_HADOOP;
+
+        @JsonProperty("info")
+        private ConcurrentHashMap<String, String> info = new ConcurrentHashMap<String, String>();
+
+        @JsonProperty("run_async")
+        private boolean runAsync = false;
+
+        private ConcurrentHashMap<String, String> getInfo() {
+            return info;
+        }
+
+        public void putInfo(String key, String value) {
+            getInfo().put(key, value);
+        }
+
+        public String getInfo(String key) {
+            return getInfo().get(key);
+        }
+
+        public void clearInfo() {
+            getInfo().clear();
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        public int getSequenceID() {
+            return sequenceID;
+        }
+
+        public void setSequenceID(int sequenceID) {
+            this.sequenceID = sequenceID;
+        }
+
+        public String getExecCmd() {
+            return execCmd;
+        }
+
+        public void setExecCmd(String execCmd) {
+            this.execCmd = execCmd;
+        }
+
+        public JobStepStatusEnum getStatus() {
+            return status;
+        }
+
+        public void setStatus(JobStepStatusEnum status) {
+            this.status = status;
+        }
+        
+        
+
+        public String getId() {
+            return id;
+        }
+
+        public void setId(String id) {
+            this.id = id;
+        }
+
+        /**
+         * @return the execStartTime
+         */
+        public long getExecStartTime() {
+            return execStartTime;
+        }
+
+        /**
+         * @param execStartTime the execStartTime to set
+         */
+        public void setExecStartTime(long execStartTime) {
+            this.execStartTime = execStartTime;
+        }
+
+        /**
+         * @return the execEndTime
+         */
+        public long getExecEndTime() {
+            return execEndTime;
+        }
+
+        /**
+         * @param execEndTime the execEndTime to set
+         */
+        public void setExecEndTime(long execEndTime) {
+            this.execEndTime = execEndTime;
+        }
+
+        public long getExecWaitTime() {
+            return execWaitTime;
+        }
+
+        public void setExecWaitTime(long execWaitTime) {
+            this.execWaitTime = execWaitTime;
+        }
+
+        public String getInterruptCmd() {
+            return InterruptCmd;
+        }
+
+        public void setInterruptCmd(String interruptCmd) {
+            InterruptCmd = interruptCmd;
+        }
+
+        public JobStepCmdTypeEnum getCmdType() {
+            return cmdType;
+        }
+
+        public void setCmdType(JobStepCmdTypeEnum cmdType) {
+            this.cmdType = cmdType;
+        }
+
+        /**
+         * @return the runAsync
+         */
+        public boolean isRunAsync() {
+            return runAsync;
+        }
+
+        /**
+         * @param runAsync the runAsync to set
+         */
+        public void setRunAsync(boolean runAsync) {
+            this.runAsync = runAsync;
+        }
+
+        /**
+         * @return the jobInstance
+         */
+        public JobInstance getJobInstance() {
+            return jobInstance;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + ((name == null) ? 0 : name.hashCode());
+            result = prime * result + sequenceID;
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+            if (obj == null)
+                return false;
+            if (getClass() != obj.getClass())
+                return false;
+            JobStep other = (JobStep) obj;
+            if (name == null) {
+                if (other.name != null)
+                    return false;
+            } else if (!name.equals(other.name))
+                return false;
+            if (sequenceID != other.sequenceID)
+                return false;
+            return true;
+        }
+
+        @Override
+        public int compareTo(JobStep o) {
+            if (this.sequenceID < o.sequenceID) {
+                return -1;
+            } else if (this.sequenceID > o.sequenceID) {
+                return 1;
+            } else {
+                return 0;
+            }
+        }
+    }
+
+    @Override
+    public int compareTo(JobInstance o) {
+        return o.lastModified<this.lastModified?-1:o.lastModified>this.lastModified?1:0;
+    }
+
+}



[28/28] incubator-kylin git commit: KYLIN-875 Fix test cases broken by class movement

Posted by li...@apache.org.
KYLIN-875 Fix test cases broken by class movement


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

Branch: refs/heads/0.8
Commit: 993e0645eda8d5232155e5004e01c6567e9fa32e
Parents: 94676a1
Author: Yang Li <li...@apache.org>
Authored: Fri Jul 24 07:17:38 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Fri Jul 24 07:17:38 2015 +0800

----------------------------------------------------------------------
 core-dictionary/src/test/resources/dict/DW_SITES | Bin 5020 -> 0 bytes
 .../engine/mr/steps/NDCuboidMapperTest.java      |   6 +++---
 .../test/resources/data/6d_cuboid/part-r-00000   | Bin 0 -> 3279797 bytes
 .../test/resources/data/base_cuboid/part-r-00000 | Bin 0 -> 870735 bytes
 engine-mr/src/test/resources/data/dict/DW_SITES  | Bin 0 -> 5020 bytes
 .../test/resources/data/6d_cuboid/part-r-00000   | Bin 3279797 -> 0 bytes
 .../test/resources/data/base_cuboid/part-r-00000 | Bin 870735 -> 0 bytes
 .../test/resources/partition_list/_partition.lst | Bin 326 -> 0 bytes
 .../test/resources/partition_list/part-r-00000   | Bin 15004 -> 0 bytes
 .../test/resources/data/6d_cuboid/part-r-00000   | Bin 0 -> 3279797 bytes
 .../test/resources/data/base_cuboid/part-r-00000 | Bin 0 -> 870735 bytes
 .../test/resources/partition_list/_partition.lst | Bin 0 -> 326 bytes
 .../test/resources/partition_list/part-r-00000   | Bin 0 -> 15004 bytes
 13 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/core-dictionary/src/test/resources/dict/DW_SITES
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/resources/dict/DW_SITES b/core-dictionary/src/test/resources/dict/DW_SITES
deleted file mode 100644
index 6f6af46..0000000
Binary files a/core-dictionary/src/test/resources/dict/DW_SITES and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidMapperTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidMapperTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidMapperTest.java
index 9f50cc6..f762ed4 100644
--- a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidMapperTest.java
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NDCuboidMapperTest.java
@@ -55,8 +55,8 @@ public class NDCuboidMapperTest extends LocalFileMetadataTestCase {
         createTestMetadata();
 
         // hack for distributed cache
-        FileUtils.deleteDirectory(new File("../job/meta"));
-        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), new File("../job/meta"));
+        FileUtils.deleteDirectory(new File("./meta"));
+        FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), new File("./meta"));
 
         NDCuboidMapper mapper = new NDCuboidMapper();
         CuboidReducer reducer = new CuboidReducer();
@@ -66,7 +66,7 @@ public class NDCuboidMapperTest extends LocalFileMetadataTestCase {
     @After
     public void after() throws Exception {
         cleanupTestMetadata();
-        FileUtils.deleteDirectory(new File("../job/meta"));
+        FileUtils.deleteDirectory(new File("./meta"));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/engine-mr/src/test/resources/data/6d_cuboid/part-r-00000
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/resources/data/6d_cuboid/part-r-00000 b/engine-mr/src/test/resources/data/6d_cuboid/part-r-00000
new file mode 100644
index 0000000..b41ba18
Binary files /dev/null and b/engine-mr/src/test/resources/data/6d_cuboid/part-r-00000 differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/engine-mr/src/test/resources/data/base_cuboid/part-r-00000
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/resources/data/base_cuboid/part-r-00000 b/engine-mr/src/test/resources/data/base_cuboid/part-r-00000
new file mode 100644
index 0000000..1a011a4
Binary files /dev/null and b/engine-mr/src/test/resources/data/base_cuboid/part-r-00000 differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/engine-mr/src/test/resources/data/dict/DW_SITES
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/resources/data/dict/DW_SITES b/engine-mr/src/test/resources/data/dict/DW_SITES
new file mode 100644
index 0000000..6f6af46
Binary files /dev/null and b/engine-mr/src/test/resources/data/dict/DW_SITES differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/job/src/test/resources/data/6d_cuboid/part-r-00000
----------------------------------------------------------------------
diff --git a/job/src/test/resources/data/6d_cuboid/part-r-00000 b/job/src/test/resources/data/6d_cuboid/part-r-00000
deleted file mode 100644
index b41ba18..0000000
Binary files a/job/src/test/resources/data/6d_cuboid/part-r-00000 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/job/src/test/resources/data/base_cuboid/part-r-00000
----------------------------------------------------------------------
diff --git a/job/src/test/resources/data/base_cuboid/part-r-00000 b/job/src/test/resources/data/base_cuboid/part-r-00000
deleted file mode 100644
index 1a011a4..0000000
Binary files a/job/src/test/resources/data/base_cuboid/part-r-00000 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/job/src/test/resources/partition_list/_partition.lst
----------------------------------------------------------------------
diff --git a/job/src/test/resources/partition_list/_partition.lst b/job/src/test/resources/partition_list/_partition.lst
deleted file mode 100644
index 64147d4..0000000
Binary files a/job/src/test/resources/partition_list/_partition.lst and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/job/src/test/resources/partition_list/part-r-00000
----------------------------------------------------------------------
diff --git a/job/src/test/resources/partition_list/part-r-00000 b/job/src/test/resources/partition_list/part-r-00000
deleted file mode 100644
index 07f1110..0000000
Binary files a/job/src/test/resources/partition_list/part-r-00000 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/storage-hbase/src/test/resources/data/6d_cuboid/part-r-00000
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/resources/data/6d_cuboid/part-r-00000 b/storage-hbase/src/test/resources/data/6d_cuboid/part-r-00000
new file mode 100644
index 0000000..b41ba18
Binary files /dev/null and b/storage-hbase/src/test/resources/data/6d_cuboid/part-r-00000 differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/storage-hbase/src/test/resources/data/base_cuboid/part-r-00000
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/resources/data/base_cuboid/part-r-00000 b/storage-hbase/src/test/resources/data/base_cuboid/part-r-00000
new file mode 100644
index 0000000..1a011a4
Binary files /dev/null and b/storage-hbase/src/test/resources/data/base_cuboid/part-r-00000 differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/storage-hbase/src/test/resources/partition_list/_partition.lst
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/resources/partition_list/_partition.lst b/storage-hbase/src/test/resources/partition_list/_partition.lst
new file mode 100644
index 0000000..64147d4
Binary files /dev/null and b/storage-hbase/src/test/resources/partition_list/_partition.lst differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/993e0645/storage-hbase/src/test/resources/partition_list/part-r-00000
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/resources/partition_list/part-r-00000 b/storage-hbase/src/test/resources/partition_list/part-r-00000
new file mode 100644
index 0000000..07f1110
Binary files /dev/null and b/storage-hbase/src/test/resources/partition_list/part-r-00000 differ


[15/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/IMROutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/IMROutput.java b/job/src/main/java/org/apache/kylin/engine/mr/IMROutput.java
deleted file mode 100644
index bc6ee1f..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/IMROutput.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine.mr;
-
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-
-public interface IMROutput {
-
-    /** Return a helper to participate in batch cubing job flow. */
-    public IMRBatchCubingOutputSide getBatchCubingOutputSide(CubeSegment seg);
-    
-    /**
-     * Participate the batch cubing flow as the output side. Responsible for saving
-     * the cuboid output to storage (Phase 3).
-     * 
-     * - Phase 1: Create Flat Table
-     * - Phase 2: Build Dictionary
-     * - Phase 3: Build Cube
-     * - Phase 4: Update Metadata & Cleanup
-     */
-    public interface IMRBatchCubingOutputSide {
-        
-        /**
-         * Add step that saves cuboid output from HDFS to storage.
-         * 
-         * The cuboid output is a directory of sequence files, where key takes format "CUBOID,D1,D2,..,Dn", 
-         * value takes format "M1,M2,..,Mm". CUBOID is 8 bytes cuboid ID; Dx is dimension value with
-         * dictionary encoding; Mx is measure value serialization form.
-         */
-        public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow, String cuboidRootPath);
-        
-        /** Add step that does any necessary clean up. */
-        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow);
-    }
-    
-    /** Return a helper to participate in batch merge job flow. */
-    public IMRBatchMergeOutputSide getBatchMergeOutputSide(CubeSegment seg);
-    
-    /**
-     * Participate the batch cubing flow as the output side. Responsible for saving
-     * the cuboid output to storage (Phase 2).
-     * 
-     * - Phase 1: Merge Dictionary
-     * - Phase 2: Merge Cube
-     * - Phase 3: Update Metadata & Cleanup
-     */
-    public interface IMRBatchMergeOutputSide {
-        
-        /**
-         * Add step that saves cuboid output from HDFS to storage.
-         * 
-         * The cuboid output is a directory of sequence files, where key takes format "CUBOID,D1,D2,..,Dn", 
-         * value takes format "M1,M2,..,Mm". CUBOID is 8 bytes cuboid ID; Dx is dimension value with
-         * dictionary encoding; Mx is measure value serialization form.
-         */
-        public void addStepPhase2_BuildCube(DefaultChainedExecutable jobFlow, String cuboidRootPath);
-        
-        /** Add step that does any necessary clean up. */
-        public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java b/job/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
deleted file mode 100644
index 974e2fc..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
+++ /dev/null
@@ -1,88 +0,0 @@
-package org.apache.kylin.engine.mr;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-
-public interface IMROutput2 {
-
-    /** Return a helper to participate in batch cubing job flow. */
-    public IMRBatchCubingOutputSide2 getBatchCubingOutputSide(CubeSegment seg);
-
-    /**
-     * Participate the batch cubing flow as the output side.
-     * 
-     * - Phase 1: Create Flat Table
-     * - Phase 2: Build Dictionary
-     * - Phase 3: Build Cube (with StorageOutputFormat)
-     * - Phase 4: Update Metadata & Cleanup
-     */
-    public interface IMRBatchCubingOutputSide2 {
-
-        public IMRStorageOutputFormat getStorageOutputFormat();
-
-        /** Add step that executes after build dictionary and before build cube. */
-        public void addStepPhase2_BuildDictionary(DefaultChainedExecutable jobFlow);
-
-        /** Add step that executes after build cube. */
-        public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow);
-
-        /** Add step that does any necessary clean up. */
-        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow);
-    }
-
-    public IMRBatchMergeInputSide2 getBatchMergeInputSide(CubeSegment seg);
-
-    public interface IMRBatchMergeInputSide2 {
-        public IMRStorageInputFormat getStorageInputFormat();
-    }
-
-    @SuppressWarnings("rawtypes")
-    public interface IMRStorageInputFormat {
-        
-        public void configureInput(Class<? extends Mapper> mapper, Class<? extends WritableComparable> outputKeyClass, Class<? extends Writable> outputValueClass, Job job) throws IOException;
-        
-        public CubeSegment findSourceSegment(Mapper.Context context, CubeInstance cubeInstance) throws IOException;
-        
-        public Pair<ByteArrayWritable, Object[]> parseMapperInput(Object inKey, Object inValue);
-    }
-
-    /** Return a helper to participate in batch merge job flow. */
-    public IMRBatchMergeOutputSide2 getBatchMergeOutputSide(CubeSegment seg);
-
-    /**
-     * Participate the batch merge flow as the output side.
-     * 
-     * - Phase 1: Merge Dictionary
-     * - Phase 2: Merge Cube (with StorageInputFormat & StorageOutputFormat)
-     * - Phase 3: Update Metadata & Cleanup
-     */
-    public interface IMRBatchMergeOutputSide2 {
-
-        public IMRStorageOutputFormat getStorageOutputFormat();
-
-        /** Add step that executes after merge dictionary and before merge cube. */
-        public void addStepPhase1_MergeDictionary(DefaultChainedExecutable jobFlow);
-
-        /** Add step that executes after merge cube. */
-        public void addStepPhase2_BuildCube(DefaultChainedExecutable jobFlow);
-
-        /** Add step that does any necessary clean up. */
-        public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow);
-    }
-
-    @SuppressWarnings("rawtypes")
-    public interface IMRStorageOutputFormat {
-        public void configureOutput(Class<? extends Reducer> reducer, String jobFlowId, Job job) throws IOException;
-        
-        public void doReducerOutput(ByteArrayWritable key, Object[] value, Reducer.Context context) throws IOException, InterruptedException;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java b/job/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
deleted file mode 100644
index 3217e4b..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
+++ /dev/null
@@ -1,168 +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;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeSegment;
-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.job.common.HadoopShellExecutable;
-import org.apache.kylin.job.common.MapReduceExecutable;
-import org.apache.kylin.job.constant.ExecutableConstants;
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.hadoop.cube.FactDistinctColumnsJob;
-import org.apache.kylin.job.hadoop.dict.CreateDictionaryJob;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Hold reusable steps for builders.
- */
-public class JobBuilderSupport {
-
-    final protected JobEngineConfig config;
-    final protected CubeSegment seg;
-    final protected String submitter;
-
-    public JobBuilderSupport(CubeSegment seg, String submitter) {
-        Preconditions.checkNotNull(seg, "segment cannot be null");
-        this.config = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
-        this.seg = seg;
-        this.submitter = submitter;
-    }
-    
-    public MapReduceExecutable createFactDistinctColumnsStep(String jobId) {
-        return createFactDistinctColumnsStep(jobId, false);
-    }
-    
-    public MapReduceExecutable createFactDistinctColumnsStepWithStats(String jobId) {
-        return createFactDistinctColumnsStep(jobId, true);
-    }
-    
-    private MapReduceExecutable createFactDistinctColumnsStep(String jobId, boolean withStats) {
-        MapReduceExecutable result = new MapReduceExecutable();
-        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());
-        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");
-
-        result.setMapReduceParams(cmd.toString());
-        return result;
-    }
-
-    public HadoopShellExecutable createBuildDictionaryStep(String jobId) {
-        // base cuboid job
-        HadoopShellExecutable buildDictionaryStep = new HadoopShellExecutable();
-        buildDictionaryStep.setName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY);
-        StringBuilder cmd = new StringBuilder();
-        appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "segmentname", seg.getName());
-        appendExecCmdParameters(cmd, "input", getFactDistinctColumnsPath(jobId));
-
-        buildDictionaryStep.setJobParams(cmd.toString());
-        buildDictionaryStep.setJobClass(CreateDictionaryJob.class);
-        return buildDictionaryStep;
-    }
-
-    public UpdateCubeInfoAfterBuildStep createUpdateCubeInfoAfterBuildStep(String jobId) {
-        final UpdateCubeInfoAfterBuildStep updateCubeInfoStep = new UpdateCubeInfoAfterBuildStep();
-        updateCubeInfoStep.setName(ExecutableConstants.STEP_NAME_UPDATE_CUBE_INFO);
-        updateCubeInfoStep.setCubeName(seg.getCubeInstance().getName());
-        updateCubeInfoStep.setSegmentId(seg.getUuid());
-        updateCubeInfoStep.setCubingJobId(jobId);
-        return updateCubeInfoStep;
-    }
-
-    public MergeDictionaryStep createMergeDictionaryStep(List<String> mergingSegmentIds) {
-        MergeDictionaryStep result = new MergeDictionaryStep();
-        result.setName(ExecutableConstants.STEP_NAME_MERGE_DICTIONARY);
-        result.setCubeName(seg.getCubeInstance().getName());
-        result.setSegmentId(seg.getUuid());
-        result.setMergingSegmentIds(mergingSegmentIds);
-        return result;
-    }
-    
-    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.setSegmentId(seg.getUuid());
-        result.setMergingSegmentIds(mergingSegmentIds);
-        result.setCubingJobId(jobId);
-        return result;
-    }
-
-    // ============================================================================
-
-    public String getJobWorkingDir(String jobId) {
-        return getJobWorkingDir(config, jobId);
-    }
-    
-    public String getCuboidRootPath(String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/cuboid/";
-    }
-    
-    public String getCuboidRootPath(CubeSegment seg) {
-        return getCuboidRootPath(seg.getLastBuildJobID());
-    }
-    
-    public void appendMapReduceParameters(StringBuilder buf, CubeSegment seg) {
-        try {
-            String jobConf = config.getHadoopJobConfFilePath(seg.getCubeDesc().getModel().getCapacity());
-            if (jobConf != null && jobConf.length() > 0) {
-                buf.append(" -conf ").append(jobConf);
-            }
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-    
-    public String getFactDistinctColumnsPath(String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/fact_distinct_columns";
-    }
-
-
-    public String getStatisticsPath(String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/statistics";
-    }
-
-    // ============================================================================
-    // static methods also shared by other job flow participant
-    // ----------------------------------------------------------------------------
-
-    public static String getJobWorkingDir(JobEngineConfig conf, String jobId) {
-        return conf.getHdfsWorkingDirectory() + "/" + "kylin-" + jobId;
-    }
-
-    public static StringBuilder appendExecCmdParameters(StringBuilder buf, String paraName, String paraValue) {
-        return buf.append(" -").append(paraName).append(" ").append(paraValue);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java b/job/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
deleted file mode 100644
index 61328c9..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
+++ /dev/null
@@ -1,47 +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;
-
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.IBatchCubingEngine;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-
-public class MRBatchCubingEngine implements IBatchCubingEngine {
-
-    @Override
-    public DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter) {
-        return new BatchCubingJobBuilder(newSegment, submitter).build();
-    }
-
-    @Override
-    public DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter) {
-        return new BatchMergeJobBuilder(mergeSegment, submitter).build();
-    }
-    
-    @Override
-    public Class<?> getSourceInterface() {
-        return IMRInput.class;
-    }
-
-    @Override
-    public Class<?> getStorageInterface() {
-        return IMROutput.class;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java b/job/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
deleted file mode 100644
index 57ec128..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
+++ /dev/null
@@ -1,47 +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;
-
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.IBatchCubingEngine;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-
-public class MRBatchCubingEngine2 implements IBatchCubingEngine {
-
-    @Override
-    public DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter) {
-        return new BatchCubingJobBuilder2(newSegment, submitter).build();
-    }
-
-    @Override
-    public DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter) {
-        return new BatchMergeJobBuilder2(mergeSegment, submitter).build();
-    }
-    
-    @Override
-    public Class<?> getSourceInterface() {
-        return IMRInput.class;
-    }
-
-    @Override
-    public Class<?> getStorageInterface() {
-        return IMROutput2.class;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/MRUtil.java b/job/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
deleted file mode 100644
index dc0533e..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
+++ /dev/null
@@ -1,55 +0,0 @@
-package org.apache.kylin.engine.mr;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
-import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
-import org.apache.kylin.engine.mr.IMROutput.IMRBatchCubingOutputSide;
-import org.apache.kylin.engine.mr.IMROutput.IMRBatchMergeOutputSide;
-import org.apache.kylin.engine.mr.IMROutput2.IMRBatchCubingOutputSide2;
-import org.apache.kylin.engine.mr.IMROutput2.IMRBatchMergeInputSide2;
-import org.apache.kylin.engine.mr.IMROutput2.IMRBatchMergeOutputSide2;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.source.TableSourceFactory;
-import org.apache.kylin.storage.StorageFactory2;
-
-public class MRUtil {
-
-    public static IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg) {
-        return TableSourceFactory.createEngineAdapter(seg, IMRInput.class).getBatchCubingInputSide(seg);
-    }
-
-    public static IMRTableInputFormat getTableInputFormat(String tableName) {
-        return getTableInputFormat(getTableDesc(tableName));
-    }
-
-    public static IMRTableInputFormat getTableInputFormat(TableDesc tableDesc) {
-        return TableSourceFactory.createEngineAdapter(tableDesc, IMRInput.class).getTableInputFormat(tableDesc);
-    }
-
-    private static TableDesc getTableDesc(String tableName) {
-        return MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(tableName);
-    }
-
-    public static IMRBatchCubingOutputSide getBatchCubingOutputSide(CubeSegment seg) {
-        return StorageFactory2.createEngineAdapter(seg, IMROutput.class).getBatchCubingOutputSide(seg);
-    }
-
-    public static IMRBatchMergeOutputSide getBatchMergeOutputSide(CubeSegment seg) {
-        return StorageFactory2.createEngineAdapter(seg, IMROutput.class).getBatchMergeOutputSide(seg);
-    }
-
-    public static IMRBatchCubingOutputSide2 getBatchCubingOutputSide2(CubeSegment seg) {
-        return StorageFactory2.createEngineAdapter(seg, IMROutput2.class).getBatchCubingOutputSide(seg);
-    }
-    
-    public static IMRBatchMergeInputSide2 getBatchMergeInputSide2(CubeSegment seg) {
-        return StorageFactory2.createEngineAdapter(seg, IMROutput2.class).getBatchMergeInputSide(seg);
-    }
-    
-    public static IMRBatchMergeOutputSide2 getBatchMergeOutputSide2(CubeSegment seg) {
-        return StorageFactory2.createEngineAdapter(seg, IMROutput2.class).getBatchMergeOutputSide(seg);
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
deleted file mode 100644
index aae5d89..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine.mr.steps;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.util.ToolRunner;
-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.ByteArrayWritable;
-import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
-import org.apache.kylin.engine.mr.IMROutput2.IMRStorageOutputFormat;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-public class InMemCuboidJob extends AbstractHadoopJob {
-
-    protected static final Logger logger = LoggerFactory.getLogger(InMemCuboidJob.class);
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_JOB_FLOW_ID);
-            options.addOption(OPTION_CUBE_NAME);
-            options.addOption(OPTION_SEGMENT_NAME);
-            parseOptions(options, args);
-
-            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
-            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
-
-            KylinConfig config = KylinConfig.getInstanceFromEnv();
-            CubeManager cubeMgr = CubeManager.getInstance(config);
-            CubeInstance cube = cubeMgr.getCube(cubeName);
-            CubeSegment cubeSeg = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
-
-            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
-            logger.info("Starting: " + job.getJobName());
-            
-            setJobClasspath(job);
-            
-            // add metadata to distributed cache
-            attachKylinPropsAndMetadata(cube, job.getConfiguration());
-
-            // set job configuration
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
-            long timeout = 1000*60*60L; // 1 hour
-            job.getConfiguration().set("mapred.task.timeout", String.valueOf(timeout));
-            
-            // set input
-            IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSeg).getFlatTableInputFormat();
-            flatTableInputFormat.configureJob(job);
-            
-            // set mapper
-            job.setMapperClass(InMemCuboidMapper.class);
-            job.setMapOutputKeyClass(ByteArrayWritable.class);
-            job.setMapOutputValueClass(ByteArrayWritable.class);
-            
-            // set output
-            IMRStorageOutputFormat storageOutputFormat = MRUtil.getBatchCubingOutputSide2(cubeSeg).getStorageOutputFormat();
-            storageOutputFormat.configureOutput(InMemCuboidReducer.class, getOptionValue(OPTION_JOB_FLOW_ID), job);
-            
-            return waitForCompletion(job);
-        } catch (Exception e) {
-            logger.error("error in CuboidJob", e);
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        InMemCuboidJob job = new InMemCuboidJob();
-        int exitCode = ToolRunner.run(job, args);
-        System.exit(exitCode);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
deleted file mode 100644
index ff6ffe5..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
+++ /dev/null
@@ -1,122 +0,0 @@
-package org.apache.kylin.engine.mr.steps;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.DimensionDesc;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.engine.mr.ByteArrayWritable;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.job.inmemcubing.DoggedCubeBuilder;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import com.google.common.collect.Maps;
-
-/**
- */
-public class InMemCuboidMapper<KEYIN> extends KylinMapper<KEYIN, Object, ByteArrayWritable, ByteArrayWritable> {
-
-    private static final Log logger = LogFactory.getLog(InMemCuboidMapper.class);
-    private CubeInstance cube;
-    private CubeDesc cubeDesc;
-    private CubeSegment cubeSegment;
-    private IMRTableInputFormat flatTableInputFormat;
-
-    private int counter;
-    private BlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(10000);
-    private Future<?> future;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        Configuration conf = context.getConfiguration();
-
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-        String cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
-        cube = CubeManager.getInstance(config).getCube(cubeName);
-        cubeDesc = cube.getDescriptor();
-        String segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME);
-        cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
-        flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSegment).getFlatTableInputFormat();
-
-        Map<TblColRef, Dictionary<?>> dictionaryMap = Maps.newHashMap();
-
-        for (DimensionDesc dim : cubeDesc.getDimensions()) {
-            // dictionary
-            for (TblColRef col : dim.getColumnRefs()) {
-                if (cubeDesc.getRowkey().isUseDictionary(col)) {
-                    Dictionary<?> dict = cubeSegment.getDictionary(col);
-                    if (dict == null) {
-                        logger.warn("Dictionary for " + col + " was not found.");
-                    }
-
-                    dictionaryMap.put(col, cubeSegment.getDictionary(col));
-                }
-            }
-        }
-
-        DoggedCubeBuilder cubeBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap);
-        ExecutorService executorService = Executors.newSingleThreadExecutor();
-        future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new MapContextGTRecordWriter(context, cubeDesc, cubeSegment)));
-
-    }
-
-    @Override
-    public void map(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
-        // put each row to the queue
-        String[] row = flatTableInputFormat.parseMapperInput(record);
-        List<String> rowAsList = Arrays.asList(row);
-        
-        while (!future.isDone()) {
-            if (queue.offer(rowAsList, 1, TimeUnit.SECONDS)) {
-                counter++;
-                if (counter % BatchConstants.COUNTER_MAX == 0) {
-                    logger.info("Handled " + counter + " records!");
-                }
-                break;
-            }
-        }
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-        logger.info("Totally handled " + counter + " records!");
-
-        while (!future.isDone()) {
-            if (queue.offer(new ArrayList<String>(0), 1, TimeUnit.SECONDS)) {
-                break;
-            }
-        }
-        
-        try {
-            future.get();
-        } catch (Exception e) {
-            throw new IOException("Failed to build cube in mapper " + context.getTaskAttemptID().getTaskID().getId(), e);
-        }
-        queue.clear();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
deleted file mode 100644
index 45da2c8..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
+++ /dev/null
@@ -1,95 +0,0 @@
-package org.apache.kylin.engine.mr.steps;
-
-import java.io.IOException;
-import java.util.List;
-
-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.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
-import org.apache.kylin.engine.mr.ByteArrayWritable;
-import org.apache.kylin.engine.mr.KylinReducer;
-import org.apache.kylin.engine.mr.IMROutput2.IMRStorageOutputFormat;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.measure.MeasureAggregators;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-public class InMemCuboidReducer extends KylinReducer<ByteArrayWritable, ByteArrayWritable, Object, Object> {
-
-    private static final Logger logger = LoggerFactory.getLogger(InMemCuboidReducer.class);
-
-    private IMRStorageOutputFormat storageOutputFormat;
-    private MeasureCodec codec;
-    private MeasureAggregators aggs;
-
-    private int counter;
-    private Object[] input;
-    private Object[] result;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-        
-        String cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
-        String segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME);
-        boolean isMerge = Boolean.parseBoolean(context.getConfiguration().get(BatchConstants.CFG_IS_MERGE));
-
-        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
-        CubeDesc cubeDesc = cube.getDescriptor();
-        CubeSegment cubeSeg = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
-        if (isMerge)
-            storageOutputFormat = MRUtil.getBatchMergeOutputSide2(cubeSeg).getStorageOutputFormat();
-        else
-            storageOutputFormat = MRUtil.getBatchCubingOutputSide2(cubeSeg).getStorageOutputFormat();
-
-        List<MeasureDesc> measuresDescs = Lists.newArrayList();
-        for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
-            for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
-                for (MeasureDesc measure : colDesc.getMeasures()) {
-                    measuresDescs.add(measure);
-                }
-            }
-        }
-
-        codec = new MeasureCodec(measuresDescs);
-        aggs = new MeasureAggregators(measuresDescs);
-
-        input = new Object[measuresDescs.size()];
-        result = new Object[measuresDescs.size()];
-    }
-
-    @Override
-    public void reduce(ByteArrayWritable key, Iterable<ByteArrayWritable> values, Context context) throws IOException, InterruptedException {
-
-        aggs.reset();
-
-        for (ByteArrayWritable value : values) {
-            codec.decode(value.asBuffer(), input);
-            aggs.aggregate(input);
-        }
-        aggs.collectStates(result);
-        
-        storageOutputFormat.doReducerOutput(key, result, context);
-        
-        counter++;
-        if (counter % BatchConstants.COUNTER_MAX == 0) {
-            logger.info("Handled " + counter + " records!");
-        }
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/MapContextGTRecordWriter.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/MapContextGTRecordWriter.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/MapContextGTRecordWriter.java
deleted file mode 100644
index f2a5fcf..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/MapContextGTRecordWriter.java
+++ /dev/null
@@ -1,96 +0,0 @@
-package org.apache.kylin.engine.mr.steps;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.BitSet;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.mapreduce.MapContext;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.engine.mr.ByteArrayWritable;
-import org.apache.kylin.job.inmemcubing.ICuboidWriter;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.gridtable.GTRecord;
-
-/**
- */
-public class MapContextGTRecordWriter implements ICuboidWriter {
-
-    private static final Log logger = LogFactory.getLog(MapContextGTRecordWriter.class);
-    protected MapContext<?, ?, ByteArrayWritable, ByteArrayWritable> mapContext;
-    private Long lastCuboidId;
-    protected CubeSegment cubeSegment;
-    protected CubeDesc cubeDesc;
-
-    private int bytesLength;
-    private int dimensions;
-    private int measureCount;
-    private byte[] keyBuf;
-    private int[] measureColumnsIndex;
-    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-    private ByteArrayWritable outputKey = new ByteArrayWritable();
-    private ByteArrayWritable outputValue = new ByteArrayWritable();
-    private long cuboidRowCount = 0;
-
-    public MapContextGTRecordWriter(MapContext<?, ?, ByteArrayWritable, ByteArrayWritable> mapContext, CubeDesc cubeDesc, CubeSegment cubeSegment) {
-        this.mapContext = mapContext;
-        this.cubeDesc = cubeDesc;
-        this.cubeSegment = cubeSegment;
-        this.measureCount = cubeDesc.getMeasures().size();
-
-    }
-
-    @Override
-    public void write(long cuboidId, GTRecord record) throws IOException {
-
-        if (lastCuboidId == null || !lastCuboidId.equals(cuboidId)) {
-            // output another cuboid
-            initVariables(cuboidId);
-            if (lastCuboidId != null) {
-                logger.info("Cuboid " + lastCuboidId + " has " + cuboidRowCount + " rows");
-                cuboidRowCount = 0;
-            }
-        }
-
-        cuboidRowCount++;
-        int offSet = RowConstants.ROWKEY_CUBOIDID_LEN;
-        for (int x = 0; x < dimensions; x++) {
-            System.arraycopy(record.get(x).array(), record.get(x).offset(), keyBuf, offSet, record.get(x).length());
-            offSet += record.get(x).length();
-        }
-
-        //output measures
-        valueBuf.clear();
-        record.exportColumns(measureColumnsIndex, valueBuf);
-
-        outputKey.set(keyBuf, 0, offSet);
-        outputValue.set(valueBuf.array(), 0, valueBuf.position());
-        try {
-            mapContext.write(outputKey, outputValue);
-        } catch (InterruptedException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private void initVariables(Long cuboidId) {
-        bytesLength = RowConstants.ROWKEY_CUBOIDID_LEN;
-        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidId);
-        for (TblColRef column : cuboid.getColumns()) {
-            bytesLength += cubeSegment.getColumnLength(column);
-        }
-
-        keyBuf = new byte[bytesLength];
-        dimensions = BitSet.valueOf(new long[]{cuboidId}).cardinality();
-        measureColumnsIndex = new int[measureCount];
-        for (int i = 0; i < measureCount; i++) {
-            measureColumnsIndex[i] = dimensions + i;
-        }
-
-        System.arraycopy(Bytes.toBytes(cuboidId), 0, keyBuf, 0, RowConstants.ROWKEY_CUBOIDID_LEN);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageJob.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageJob.java
deleted file mode 100644
index 3d3b1f4..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageJob.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine.mr.steps;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.mapreduce.Job;
-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.ByteArrayWritable;
-import org.apache.kylin.engine.mr.IMROutput2.IMRStorageInputFormat;
-import org.apache.kylin.engine.mr.IMROutput2.IMRStorageOutputFormat;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.cube.CuboidJob;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-
-/**
- * @author shaoshi
- */
-public class MergeCuboidFromStorageJob extends CuboidJob {
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_JOB_FLOW_ID);
-            options.addOption(OPTION_CUBE_NAME);
-            options.addOption(OPTION_SEGMENT_NAME);
-            parseOptions(options, args);
-
-            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
-            String segmentName = getOptionValue(OPTION_SEGMENT_NAME).toUpperCase();
-            KylinConfig config = KylinConfig.getInstanceFromEnv();
-            CubeManager cubeMgr = CubeManager.getInstance(config);
-            CubeInstance cube = cubeMgr.getCube(cubeName);
-            CubeSegment cubeSeg = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
-
-            Configuration conf = this.getConf();
-            HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
-
-            // start job
-            String jobName = getOptionValue(OPTION_JOB_NAME);
-            System.out.println("Starting: " + jobName);
-            job = Job.getInstance(conf, jobName);
-
-            setJobClasspath(job);
-            
-            // add metadata to distributed cache
-            attachKylinPropsAndMetadata(cube, job.getConfiguration());
-
-            // set job configuration
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
-            job.getConfiguration().set(BatchConstants.CFG_IS_MERGE, "true");
-
-            // configure mapper input
-            IMRStorageInputFormat storageInputFormat = MRUtil.getBatchMergeInputSide2(cubeSeg).getStorageInputFormat();
-            storageInputFormat.configureInput(MergeCuboidFromStorageMapper.class, ByteArrayWritable.class, ByteArrayWritable.class, job);
-
-            // configure reducer output
-            IMRStorageOutputFormat storageOutputFormat = MRUtil.getBatchMergeOutputSide2(cubeSeg).getStorageOutputFormat();
-            storageOutputFormat.configureOutput(InMemCuboidReducer.class, getOptionValue(OPTION_JOB_FLOW_ID), job);
-            
-            return waitForCompletion(job);
-        } catch (Exception e) {
-            logger.error("error in MergeCuboidFromHBaseJob", e);
-            printUsage(options);
-            throw e;
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
deleted file mode 100644
index 1162f14..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java
+++ /dev/null
@@ -1,197 +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.steps;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.util.SplittedBytes;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.common.RowKeySplitter;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.dict.DictionaryManager;
-import org.apache.kylin.engine.mr.ByteArrayWritable;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.engine.mr.IMROutput2.IMRStorageInputFormat;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-/**
- * @author shaoshi
- */
-public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, ByteArrayWritable, ByteArrayWritable> {
-
-    private static final Logger logger = LoggerFactory.getLogger(MergeCuboidFromStorageMapper.class);
-    
-    private KylinConfig config;
-    private String cubeName;
-    private String segmentName;
-    private CubeManager cubeManager;
-    private CubeInstance cube;
-    private CubeDesc cubeDesc;
-    private CubeSegment mergedCubeSegment;
-    private CubeSegment sourceCubeSegment; // Must be unique during a mapper's life cycle
-    private IMRStorageInputFormat storageInputFormat;
-
-    private ByteArrayWritable outputKey = new ByteArrayWritable();
-    private byte[] newKeyBuf;
-    private RowKeySplitter rowKeySplitter;
-
-    private HashMap<TblColRef, Boolean> dictsNeedMerging = new HashMap<TblColRef, Boolean>();
-
-    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-    private MeasureCodec codec;
-    private ByteArrayWritable outputValue = new ByteArrayWritable();
-
-    private Boolean checkNeedMerging(TblColRef col) throws IOException {
-        Boolean ret = dictsNeedMerging.get(col);
-        if (ret != null)
-            return ret;
-        else {
-            ret = cubeDesc.getRowkey().isUseDictionary(col);
-            if (ret) {
-                String dictTable = (String) DictionaryManager.getInstance(config).decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0];
-                ret = cubeDesc.getFactTable().equalsIgnoreCase(dictTable);
-            }
-            dictsNeedMerging.put(col, ret);
-            return ret;
-        }
-    }
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-        config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-
-        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
-        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME).toUpperCase();
-
-        cubeManager = CubeManager.getInstance(config);
-        cube = cubeManager.getCube(cubeName);
-        cubeDesc = cube.getDescriptor();
-        mergedCubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
-        storageInputFormat = MRUtil.getBatchMergeInputSide2(mergedCubeSegment).getStorageInputFormat();
-
-        newKeyBuf = new byte[256];// size will auto-grow
-
-        sourceCubeSegment = storageInputFormat.findSourceSegment(context, cube);
-        logger.info(sourceCubeSegment.toString());
-
-        this.rowKeySplitter = new RowKeySplitter(sourceCubeSegment, 65, 255);
-
-        List<MeasureDesc> measuresDescs = Lists.newArrayList();
-        for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
-            for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
-                for (MeasureDesc measure : colDesc.getMeasures()) {
-                    measuresDescs.add(measure);
-                }
-            }
-        }
-        codec = new MeasureCodec(measuresDescs);
-    }
-
-    @Override
-    public void map(Object inKey, Object inValue, Context context) throws IOException, InterruptedException {
-        Pair<ByteArrayWritable, Object[]> pair = storageInputFormat.parseMapperInput(inKey, inValue);
-        ByteArrayWritable key = pair.getFirst();
-        Object[] value = pair.getSecond();
-        
-        Preconditions.checkState(key.offset() == 0);
-        
-        long cuboidID = rowKeySplitter.split(key.array(), key.length());
-        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidID);
-
-        SplittedBytes[] splittedByteses = rowKeySplitter.getSplitBuffers();
-        int bufOffset = 0;
-        BytesUtil.writeLong(cuboidID, newKeyBuf, bufOffset, RowConstants.ROWKEY_CUBOIDID_LEN);
-        bufOffset += RowConstants.ROWKEY_CUBOIDID_LEN;
-
-        for (int i = 0; i < cuboid.getColumns().size(); ++i) {
-            TblColRef col = cuboid.getColumns().get(i);
-
-            if (this.checkNeedMerging(col)) {
-                // if dictionary on fact table column, needs rewrite
-                DictionaryManager dictMgr = DictionaryManager.getInstance(config);
-                Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(col));
-                Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(col));
-
-                while (sourceDict.getSizeOfValue() > newKeyBuf.length - bufOffset || mergedDict.getSizeOfValue() > newKeyBuf.length - bufOffset) {
-                    byte[] oldBuf = newKeyBuf;
-                    newKeyBuf = new byte[2 * newKeyBuf.length];
-                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
-                }
-
-                int idInSourceDict = BytesUtil.readUnsigned(splittedByteses[i + 1].value, 0, splittedByteses[i + 1].length);
-
-                int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBuf, bufOffset);
-                int idInMergedDict;
-                if (size < 0) {
-                    idInMergedDict = mergedDict.nullId();
-                } else {
-                    idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBuf, bufOffset, size);
-                }
-                BytesUtil.writeUnsigned(idInMergedDict, newKeyBuf, bufOffset, mergedDict.getSizeOfId());
-
-                bufOffset += mergedDict.getSizeOfId();
-            } else {
-                // keep as it is
-                while (splittedByteses[i + 1].length > newKeyBuf.length - bufOffset) {
-                    byte[] oldBuf = newKeyBuf;
-                    newKeyBuf = new byte[2 * newKeyBuf.length];
-                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
-                }
-
-                System.arraycopy(splittedByteses[i + 1].value, 0, newKeyBuf, bufOffset, splittedByteses[i + 1].length);
-                bufOffset += splittedByteses[i + 1].length;
-            }
-        }
-        byte[] newKey = Arrays.copyOf(newKeyBuf, bufOffset);
-        outputKey.set(newKey, 0, newKey.length);
-
-        valueBuf.clear();
-        codec.encode(value, valueBuf);
-        outputValue.set(valueBuf.array(), 0, valueBuf.position());
-        
-        context.write(outputKey, outputValue);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
deleted file mode 100644
index d99cb03..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
+++ /dev/null
@@ -1,197 +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.steps;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeUpdate;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.DimensionDesc;
-import org.apache.kylin.dict.DictionaryInfo;
-import org.apache.kylin.dict.DictionaryManager;
-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.TblColRef;
-
-import java.io.IOException;
-import java.util.*;
-
-public class MergeDictionaryStep extends AbstractExecutable {
-
-    private static final String CUBE_NAME = "cubeName";
-    private static final String SEGMENT_ID = "segmentId";
-    private static final String MERGING_SEGMENT_IDS = "mergingSegmentIds";
-
-    public MergeDictionaryStep() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        KylinConfig conf = context.getConfig();
-        final CubeManager mgr = CubeManager.getInstance(conf);
-        final CubeInstance cube = mgr.getCube(getCubeName());
-        final CubeSegment newSegment = cube.getSegmentById(getSegmentId());
-        final List<CubeSegment> mergingSegments = getMergingSegments(cube);
-        
-        Collections.sort(mergingSegments);
-        
-        try {
-            checkLookupSnapshotsMustIncremental(mergingSegments);
-            
-            makeDictForNewSegment(conf, cube, newSegment, mergingSegments);
-            makeSnapshotForNewSegment(cube, newSegment, mergingSegments);
-
-            CubeUpdate cubeBuilder = new CubeUpdate(cube);
-            cubeBuilder.setToUpdateSegs(newSegment);
-            mgr.updateCube(cubeBuilder);
-            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
-        } catch (IOException e) {
-            logger.error("fail to merge dictionary or lookup snapshots", e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-    
-    private List<CubeSegment> getMergingSegments(CubeInstance cube) {
-        List<String> mergingSegmentIds = getMergingSegmentIds();
-        List<CubeSegment> result = Lists.newArrayListWithCapacity(mergingSegmentIds.size());
-        for (String id : mergingSegmentIds) {
-            result.add(cube.getSegmentById(id));
-        }
-        return result;
-    }
-
-    private void checkLookupSnapshotsMustIncremental(List<CubeSegment> mergingSegments) {
-
-        // FIXME check each newer snapshot has only NEW rows but no MODIFIED rows
-    }
-
-    /**
-     * For the new segment, we need to create dictionaries for it, too. For
-     * those dictionaries on fact table, create it by merging underlying
-     * dictionaries For those dictionaries on lookup table, just copy it from
-     * any one of the merging segments, it's guaranteed to be consistent(checked
-     * in CubeSegmentValidator)
-     *
-     * @param cube
-     * @param newSeg
-     * @throws IOException
-     */
-    private void makeDictForNewSegment(KylinConfig conf, CubeInstance cube, CubeSegment newSeg, List<CubeSegment> mergingSegments) throws IOException {
-        HashSet<TblColRef> colsNeedMeringDict = new HashSet<TblColRef>();
-        HashSet<TblColRef> colsNeedCopyDict = new HashSet<TblColRef>();
-        DictionaryManager dictMgr = DictionaryManager.getInstance(conf);
-
-        CubeDesc cubeDesc = cube.getDescriptor();
-        for (DimensionDesc dim : cubeDesc.getDimensions()) {
-            for (TblColRef col : dim.getColumnRefs()) {
-                if (newSeg.getCubeDesc().getRowkey().isUseDictionary(col)) {
-                    String dictTable = (String) dictMgr.decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0];
-                    if (cubeDesc.getFactTable().equalsIgnoreCase(dictTable)) {
-                        colsNeedMeringDict.add(col);
-                    } else {
-                        colsNeedCopyDict.add(col);
-                    }
-                }
-            }
-        }
-
-        for (TblColRef col : colsNeedMeringDict) {
-            logger.info("Merging fact table dictionary on : " + col);
-            List<DictionaryInfo> dictInfos = new ArrayList<DictionaryInfo>();
-            for (CubeSegment segment : mergingSegments) {
-                logger.info("Including fact table dictionary of segment : " + segment);
-                if (segment.getDictResPath(col) != null) {
-                    DictionaryInfo dictInfo = dictMgr.getDictionaryInfo(segment.getDictResPath(col));
-                    dictInfos.add(dictInfo);
-                }
-            }
-            mergeDictionaries(dictMgr, newSeg, dictInfos, col);
-        }
-
-        for (TblColRef col : colsNeedCopyDict) {
-            String path = mergingSegments.get(0).getDictResPath(col);
-            newSeg.putDictResPath(col, path);
-        }
-    }
-
-    private DictionaryInfo mergeDictionaries(DictionaryManager dictMgr, CubeSegment cubeSeg, List<DictionaryInfo> dicts, TblColRef col) throws IOException {
-        DictionaryInfo dictInfo = dictMgr.mergeDictionary(dicts);
-        if (dictInfo != null)
-            cubeSeg.putDictResPath(col, dictInfo.getResourcePath());
-
-        return dictInfo;
-    }
-
-    /**
-     * make snapshots for the new segment by copying from one of the underlying
-     * merging segments. it's guaranteed to be consistent(checked in
-     * CubeSegmentValidator)
-     *
-     * @param cube
-     * @param newSeg
-     */
-    private void makeSnapshotForNewSegment(CubeInstance cube, CubeSegment newSeg, List<CubeSegment> mergingSegments) {
-        CubeSegment lastSeg = mergingSegments.get(mergingSegments.size() - 1);
-        for (Map.Entry<String, String> entry : lastSeg.getSnapshots().entrySet()) {
-            newSeg.putSnapshotResPath(entry.getKey(), entry.getValue());
-        }
-    }
-
-    public void setCubeName(String cubeName) {
-        this.setParam(CUBE_NAME, cubeName);
-    }
-
-    private String getCubeName() {
-        return getParam(CUBE_NAME);
-    }
-
-    public void setSegmentId(String segmentId) {
-        this.setParam(SEGMENT_ID, segmentId);
-    }
-
-    private String getSegmentId() {
-        return getParam(SEGMENT_ID);
-    }
-
-    public void setMergingSegmentIds(List<String> ids) {
-        setParam(MERGING_SEGMENT_IDS, StringUtils.join(ids, ","));
-    }
-
-    private List<String> getMergingSegmentIds() {
-        final String ids = getParam(MERGING_SEGMENT_IDS);
-        if (ids != null) {
-            final String[] splitted = StringUtils.split(ids, ",");
-            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
-            for (String id: splitted) {
-                result.add(id);
-            }
-            return result;
-        } else {
-            return Collections.emptyList();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
deleted file mode 100644
index 8bd6ea2..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
+++ /dev/null
@@ -1,189 +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.steps;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-import org.apache.kylin.job.hadoop.cube.FactDistinctColumnsReducer;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-public class MergeStatisticsStep extends AbstractExecutable {
-
-    private static final String CUBE_NAME = "cubeName";
-    private static final String SEGMENT_ID = "segmentId";
-    private static final String MERGING_SEGMENT_IS = "mergingSegmentIds";
-    private static final String MERGED_STATISTICS_PATH = "mergedStatisticsPath";
-    protected Map<Long, HyperLogLogPlusCounter> cuboidHLLMap = Maps.newHashMap();
-
-    public MergeStatisticsStep() {
-        super();
-    }
-
-    @Override
-    @SuppressWarnings("deprecation")
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        KylinConfig kylinConf = context.getConfig();
-        final CubeManager mgr = CubeManager.getInstance(kylinConf);
-        final CubeInstance cube = mgr.getCube(getCubeName());
-        final CubeSegment newSegment = cube.getSegmentById(getSegmentId());
-
-        Configuration conf = new Configuration();
-        ResourceStore rs = ResourceStore.getStore(kylinConf);
-        try {
-
-            int averageSamplingPercentage = 0;
-            for (String segmentId : this.getMergingSegmentIds()) {
-                String fileKey = CubeSegment.getStatisticsResourcePath(getCubeName(), segmentId);
-                InputStream is = rs.getResource(fileKey);
-                File tempFile = null;
-                FileOutputStream tempFileStream = null;
-                try {
-                    tempFile = File.createTempFile(segmentId, ".seq");
-                    tempFileStream = new FileOutputStream(tempFile);
-                    org.apache.commons.io.IOUtils.copy(is, tempFileStream);
-                } finally {
-                    IOUtils.closeStream(is);
-                    IOUtils.closeStream(tempFileStream);
-                }
-
-                FileSystem fs = HadoopUtil.getFileSystem("file:///" + tempFile.getAbsolutePath());
-                SequenceFile.Reader reader = null;
-                try {
-                    reader = new SequenceFile.Reader(fs, new Path(tempFile.getAbsolutePath()), conf);
-                    LongWritable key = (LongWritable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
-                    BytesWritable value = (BytesWritable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
-                    while (reader.next(key, value)) {
-                        if (key.get() == 0l) {
-                            // sampling percentage;
-                            averageSamplingPercentage += Bytes.toInt(value.getBytes());
-                        } else {
-                            HyperLogLogPlusCounter hll = new HyperLogLogPlusCounter(14);
-                            ByteArray byteArray = new ByteArray(value.getBytes());
-                            hll.readRegisters(byteArray.asBuffer());
-
-                            if (cuboidHLLMap.get(key.get()) != null) {
-                                cuboidHLLMap.get(key.get()).merge(hll);
-                            } else {
-                                cuboidHLLMap.put(key.get(), hll);
-                            }
-                        }
-                    }
-                } catch (Exception e) {
-                    e.printStackTrace();
-                    throw e;
-                } finally {
-                    IOUtils.closeStream(reader);
-                }
-            }
-            averageSamplingPercentage = averageSamplingPercentage / this.getMergingSegmentIds().size();
-            FactDistinctColumnsReducer.writeCuboidStatistics(conf, new Path(getMergedStatisticsPath()), cuboidHLLMap, averageSamplingPercentage);
-            Path statisticsFilePath = new Path(getMergedStatisticsPath(), BatchConstants.CFG_STATISTICS_CUBOID_ESTIMATION);
-            FileSystem fs = statisticsFilePath.getFileSystem(conf);
-            FSDataInputStream is = fs.open(statisticsFilePath);
-            try {
-                // put the statistics to metadata store
-                String statisticsFileName = newSegment.getStatisticsResourcePath();
-                rs.putResource(statisticsFileName, is, System.currentTimeMillis());
-            } finally {
-                IOUtils.closeStream(is);
-            }
-
-            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
-        } catch (IOException e) {
-            logger.error("fail to merge cuboid statistics", e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-
-
-    public void setCubeName(String cubeName) {
-        this.setParam(CUBE_NAME, cubeName);
-    }
-
-    private String getCubeName() {
-        return getParam(CUBE_NAME);
-    }
-
-    public void setSegmentId(String segmentId) {
-        this.setParam(SEGMENT_ID, segmentId);
-    }
-
-    private String getSegmentId() {
-        return getParam(SEGMENT_ID);
-    }
-
-    public void setMergingSegmentIds(List<String> ids) {
-        setParam(MERGING_SEGMENT_IS, StringUtils.join(ids, ","));
-    }
-
-    private List<String> getMergingSegmentIds() {
-        final String ids = getParam(MERGING_SEGMENT_IS);
-        if (ids != null) {
-            final String[] splitted = StringUtils.split(ids, ",");
-            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
-            for (String id : splitted) {
-                result.add(id);
-            }
-            return result;
-        } else {
-            return Collections.emptyList();
-        }
-    }
-
-    public void setMergedStatisticsPath(String path) {
-        setParam(MERGED_STATISTICS_PATH, path);
-    }
-
-    private String getMergedStatisticsPath() {
-        return getParam(MERGED_STATISTICS_PATH);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
deleted file mode 100644
index 14eef1a..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/SaveStatisticsStep.java
+++ /dev/null
@@ -1,110 +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.steps;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-
-/**
- * Save the cube segment statistic to Kylin metadata store
- *
- */
-public class SaveStatisticsStep extends AbstractExecutable {
-
-    private static final String CUBE_NAME = "cubeName";
-    private static final String SEGMENT_ID = "segmentId";
-    private static final String STATISTICS_PATH = "statisticsPath";
-
-    public SaveStatisticsStep() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        KylinConfig kylinConf = context.getConfig();
-        final CubeManager mgr = CubeManager.getInstance(kylinConf);
-        final CubeInstance cube = mgr.getCube(getCubeName());
-        final CubeSegment newSegment = cube.getSegmentById(getSegmentId());
-
-        ResourceStore rs = ResourceStore.getStore(kylinConf);
-        try {
-            Path statisticsFilePath = new Path(getStatisticsPath(), BatchConstants.CFG_STATISTICS_CUBOID_ESTIMATION);
-            FileSystem fs = FileSystem.get(HadoopUtil.getCurrentConfiguration());
-            if (!fs.exists(statisticsFilePath))
-                throw new IOException("File " + statisticsFilePath + " does not exists;");
-
-            FSDataInputStream is = fs.open(statisticsFilePath);
-            try {
-                // put the statistics to metadata store
-                String statisticsFileName = newSegment.getStatisticsResourcePath();
-                rs.putResource(statisticsFileName, is, System.currentTimeMillis());
-                fs.delete(statisticsFilePath, false);
-            } finally {
-                IOUtils.closeStream(is);
-            }
-
-
-            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
-        } catch (IOException e) {
-            logger.error("fail to save cuboid statistics", e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-
-
-    public void setCubeName(String cubeName) {
-        this.setParam(CUBE_NAME, cubeName);
-    }
-
-    private String getCubeName() {
-        return getParam(CUBE_NAME);
-    }
-
-    public void setSegmentId(String segmentId) {
-        this.setParam(SEGMENT_ID, segmentId);
-    }
-
-    private String getSegmentId() {
-        return getParam(SEGMENT_ID);
-    }
-
-    public void setStatisticsPath(String path) {
-        this.setParam(STATISTICS_PATH, path);
-    }
-
-    private String getStatisticsPath() {
-        return getParam(STATISTICS_PATH);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterBuildStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterBuildStep.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterBuildStep.java
deleted file mode 100644
index dd99a64..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterBuildStep.java
+++ /dev/null
@@ -1,101 +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.steps;
-
-import java.io.IOException;
-
-import org.apache.kylin.cube.CubeUpdate;
-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.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-
-/**
- */
-public class UpdateCubeInfoAfterBuildStep extends AbstractExecutable {
-
-    private static final String SEGMENT_ID = "segmentId";
-    private static final String CUBE_NAME = "cubeName";
-    private static final String CUBING_JOB_ID = "cubingJobId";
-
-    public UpdateCubeInfoAfterBuildStep() {
-        super();
-    }
-
-    public void setCubeName(String cubeName) {
-        this.setParam(CUBE_NAME, cubeName);
-    }
-
-    private String getCubeName() {
-        return getParam(CUBE_NAME);
-    }
-
-    public void setSegmentId(String segmentId) {
-        this.setParam(SEGMENT_ID, segmentId);
-    }
-
-    private String getSegmentId() {
-        return getParam(SEGMENT_ID);
-    }
-
-    public void setCubingJobId(String id) {
-        setParam(CUBING_JOB_ID, id);
-    }
-
-    private String getCubingJobId() {
-        return getParam(CUBING_JOB_ID);
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        final CubeManager cubeManager = CubeManager.getInstance(context.getConfig());
-        final CubeInstance cube = cubeManager.getCube(getCubeName());
-        final CubeSegment segment = cube.getSegmentById(getSegmentId());
-        
-        CubingJob cubingJob = (CubingJob) executableManager.getJob(getCubingJobId());
-        long sourceCount = cubingJob.findSourceRecordCount();
-        long sourceSizeBytes = cubingJob.findSourceSizeBytes();
-        long cubeSizeBytes = cubingJob.findCubeSizeBytes();
-        boolean segmentReady = cubeSizeBytes > 0; // for build+merge scenario, convert HFile not happen yet, so cube size is 0
-
-        segment.setLastBuildJobID(getCubingJobId());
-        segment.setLastBuildTime(System.currentTimeMillis());
-        segment.setSizeKB(cubeSizeBytes / 1024);
-        segment.setInputRecords(sourceCount);
-        segment.setInputRecordsSize(sourceSizeBytes);
-
-        try {
-            if (segmentReady) {
-                cubeManager.promoteNewlyBuiltSegments(cube, segment);
-            } else {
-                CubeUpdate cubeBuilder = new CubeUpdate(cube);
-                cubeBuilder.setToUpdateSegs(segment);
-                cubeManager.updateCube(cubeBuilder);
-            }
-            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
-        } catch (IOException e) {
-            logger.error("fail to update cube after build", e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterMergeStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterMergeStep.java b/job/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterMergeStep.java
deleted file mode 100644
index d237908..0000000
--- a/job/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterMergeStep.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.engine.mr.steps;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.commons.lang.StringUtils;
-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.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 com.google.common.collect.Lists;
-
-/**
- */
-public class UpdateCubeInfoAfterMergeStep extends AbstractExecutable {
-
-    private static final String CUBE_NAME = "cubeName";
-    private static final String SEGMENT_ID = "segmentId";
-    private static final String MERGING_SEGMENT_IDS = "mergingSegmentIds";
-    private static final String CUBING_JOB_ID = "cubingJobId";
-
-    private final CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
-
-    public UpdateCubeInfoAfterMergeStep() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        final CubeInstance cube = cubeManager.getCube(getCubeName());
-
-        CubeSegment mergedSegment = cube.getSegmentById(getSegmentId());
-        if (mergedSegment == null) {
-            return new ExecuteResult(ExecuteResult.State.FAILED, "there is no segment with id:" + getSegmentId());
-        }
-        
-        CubingJob cubingJob = (CubingJob) executableManager.getJob(getCubingJobId());
-        long cubeSizeBytes = cubingJob.findCubeSizeBytes();
-
-        // collect source statistics
-        List<String> mergingSegmentIds = getMergingSegmentIds();
-        if (mergingSegmentIds.isEmpty()) {
-            return new ExecuteResult(ExecuteResult.State.FAILED, "there are no merging segments");
-        }
-        long sourceCount = 0L;
-        long sourceSize = 0L;
-        for (String id : mergingSegmentIds) {
-            CubeSegment segment = cube.getSegmentById(id);
-            sourceCount += segment.getInputRecords();
-            sourceSize += segment.getInputRecordsSize();
-        }
-
-        // update segment info
-        mergedSegment.setSizeKB(cubeSizeBytes / 1024);
-        mergedSegment.setInputRecords(sourceCount);
-        mergedSegment.setInputRecordsSize(sourceSize);
-        mergedSegment.setLastBuildJobID(getCubingJobId());
-        mergedSegment.setLastBuildTime(System.currentTimeMillis());
-
-        try {
-            cubeManager.promoteNewlyBuiltSegments(cube, mergedSegment);
-            return new ExecuteResult(ExecuteResult.State.SUCCEED);
-        } catch (IOException e) {
-            logger.error("fail to update cube after merge", e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-        }
-    }
-
-    public void setSegmentId(String segmentId) {
-        this.setParam(SEGMENT_ID, segmentId);
-    }
-
-    private String getSegmentId() {
-        return getParam(SEGMENT_ID);
-    }
-
-    public void setCubeName(String cubeName) {
-        this.setParam(CUBE_NAME, cubeName);
-    }
-
-    private String getCubeName() {
-        return getParam(CUBE_NAME);
-    }
-
-    public void setMergingSegmentIds(List<String> ids) {
-        setParam(MERGING_SEGMENT_IDS, StringUtils.join(ids, ","));
-    }
-
-    private List<String> getMergingSegmentIds() {
-        final String ids = getParam(MERGING_SEGMENT_IDS);
-        if (ids != null) {
-            final String[] splitted = StringUtils.split(ids, ",");
-            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
-            for (String id : splitted) {
-                result.add(id);
-            }
-            return result;
-        } else {
-            return Collections.emptyList();
-        }
-    }
-
-    public void setCubingJobId(String id) {
-        setParam(CUBING_JOB_ID, id);
-    }
-
-    private String getCubingJobId() {
-        return getParam(CUBING_JOB_ID);
-    }
-}



[05/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
new file mode 100644
index 0000000..31878a8
--- /dev/null
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.source.hive;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.engine.mr.HadoopUtil;
+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.TableDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * Management class to sync hive table metadata with command See main method for
+ * how to use the class
+ *
+ * @author jianliu
+ */
+public class HiveSourceTableLoader {
+
+    @SuppressWarnings("unused")
+    private static final Logger logger = LoggerFactory.getLogger(HiveSourceTableLoader.class);
+
+    public static final String OUTPUT_SURFIX = "json";
+    public static final String TABLE_FOLDER_NAME = "table";
+    public static final String TABLE_EXD_FOLDER_NAME = "table_exd";
+
+    public static Set<String> reloadHiveTables(String[] hiveTables, KylinConfig config) throws IOException {
+
+        Map<String, Set<String>> db2tables = Maps.newHashMap();
+        for (String table : hiveTables) {
+            String[] parts = HadoopUtil.parseHiveTableName(table);
+            Set<String> set = db2tables.get(parts[0]);
+            if (set == null) {
+                set = Sets.newHashSet();
+                db2tables.put(parts[0], set);
+            }
+            set.add(parts[1]);
+        }
+
+        // extract from hive
+        Set<String> loadedTables = Sets.newHashSet();
+        for (String database : db2tables.keySet()) {
+            List<String> loaded = extractHiveTables(database, db2tables.get(database), config);
+            loadedTables.addAll(loaded);
+        }
+
+        return loadedTables;
+    }
+
+    private static List<String> extractHiveTables(String database, Set<String> tables, KylinConfig config) throws IOException {
+
+        List<String> loadedTables = Lists.newArrayList();
+        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+        for (String tableName : tables) {
+            Table table = null;
+            HiveClient hiveClient = new HiveClient();
+            List<FieldSchema> partitionFields = null;
+            List<FieldSchema> fields = null;
+            try {
+                table = hiveClient.getHiveTable(database, tableName);
+                partitionFields = table.getPartitionKeys();
+                fields = hiveClient.getHiveTableFields(database, tableName);
+            } catch (Exception e) {
+                e.printStackTrace();
+                throw new IOException(e);
+            }
+
+            if (fields != null && partitionFields != null && partitionFields.size() > 0) {
+                fields.addAll(partitionFields);
+            }
+
+            long tableSize = hiveClient.getFileSizeForTable(table);
+            long tableFileNum = hiveClient.getFileNumberForTable(table);
+            TableDesc tableDesc = metaMgr.getTableDesc(database + "." + tableName);
+            if (tableDesc == null) {
+                tableDesc = new TableDesc();
+                tableDesc.setDatabase(database.toUpperCase());
+                tableDesc.setName(tableName.toUpperCase());
+                tableDesc.setUuid(UUID.randomUUID().toString());
+                tableDesc.setLastModified(0);
+            }
+
+            int columnNumber = fields.size();
+            List<ColumnDesc> columns = new ArrayList<ColumnDesc>(columnNumber);
+            for (int i = 0; i < columnNumber; i++) {
+                FieldSchema field = fields.get(i);
+                ColumnDesc cdesc = new ColumnDesc();
+                cdesc.setName(field.getName().toUpperCase());
+                cdesc.setDatatype(field.getType());
+                cdesc.setId(String.valueOf(i + 1));
+                columns.add(cdesc);
+            }
+            tableDesc.setColumns(columns.toArray(new ColumnDesc[columnNumber]));
+
+            StringBuffer partitionColumnString = new StringBuffer();
+            for (int i = 0, n = partitionFields.size(); i < n; i++) {
+                if (i > 0)
+                    partitionColumnString.append(", ");
+                partitionColumnString.append(partitionFields.get(i).getName().toUpperCase());
+            }
+
+            Map<String, String> map = metaMgr.getTableDescExd(tableDesc.getIdentity());
+
+            if (map == null) {
+                map = Maps.newHashMap();
+            }
+            map.put(MetadataConstants.TABLE_EXD_TABLENAME, table.getTableName());
+            map.put(MetadataConstants.TABLE_EXD_LOCATION, table.getSd().getLocation());
+            map.put(MetadataConstants.TABLE_EXD_IF, table.getSd().getInputFormat());
+            map.put(MetadataConstants.TABLE_EXD_OF, table.getSd().getOutputFormat());
+            map.put(MetadataConstants.TABLE_EXD_OWNER, table.getOwner());
+            map.put(MetadataConstants.TABLE_EXD_LAT, String.valueOf(table.getLastAccessTime()));
+            map.put(MetadataConstants.TABLE_EXD_PC, partitionColumnString.toString());
+            map.put(MetadataConstants.TABLE_EXD_TFS, String.valueOf(tableSize));
+            map.put(MetadataConstants.TABLE_EXD_TNF, String.valueOf(tableFileNum));
+            map.put(MetadataConstants.TABLE_EXD_PARTITIONED, Boolean.valueOf(partitionFields != null && partitionFields.size() > 0).toString());
+
+            metaMgr.saveSourceTable(tableDesc);
+            metaMgr.saveTableExd(tableDesc.getIdentity(), map);
+            loadedTables.add(tableDesc.getIdentity());
+        }
+
+
+        return loadedTables;
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTable.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTable.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTable.java
new file mode 100644
index 0000000..c09b6fc
--- /dev/null
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTable.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.source.hive;
+
+import java.io.IOException;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.engine.mr.DFSFileTable;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.source.ReadableTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class HiveTable implements ReadableTable {
+
+    private static final Logger logger = LoggerFactory.getLogger(HiveTable.class);
+
+    final private String database;
+    final private String hiveTable;
+    
+    private HiveClient hiveClient;
+
+    public HiveTable(TableDesc tableDesc) {
+        this.database = tableDesc.getDatabase();
+        this.hiveTable = tableDesc.getName();
+    }
+
+    @Override
+    public TableReader getReader() throws IOException {
+        return new HiveTableReader(database, hiveTable);
+    }
+
+    @Override
+    public TableSignature getSignature() throws IOException {
+        try {
+            String path = computeHDFSLocation();
+            Pair<Long, Long> sizeAndLastModified = DFSFileTable.getSizeAndLastModified(path);
+            long size = sizeAndLastModified.getFirst();
+            long lastModified = sizeAndLastModified.getSecond();
+
+            // for non-native hive table, cannot rely on size & last modified on HDFS
+            if (getHiveClient().isNativeTable(database, hiveTable) == false) {
+                lastModified = System.currentTimeMillis(); // assume table is ever changing
+            }
+
+            return new TableSignature(path, size, lastModified);
+
+        } catch (Exception e) {
+            if (e instanceof IOException)
+                throw (IOException) e;
+            else
+                throw new IOException(e);
+        }
+    }
+
+    private String computeHDFSLocation() throws Exception {
+
+        String override = KylinConfig.getInstanceFromEnv().getOverrideHiveTableLocation(hiveTable);
+        if (override != null) {
+            logger.debug("Override hive table location " + hiveTable + " -- " + override);
+            return override;
+        }
+
+        return getHiveClient().getHiveTableLocation(database, hiveTable);
+    }
+
+    public HiveClient getHiveClient() {
+
+        if (hiveClient == null) {
+            hiveClient = new HiveClient();
+        }
+        return hiveClient;
+    }
+
+    @Override
+    public String toString() {
+        return "hive: database=[" + database + "], table=[" + hiveTable + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableReader.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableReader.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableReader.java
new file mode 100644
index 0000000..35e24fe
--- /dev/null
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableReader.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.source.hive;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.hcatalog.common.HCatException;
+import org.apache.hive.hcatalog.data.HCatRecord;
+import org.apache.hive.hcatalog.data.transfer.DataTransferFactory;
+import org.apache.hive.hcatalog.data.transfer.HCatReader;
+import org.apache.hive.hcatalog.data.transfer.ReadEntity;
+import org.apache.hive.hcatalog.data.transfer.ReaderContext;
+import org.apache.kylin.source.ReadableTable.TableReader;
+
+/**
+ * An implementation of TableReader with HCatalog for Hive table.
+ */
+public class HiveTableReader implements TableReader {
+
+    private String dbName;
+    private String tableName;
+    private int currentSplit = -1;
+    private ReaderContext readCntxt = null;
+    private Iterator<HCatRecord> currentHCatRecordItr = null;
+    private HCatRecord currentHCatRecord;
+    private int numberOfSplits = 0;
+    private Map<String, String> partitionKV = null;
+
+    /**
+     * Constructor for reading whole hive table
+     * @param dbName
+     * @param tableName
+     * @throws IOException
+     */
+    public HiveTableReader(String dbName, String tableName) throws IOException {
+        this(dbName, tableName, null);
+    }
+
+    /**
+     * Constructor for reading a partition of the hive table
+     * @param dbName
+     * @param tableName
+     * @param partitionKV key-value pairs condition on the partition
+     * @throws IOException
+     */
+    public HiveTableReader(String dbName, String tableName, Map<String, String> partitionKV) throws IOException {
+        this.dbName = dbName;
+        this.tableName = tableName;
+        this.partitionKV = partitionKV;
+        initialize();
+    }
+
+    private void initialize() throws IOException {
+        try {
+            this.readCntxt = getHiveReaderContext(dbName, tableName, partitionKV);
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new IOException(e);
+        }
+
+        this.numberOfSplits = readCntxt.numSplits();
+
+//        HCatTableInfo tableInfo = HCatTableInfo.
+//        HCatSchema schema = HCatBaseInputFormat.getTableSchema(context.getConfiguration);
+    }
+
+    @Override
+    public boolean next() throws IOException {
+
+        while (currentHCatRecordItr == null || !currentHCatRecordItr.hasNext()) {
+            currentSplit++;
+            if (currentSplit == numberOfSplits) {
+                return false;
+            }
+
+            currentHCatRecordItr = loadHCatRecordItr(readCntxt, currentSplit);
+        }
+
+        currentHCatRecord = currentHCatRecordItr.next();
+
+        return true;
+    }
+
+    @Override
+    public String[] getRow() {
+        return getRowAsStringArray(currentHCatRecord);
+    }
+
+    public List<String> getRowAsList() {
+        return getRowAsList(currentHCatRecord);
+    }
+
+    public static List<String> getRowAsList(HCatRecord record, List<String> rowValues) {
+        List<Object> allFields = record.getAll();
+        for (Object o : allFields) {
+            rowValues.add((o == null) ? null : o.toString());
+        }
+        return rowValues;
+    }
+
+    public static List<String> getRowAsList(HCatRecord record) {
+        return Arrays.asList(getRowAsStringArray(record));
+    }
+
+    public static String[] getRowAsStringArray(HCatRecord record) {
+        String[] arr = new String[record.size()];
+        for (int i = 0; i < arr.length; i++) {
+            Object o = record.get(i);
+            arr[i] = (o == null) ? null : o.toString();
+        }
+        return arr;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.readCntxt = null;
+        this.currentHCatRecordItr = null;
+        this.currentHCatRecord = null;
+        this.currentSplit = -1;
+    }
+
+    public String toString() {
+        return "hive table reader for: " + dbName + "." + tableName;
+    }
+
+    private static ReaderContext getHiveReaderContext(String database, String table, Map<String, String> partitionKV) throws Exception {
+        HiveConf hiveConf = new HiveConf(HiveTableReader.class);
+        Iterator<Entry<String, String>> itr = hiveConf.iterator();
+        Map<String, String> map = new HashMap<String, String>();
+        while (itr.hasNext()) {
+            Entry<String, String> kv = itr.next();
+            map.put(kv.getKey(), kv.getValue());
+        }
+
+        ReadEntity entity;
+        if (partitionKV == null || partitionKV.size() == 0) {
+            entity = new ReadEntity.Builder().withDatabase(database).withTable(table).build();
+        } else {
+            entity = new ReadEntity.Builder().withDatabase(database).withTable(table).withPartition(partitionKV).build();
+        }
+
+        HCatReader reader = DataTransferFactory.getHCatReader(entity, map);
+        ReaderContext cntxt = reader.prepareRead();
+
+        return cntxt;
+    }
+
+    private static Iterator<HCatRecord> loadHCatRecordItr(ReaderContext readCntxt, int dataSplit) throws HCatException {
+        HCatReader currentHCatReader = DataTransferFactory.getHCatReader(readCntxt, dataSplit);
+
+        return currentHCatReader.read();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
new file mode 100644
index 0000000..a9e95a4
--- /dev/null
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.source.hive;
+
+import org.apache.kylin.engine.mr.IMRInput;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.source.ITableSource;
+import org.apache.kylin.source.ReadableTable;
+
+public class HiveTableSource implements ITableSource {
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <I> I adaptToBuildEngine(Class<I> engineInterface) {
+        if (engineInterface == IMRInput.class) {
+            return (I) new HiveMRInput();
+        } else {
+            throw new RuntimeException("Cannot adapt to " + engineInterface);
+        }
+    }
+
+    @Override
+    public ReadableTable createReadableTable(TableDesc tableDesc) {
+        return new HiveTable(tableDesc);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/source-hive/src/main/java/org/apache/kylin/source/hive/HqlExecutable.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HqlExecutable.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HqlExecutable.java
new file mode 100644
index 0000000..70ddb30
--- /dev/null
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HqlExecutable.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.source.hive;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+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.datanucleus.store.types.backed.HashMap;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.collect.Lists;
+
+import org.apache.kylin.common.util.JsonUtil;
+
+/**
+ */
+public class HqlExecutable extends AbstractExecutable {
+
+    private static final String HQL = "hql";
+    private static final String HIVE_CONFIG = "hive-config";
+
+    public HqlExecutable() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        try {
+            Map<String, String> configMap = getConfiguration();
+            HiveClient hiveClient = new HiveClient(configMap);
+            
+            for (String hql: getHqls()) {
+                hiveClient.executeHQL(hql);
+            }
+            return new ExecuteResult(ExecuteResult.State.SUCCEED);
+        } catch (Exception e) {
+            logger.error("error run hive query:" + getHqls(), e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+    
+    public void setConfiguration(Map<String, String> configMap) {
+        if(configMap != null) {
+            String configStr = "";
+            try {
+                configStr = JsonUtil.writeValueAsString(configMap);
+            } catch (JsonProcessingException e) {
+                e.printStackTrace();
+            }
+            setParam(HIVE_CONFIG, configStr);
+        }
+    }
+
+
+    @SuppressWarnings("unchecked")
+    private Map<String, String> getConfiguration() {
+        String configStr = getParam(HIVE_CONFIG);
+        Map<String, String> result = null;
+        if(configStr != null) {
+            try {
+                result = JsonUtil.readValue(configStr, HashMap.class);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+        
+        return result;
+    }
+    
+    public void setHqls(List<String> hqls) {
+        setParam(HQL, StringUtils.join(hqls, ";"));
+    }
+
+    private List<String> getHqls() {
+        final String hqls = getParam(HQL);
+        if (hqls != null) {
+            return Lists.newArrayList(StringUtils.split(hqls, ";"));
+        } else {
+            return Collections.emptyList();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/spark/src/main/java/org/apache/kylin/job/spark/AbstractSparkApplication.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/kylin/job/spark/AbstractSparkApplication.java b/spark/src/main/java/org/apache/kylin/job/spark/AbstractSparkApplication.java
index dd884d5..48a62c4 100644
--- a/spark/src/main/java/org/apache/kylin/job/spark/AbstractSparkApplication.java
+++ b/spark/src/main/java/org/apache/kylin/job/spark/AbstractSparkApplication.java
@@ -3,7 +3,7 @@ package org.apache.kylin.job.spark;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.kylin.job.tools.OptionsHelper;
+import org.apache.kylin.job.common.OptionsHelper;
 
 import java.io.Serializable;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/spark/src/main/java/org/apache/kylin/job/spark/SparkCountDemo.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/kylin/job/spark/SparkCountDemo.java b/spark/src/main/java/org/apache/kylin/job/spark/SparkCountDemo.java
index c648324..9946bb0 100644
--- a/spark/src/main/java/org/apache/kylin/job/spark/SparkCountDemo.java
+++ b/spark/src/main/java/org/apache/kylin/job/spark/SparkCountDemo.java
@@ -6,12 +6,13 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
-import org.apache.kylin.job.tools.OptionsHelper;
+import org.apache.kylin.job.common.OptionsHelper;
 import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.api.java.function.PairFunction;
 import org.apache.spark.storage.StorageLevel;
+
 import scala.Tuple2;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/spark/src/main/java/org/apache/kylin/job/spark/SparkHelloWorld.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/kylin/job/spark/SparkHelloWorld.java b/spark/src/main/java/org/apache/kylin/job/spark/SparkHelloWorld.java
index fee3583..30f9a53 100644
--- a/spark/src/main/java/org/apache/kylin/job/spark/SparkHelloWorld.java
+++ b/spark/src/main/java/org/apache/kylin/job/spark/SparkHelloWorld.java
@@ -1,7 +1,7 @@
 package org.apache.kylin.job.spark;
 
 import org.apache.commons.cli.Options;
-import org.apache.kylin.job.tools.OptionsHelper;
+import org.apache.kylin.job.common.OptionsHelper;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/spark/src/main/java/org/apache/kylin/job/spark/SparkHiveDemo.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/kylin/job/spark/SparkHiveDemo.java b/spark/src/main/java/org/apache/kylin/job/spark/SparkHiveDemo.java
index 48a1449..25873aa 100644
--- a/spark/src/main/java/org/apache/kylin/job/spark/SparkHiveDemo.java
+++ b/spark/src/main/java/org/apache/kylin/job/spark/SparkHiveDemo.java
@@ -1,7 +1,7 @@
 package org.apache.kylin.job.spark;
 
 import org.apache.commons.cli.Options;
-import org.apache.kylin.job.tools.OptionsHelper;
+import org.apache.kylin.job.common.OptionsHelper;
 import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.DataFrame;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/storage-hbase/pom.xml b/storage-hbase/pom.xml
index 013b009..3cdebbd 100644
--- a/storage-hbase/pom.xml
+++ b/storage-hbase/pom.xml
@@ -40,6 +40,11 @@
             <artifactId>kylin-engine-mr</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-invertedindex</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
 
         <!-- Env & Test -->
         <dependency>
@@ -81,6 +86,12 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>org.apache.mrunit</groupId>
+            <artifactId>mrunit</artifactId>
+            <classifier>hadoop2</classifier>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput.java
new file mode 100644
index 0000000..2447a39
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.IMROutput;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.storage.hbase.steps.HBaseMRSteps;
+
+public class HBaseMROutput implements IMROutput {
+
+    @Override
+    public IMRBatchCubingOutputSide getBatchCubingOutputSide(final CubeSegment seg) {
+        return new IMRBatchCubingOutputSide() {
+            HBaseMRSteps steps = new HBaseMRSteps(seg);
+
+            @Override
+            public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow, String cuboidRootPath) {
+                steps.addSaveCuboidToHTableSteps(jobFlow, cuboidRootPath);
+            }
+
+            @Override
+            public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
+                // nothing to do
+            }
+        };
+    }
+
+    @Override
+    public IMRBatchMergeOutputSide getBatchMergeOutputSide(final CubeSegment seg) {
+        return new IMRBatchMergeOutputSide() {
+            HBaseMRSteps steps = new HBaseMRSteps(seg);
+
+            @Override
+            public void addStepPhase2_BuildCube(DefaultChainedExecutable jobFlow, String cuboidRootPath) {
+                steps.addSaveCuboidToHTableSteps(jobFlow, cuboidRootPath);
+            }
+
+            @Override
+            public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow) {
+                jobFlow.addTask(steps.createMergeGCStep());
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
new file mode 100644
index 0000000..4f8c0e9
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseMROutput2.java
@@ -0,0 +1,297 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+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.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.mapreduce.TableMapper;
+import org.apache.hadoop.hbase.mapreduce.TableSplit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.engine.mr.IMROutput2;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.storage.hbase.steps.HBaseMRSteps;
+import org.apache.kylin.storage.hbase.steps.InMemKeyValueCreator;
+import org.apache.kylin.storage.hbase.steps.RowValueDecoder;
+
+import com.google.common.collect.Lists;
+
+public class HBaseMROutput2 implements IMROutput2 {
+
+    @Override
+    public IMRBatchCubingOutputSide2 getBatchCubingOutputSide(final CubeSegment seg) {
+        return new IMRBatchCubingOutputSide2() {
+            HBaseMRSteps steps = new HBaseMRSteps(seg);
+
+            @Override
+            public IMRStorageOutputFormat getStorageOutputFormat() {
+                return new HBaseOutputFormat(seg);
+            }
+
+            @Override
+            public void addStepPhase2_BuildDictionary(DefaultChainedExecutable jobFlow) {
+                jobFlow.addTask(steps.createCreateHTableStepWithStats(jobFlow.getId()));
+            }
+
+            @Override
+            public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow) {
+                jobFlow.addTask(steps.createBulkLoadStep(jobFlow.getId()));
+            }
+
+            @Override
+            public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
+                // nothing to do
+            }
+        };
+    }
+
+    @Override
+    public IMRBatchMergeInputSide2 getBatchMergeInputSide(final CubeSegment seg) {
+        return new IMRBatchMergeInputSide2() {
+            @Override
+            public IMRStorageInputFormat getStorageInputFormat() {
+                return new HBaseInputFormat(seg);
+            }
+        };
+    }
+
+    @Override
+    public IMRBatchMergeOutputSide2 getBatchMergeOutputSide(final CubeSegment seg) {
+        return new IMRBatchMergeOutputSide2() {
+            HBaseMRSteps steps = new HBaseMRSteps(seg);
+
+            @Override
+            public IMRStorageOutputFormat getStorageOutputFormat() {
+                return new HBaseOutputFormat(seg);
+            }
+
+            @Override
+            public void addStepPhase1_MergeDictionary(DefaultChainedExecutable jobFlow) {
+                jobFlow.addTask(steps.createCreateHTableStepWithStats(jobFlow.getId()));
+            }
+
+            @Override
+            public void addStepPhase2_BuildCube(DefaultChainedExecutable jobFlow) {
+                jobFlow.addTask(steps.createBulkLoadStep(jobFlow.getId()));
+            }
+
+            @Override
+            public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow) {
+                jobFlow.addTask(steps.createMergeGCStep());
+            }
+        };
+    }
+
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    private static class HBaseInputFormat implements IMRStorageInputFormat {
+        final Iterable<String> htables;
+
+        final RowValueDecoder[] rowValueDecoders;
+        final ByteArrayWritable parsedKey;
+        final Object[] parsedValue;
+        final Pair<ByteArrayWritable, Object[]> parsedPair;
+
+        public HBaseInputFormat(CubeSegment seg) {
+            this.htables = new HBaseMRSteps(seg).getMergingHTables();
+
+            List<RowValueDecoder> valueDecoderList = Lists.newArrayList();
+            List<MeasureDesc> measuresDescs = Lists.newArrayList();
+            for (HBaseColumnFamilyDesc cfDesc : seg.getCubeDesc().getHBaseMapping().getColumnFamily()) {
+                for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
+                    valueDecoderList.add(new RowValueDecoder(colDesc));
+                    for (MeasureDesc measure : colDesc.getMeasures()) {
+                        measuresDescs.add(measure);
+                    }
+                }
+            }
+            this.rowValueDecoders = valueDecoderList.toArray(new RowValueDecoder[valueDecoderList.size()]);
+
+            this.parsedKey = new ByteArrayWritable();
+            this.parsedValue = new Object[measuresDescs.size()];
+            this.parsedPair = new Pair<ByteArrayWritable, Object[]>(parsedKey, parsedValue);
+        }
+
+        @Override
+        public void configureInput(Class<? extends Mapper> mapper, Class<? extends WritableComparable> outputKeyClass, Class<? extends Writable> outputValueClass, Job job) throws IOException {
+            Configuration conf = job.getConfiguration();
+            HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
+            
+            List<Scan> scans = new ArrayList<Scan>();
+            for (String htable : htables) {
+                Scan scan = new Scan();
+                scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs
+                scan.setCacheBlocks(false); // don't set to true for MR jobs
+                scan.setAttribute(Scan.SCAN_ATTRIBUTES_TABLE_NAME, Bytes.toBytes(htable));
+                scans.add(scan);
+            }
+            TableMapReduceUtil.initTableMapperJob(scans, (Class<? extends TableMapper>) mapper, outputKeyClass, outputValueClass, job);
+        }
+
+        @Override
+        public CubeSegment findSourceSegment(Context context, CubeInstance cubeInstance) throws IOException {
+            TableSplit currentSplit = (TableSplit) context.getInputSplit();
+            byte[] tableName = currentSplit.getTableName();
+            String htableName = Bytes.toString(tableName);
+
+            // decide which source segment
+            for (CubeSegment segment : cubeInstance.getSegments()) {
+                String segmentHtable = segment.getStorageLocationIdentifier();
+                if (segmentHtable != null && segmentHtable.equalsIgnoreCase(htableName)) {
+                    return segment;
+                }
+            }
+            throw new IllegalStateException("No merging segment's storage location identifier equals " + htableName);
+        }
+
+        @Override
+        public Pair<ByteArrayWritable, Object[]> parseMapperInput(Object inKey, Object inValue) {
+            ImmutableBytesWritable key = (ImmutableBytesWritable) inKey;
+            parsedKey.set(key.get(), key.getOffset(), key.getLength());
+
+            Result value = (Result) inValue;
+            int position = 0;
+            for (int i = 0; i < rowValueDecoders.length; i++) {
+                rowValueDecoders[i].decode(value, false);
+                Object[] measureValues = rowValueDecoders[i].getValues();
+                for (int j = 0; j < measureValues.length; j++) {
+                    parsedValue[position++] = measureValues[j];
+                }
+            }
+
+            return parsedPair;
+        }
+    }
+
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    private static class HBaseOutputFormat implements IMRStorageOutputFormat {
+        final CubeSegment seg;
+
+        final List<InMemKeyValueCreator> keyValueCreators = Lists.newArrayList();
+        final ImmutableBytesWritable outputKey = new ImmutableBytesWritable();
+
+        public HBaseOutputFormat(CubeSegment seg) {
+            this.seg = seg;
+        }
+
+        @Override
+        public void configureOutput(Class<? extends Reducer> reducer, String jobFlowId, Job job) throws IOException {
+            Path hfilePath = new Path(new HBaseMRSteps(seg).getHFilePath(jobFlowId));
+            FileOutputFormat.setOutputPath(job, hfilePath);
+
+            String htableName = seg.getStorageLocationIdentifier();
+            Configuration conf = HBaseConfiguration.create(job.getConfiguration());
+            HTable htable = new HTable(conf, htableName);
+            HFileOutputFormat.configureIncrementalLoad(job, htable);
+
+            // set Reducer; This need be after configureIncrementalLoad, to overwrite the default reducer class
+            job.setReducerClass(reducer);
+
+            // kylin uses ByteArrayWritable instead of ImmutableBytesWritable as mapper output key
+            rewriteTotalOrderPartitionerFile(job);
+
+            HadoopUtil.deletePath(job.getConfiguration(), hfilePath);
+        }
+
+        @Override
+        public void doReducerOutput(ByteArrayWritable key, Object[] value, Reducer.Context context) throws IOException, InterruptedException {
+            if (keyValueCreators.size() == 0) {
+                int startPosition = 0;
+                for (HBaseColumnFamilyDesc cfDesc : seg.getCubeDesc().getHBaseMapping().getColumnFamily()) {
+                    for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
+                        keyValueCreators.add(new InMemKeyValueCreator(colDesc, startPosition));
+                        startPosition += colDesc.getMeasures().length;
+                    }
+                }
+            }
+
+            outputKey.set(key.array(), key.offset(), key.length());
+
+            KeyValue outputValue;
+            for (int i = 0; i < keyValueCreators.size(); i++) {
+                outputValue = keyValueCreators.get(i).create(key.array(), key.offset(), key.length(), value);
+                context.write(outputKey, outputValue);
+            }
+        }
+
+        private void rewriteTotalOrderPartitionerFile(Job job) throws IOException {
+            Configuration conf = job.getConfiguration();
+            String partitionsFile = TotalOrderPartitioner.getPartitionFile(conf);
+            if (StringUtils.isBlank(partitionsFile))
+                throw new IllegalStateException("HFileOutputFormat.configureIncrementalLoad don't configure TotalOrderPartitioner any more?");
+
+            Path partitionsPath = new Path(partitionsFile);
+
+            // read in partition file in ImmutableBytesWritable
+            List<ByteArrayWritable> keys = Lists.newArrayList();
+            Reader reader = new SequenceFile.Reader(conf, Reader.file(partitionsPath));
+            try {
+                ImmutableBytesWritable key = new ImmutableBytesWritable();
+                while (reader.next(key, NullWritable.get())) {
+                    keys.add(new ByteArrayWritable(key.copyBytes()));
+                }
+            } finally {
+                reader.close();
+            }
+
+            // write out again in ByteArrayWritable
+            Writer writer = SequenceFile.createWriter(conf, Writer.file(partitionsPath), Writer.keyClass(ByteArrayWritable.class), Writer.valueClass(NullWritable.class));
+            try {
+                for (ByteArrayWritable key : keys) {
+                    writer.append(key, NullWritable.get());
+                }
+            } finally {
+                writer.close();
+            }
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseRegionSizeCalculator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseRegionSizeCalculator.java
deleted file mode 100644
index bd41a99..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseRegionSizeCalculator.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-/** This class will come with HBase 2.0 in package org.apache.hadoop.hbase.util **/
-package org.apache.kylin.storage.hbase;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.RegionLoad;
-import org.apache.hadoop.hbase.ServerLoad;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class HBaseRegionSizeCalculator {
-
-    private static final Logger logger = LoggerFactory.getLogger(HBaseRegionSizeCalculator.class);
-
-    /**
-     * Maps each region to its size in bytes.
-     **/
-    private final Map<byte[], Long> sizeMap = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
-
-    static final String ENABLE_REGIONSIZECALCULATOR = "hbase.regionsizecalculator.enable";
-
-    /**
-     * Computes size of each region for table and given column families.
-     * */
-    public HBaseRegionSizeCalculator(HTable table) throws IOException {
-        this(table, new HBaseAdmin(table.getConfiguration()));
-    }
-
-    /** Constructor for unit testing */
-    HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
-
-        try {
-            if (!enabled(table.getConfiguration())) {
-                logger.info("Region size calculation disabled.");
-                return;
-            }
-
-            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
-
-            // Get regions for table.
-            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
-            Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
-
-            for (HRegionInfo regionInfo : tableRegionInfos) {
-                tableRegions.add(regionInfo.getRegionName());
-            }
-
-            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
-            Collection<ServerName> servers = clusterStatus.getServers();
-            final long megaByte = 1024L * 1024L;
-
-            // Iterate all cluster regions, filter regions from our table and
-            // compute their size.
-            for (ServerName serverName : servers) {
-                ServerLoad serverLoad = clusterStatus.getLoad(serverName);
-
-                for (RegionLoad regionLoad : serverLoad.getRegionsLoad().values()) {
-                    byte[] regionId = regionLoad.getName();
-
-                    if (tableRegions.contains(regionId)) {
-
-                        long regionSizeBytes = regionLoad.getStorefileSizeMB() * megaByte;
-                        sizeMap.put(regionId, regionSizeBytes);
-
-                        // logger.info("Region " + regionLoad.getNameAsString()
-                        // + " has size " + regionSizeBytes);
-                    }
-                }
-            }
-        } finally {
-            hBaseAdmin.close();
-        }
-
-    }
-
-    boolean enabled(Configuration configuration) {
-        return configuration.getBoolean(ENABLE_REGIONSIZECALCULATOR, true);
-    }
-
-    /**
-     * Returns size of given region in bytes. Returns 0 if region was not found.
-     **/
-    public long getRegionSize(byte[] regionId) {
-        Long size = sizeMap.get(regionId);
-        if (size == null) {
-            logger.info("Unknown region:" + Arrays.toString(regionId));
-            return 0;
-        } else {
-            return size;
-        }
-    }
-
-    public Map<byte[], Long> getRegionSizeMap() {
-        return Collections.unmodifiableMap(sizeMap);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
new file mode 100644
index 0000000..3bd0300
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase;
+
+import org.apache.kylin.engine.mr.IMROutput;
+import org.apache.kylin.engine.mr.IMROutput2;
+import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.storage.IStorage;
+import org.apache.kylin.storage.IStorageQuery;
+
+public class HBaseStorage implements IStorage {
+
+    @Override
+    public IStorageQuery createStorageQuery(IRealization realization) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <I> I adaptToBuildEngine(Class<I> engineInterface) {
+        if (engineInterface == IMROutput.class) {
+            return (I) new HBaseMROutput();
+        } else if (engineInterface == IMROutput2.class) {
+            return (I) new HBaseMROutput2();
+        } else {
+            throw new RuntimeException("Cannot adapt to " + engineInterface);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/RowValueDecoder.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/RowValueDecoder.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/RowValueDecoder.java
deleted file mode 100644
index 30d5613..0000000
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/RowValueDecoder.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage.hbase;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.metadata.measure.DoubleMutable;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-import java.nio.ByteBuffer;
-import java.util.BitSet;
-import java.util.Collection;
-
-/**
- */
-public class RowValueDecoder implements Cloneable {
-
-    private final HBaseColumnDesc hbaseColumn;
-    private final byte[] hbaseColumnFamily;
-    private final byte[] hbaseColumnQualifier;
-
-    private final MeasureCodec codec;
-    private final BitSet projectionIndex;
-    private final MeasureDesc[] measures;
-    private Object[] values;
-
-    public RowValueDecoder(HBaseColumnDesc hbaseColumn) {
-        this.hbaseColumn = hbaseColumn;
-        this.hbaseColumnFamily = Bytes.toBytes(hbaseColumn.getColumnFamilyName());
-        this.hbaseColumnQualifier = Bytes.toBytes(hbaseColumn.getQualifier());
-        this.projectionIndex = new BitSet();
-        this.measures = hbaseColumn.getMeasures();
-        this.codec = new MeasureCodec(measures);
-        this.values = new Object[measures.length];
-    }
-
-    public void decode(Result hbaseRow) {
-        decode(hbaseRow, true);
-    }
-
-    public void decode(Result hbaseRow, boolean convertToJavaObject) {
-        decode(hbaseRow.getValueAsByteBuffer(hbaseColumnFamily, hbaseColumnQualifier), convertToJavaObject);
-    }
-
-    public void decode(byte[] bytes) {
-        decode(bytes, true);
-    }
-
-    public void decode(byte[] bytes, boolean convertToJavaObject) {
-        decode(ByteBuffer.wrap(bytes), convertToJavaObject);
-    }
-
-    private void decode(ByteBuffer buffer, boolean convertToJavaObject) {
-        codec.decode(buffer, values);
-        if (convertToJavaObject) {
-            convertToJavaObjects(values, values, convertToJavaObject);
-        }
-    }
-
-    private void convertToJavaObjects(Object[] mapredObjs, Object[] results, boolean convertToJavaObject) {
-        for (int i = 0; i < mapredObjs.length; i++) {
-            Object o = mapredObjs[i];
-
-            if (o instanceof LongMutable)
-                o = ((LongMutable) o).get();
-            else if (o instanceof DoubleMutable)
-                o = ((DoubleMutable) o).get();
-
-            results[i] = o;
-        }
-    }
-
-    public void setIndex(int bitIndex) {
-        projectionIndex.set(bitIndex);
-    }
-
-    public HBaseColumnDesc getHBaseColumn() {
-        return hbaseColumn;
-    }
-
-    public BitSet getProjectionIndex() {
-        return projectionIndex;
-    }
-
-    public Object[] getValues() {
-        return values;
-    }
-
-    public MeasureDesc[] getMeasures() {
-        return measures;
-    }
-
-    public boolean hasMemHungryCountDistinct() {
-        for (int i = projectionIndex.nextSetBit(0); i >= 0; i = projectionIndex.nextSetBit(i + 1)) {
-            FunctionDesc func = measures[i].getFunction();
-            if (func.isCountDistinct() && !func.isHolisticCountDistinct()) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    public static boolean hasMemHungryCountDistinct(Collection<RowValueDecoder> rowValueDecoders) {
-        for (RowValueDecoder decoder : rowValueDecoders) {
-            if (decoder.hasMemHungryCountDistinct())
-                return true;
-        }
-        return false;
-    }
-
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
new file mode 100644
index 0000000..b028391
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase.steps;
+
+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.HBaseConfiguration;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class BulkLoadJob extends AbstractHadoopJob {
+
+    protected static final Logger log = LoggerFactory.getLogger(BulkLoadJob.class);
+
+    @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_CUBE_NAME);
+            parseOptions(options, args);
+
+            String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
+            // e.g
+            // /tmp/kylin-3f150b00-3332-41ca-9d3d-652f67f044d7/test_kylin_cube_with_slr_ready_2_segments/hfile/
+            // end with "/"
+            String input = getOptionValue(OPTION_INPUT_PATH);
+
+            Configuration conf = HBaseConfiguration.create(getConf());
+            FileSystem fs = FileSystem.get(conf);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            CubeManager cubeMgr = CubeManager.getInstance(config);
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            CubeDesc cubeDesc = cube.getDescriptor();
+            FsPermission permission = new FsPermission((short) 0777);
+            for (HBaseColumnFamilyDesc cf : cubeDesc.getHBaseMapping().getColumnFamily()) {
+                String cfName = cf.getName();
+                Path columnFamilyPath = new Path(input + cfName);
+
+                // File may have already been auto-loaded (in the case of MapR DB)
+                if(fs.exists(columnFamilyPath)) {
+                    fs.setPermission(columnFamilyPath, permission);
+                }
+            }
+
+            String[] newArgs = new String[2];
+            newArgs[0] = input;
+            newArgs[1] = tableName;
+
+            log.debug("Start to run LoadIncrementalHFiles");
+            int ret = ToolRunner.run(new LoadIncrementalHFiles(conf), newArgs);
+            log.debug("End to run LoadIncrementalHFiles");
+            return ret;
+        } catch (Exception e) {
+            printUsage(options);
+            throw e;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new BulkLoadJob(), args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
new file mode 100644
index 0000000..baa3efe
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
@@ -0,0 +1,314 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase.steps;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ */
+public class CreateHTableJob extends AbstractHadoopJob {
+
+    protected static final Logger logger = LoggerFactory.getLogger(CreateHTableJob.class);
+
+    CubeInstance cube = null;
+    CubeDesc cubeDesc = null;
+    String segmentName = null;
+    KylinConfig kylinConfig;
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        options.addOption(OPTION_CUBE_NAME);
+        options.addOption(OPTION_SEGMENT_NAME);
+        options.addOption(OPTION_PARTITION_FILE_PATH);
+        options.addOption(OPTION_HTABLE_NAME);
+        options.addOption(OPTION_STATISTICS_ENABLED);
+        parseOptions(options, args);
+
+        Path partitionFilePath = new Path(getOptionValue(OPTION_PARTITION_FILE_PATH));
+        boolean statistics_enabled = Boolean.parseBoolean(getOptionValue(OPTION_STATISTICS_ENABLED));
+
+        String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+        kylinConfig = KylinConfig.getInstanceFromEnv();
+        CubeManager cubeMgr = CubeManager.getInstance(kylinConfig);
+        cube = cubeMgr.getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+        segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+        CubeSegment cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
+
+        String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
+        Configuration conf = HBaseConfiguration.create(getConf());
+
+        try {
+
+            byte[][] splitKeys;
+            if (statistics_enabled) {
+                List<Integer> rowkeyColumnSize = Lists.newArrayList();
+                long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+                Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+                List<TblColRef> columnList = baseCuboid.getColumns();
+
+                for (int i = 0; i < columnList.size(); i++) {
+                    logger.info("Rowkey column " + i + " length " + cubeSegment.getColumnLength(columnList.get(i)));
+                    rowkeyColumnSize.add(cubeSegment.getColumnLength(columnList.get(i)));
+                }
+
+                splitKeys = getSplitsFromCuboidStatistics(conf, kylinConfig, rowkeyColumnSize, cubeSegment);
+            } else {
+                splitKeys = getSplits(conf, partitionFilePath);
+            }
+
+            CubeHTableUtil.createHTable(cubeDesc, tableName, splitKeys);
+            return 0;
+        } catch (Exception e) {
+            printUsage(options);
+            e.printStackTrace(System.err);
+            logger.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    public byte[][] getSplits(Configuration conf, Path path) throws Exception {
+        FileSystem fs = path.getFileSystem(conf);
+        if (fs.exists(path) == false) {
+            System.err.println("Path " + path + " not found, no region split, HTable will be one region");
+            return null;
+        }
+
+        List<byte[]> rowkeyList = new ArrayList<byte[]>();
+        SequenceFile.Reader reader = null;
+        try {
+            reader = new SequenceFile.Reader(fs, path, conf);
+            Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+            Writable value = (Writable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
+            while (reader.next(key, value)) {
+                rowkeyList.add(((Text) key).copyBytes());
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw e;
+        } finally {
+            IOUtils.closeStream(reader);
+        }
+
+        logger.info((rowkeyList.size() + 1) + " regions");
+        logger.info(rowkeyList.size() + " splits");
+        for (byte[] split : rowkeyList) {
+            logger.info(StringUtils.byteToHexString(split));
+        }
+
+        byte[][] retValue = rowkeyList.toArray(new byte[rowkeyList.size()][]);
+        return retValue.length == 0 ? null : retValue;
+    }
+
+
+    @SuppressWarnings("deprecation")
+    public static byte[][] getSplitsFromCuboidStatistics(Configuration conf, KylinConfig kylinConfig,  List<Integer> rowkeyColumnSize, CubeSegment cubeSegment) throws IOException {
+
+        CubeDesc cubeDesc = cubeSegment.getCubeDesc();
+        DataModelDesc.RealizationCapacity cubeCapacity = cubeDesc.getModel().getCapacity();
+        int cut = kylinConfig.getHBaseRegionCut(cubeCapacity.toString());
+
+        logger.info("Cube capacity " + cubeCapacity.toString() + ", chosen cut for HTable is " + cut + "GB");
+
+        Map<Long, Long> cuboidSizeMap = Maps.newHashMap();
+        long totalSizeInM = 0;
+
+        ResourceStore rs = ResourceStore.getStore(kylinConfig);
+        String fileKey = cubeSegment.getStatisticsResourcePath();
+        InputStream is = rs.getResource(fileKey);
+        File tempFile = null;
+        FileOutputStream tempFileStream = null;
+        try {
+            tempFile = File.createTempFile(cubeSegment.getUuid(), ".seq");
+            tempFileStream = new FileOutputStream(tempFile);
+            org.apache.commons.io.IOUtils.copy(is, tempFileStream);
+        } finally {
+            IOUtils.closeStream(is);
+            IOUtils.closeStream(tempFileStream);
+        }
+
+        FileSystem fs = HadoopUtil.getFileSystem("file:///" + tempFile.getAbsolutePath());
+        SequenceFile.Reader reader = null;
+        try {
+            reader = new SequenceFile.Reader(fs, new Path(tempFile.getAbsolutePath()), conf);
+            LongWritable key = (LongWritable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+            BytesWritable value = (BytesWritable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
+            int samplingPercentage = 25;
+            while (reader.next(key, value)) {
+                if (key.get() == 0l) {
+                    samplingPercentage = Bytes.toInt(value.getBytes());
+                } else {
+                    HyperLogLogPlusCounter hll = new HyperLogLogPlusCounter(14);
+                    ByteArray byteArray = new ByteArray(value.getBytes());
+                    hll.readRegisters(byteArray.asBuffer());
+
+                    cuboidSizeMap.put(key.get(), hll.getCountEstimate() * 100 / samplingPercentage);
+                }
+
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw e;
+        } finally {
+            IOUtils.closeStream(reader);
+        }
+
+        List<Long> allCuboids = Lists.newArrayList();
+        allCuboids.addAll(cuboidSizeMap.keySet());
+        Collections.sort(allCuboids);
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        for (long cuboidId : allCuboids) {
+            long cuboidSize = estimateCuboidStorageSize(cubeDesc, cuboidId, cuboidSizeMap.get(cuboidId), baseCuboidId, rowkeyColumnSize);
+            cuboidSizeMap.put(cuboidId, cuboidSize);
+            totalSizeInM += cuboidSize;
+        }
+
+        int nRegion = Math.round((float) totalSizeInM / ((float) cut * 1024l));
+        nRegion = Math.max(kylinConfig.getHBaseRegionCutMin(), nRegion);
+        nRegion = Math.min(kylinConfig.getHBaseRegionCutMax(), nRegion);
+
+        int mbPerRegion = (int) (totalSizeInM / (nRegion));
+        mbPerRegion = Math.max(1, mbPerRegion);
+
+        logger.info("Total size " + totalSizeInM + "M (estimated)");
+        logger.info(nRegion + " regions (estimated)");
+        logger.info(mbPerRegion + " MB per region (estimated)");
+
+        List<Long> regionSplit = Lists.newArrayList();
+
+
+        long size = 0;
+        int regionIndex = 0;
+        int cuboidCount = 0;
+        for (int i = 0; i < allCuboids.size(); i++) {
+            long cuboidId = allCuboids.get(i);
+            if (size >= mbPerRegion || (size + cuboidSizeMap.get(cuboidId)) >= mbPerRegion * 1.2) {
+                // if the size already bigger than threshold, or it will exceed by 20%, cut for next region
+                regionSplit.add(cuboidId);
+                logger.info("Region " + regionIndex + " will be " + size + " MB, contains cuboids < " + cuboidId + " (" + cuboidCount + ") cuboids");
+                size = 0;
+                cuboidCount = 0;
+                regionIndex++;
+            }
+            size += cuboidSizeMap.get(cuboidId);
+            cuboidCount++;
+        }
+
+
+        byte[][] result = new byte[regionSplit.size()][];
+        for (int i = 0; i < regionSplit.size(); i++) {
+            result[i] = Bytes.toBytes(regionSplit.get(i));
+        }
+
+        return result;
+    }
+
+    /**
+     * Estimate the cuboid's size
+     *
+     * @param cubeDesc
+     * @param cuboidId
+     * @param rowCount
+     * @return the cuboid size in M bytes
+     */
+    private static long estimateCuboidStorageSize(CubeDesc cubeDesc, long cuboidId, long rowCount, long baseCuboidId, List<Integer> rowKeyColumnLength) {
+
+        int bytesLength = RowConstants.ROWKEY_CUBOIDID_LEN;
+
+        long mask = Long.highestOneBit(baseCuboidId);
+        long parentCuboidIdActualLength = Long.SIZE - Long.numberOfLeadingZeros(baseCuboidId);
+        for (int i = 0; i < parentCuboidIdActualLength; i++) {
+            if ((mask & cuboidId) > 0) {
+                bytesLength += rowKeyColumnLength.get(i); //colIO.getColumnLength(columnList.get(i));
+            }
+            mask = mask >> 1;
+        }
+
+        // add the measure length
+        int space = 0;
+        for (MeasureDesc measureDesc : cubeDesc.getMeasures()) {
+            DataType returnType = measureDesc.getFunction().getReturnDataType();
+            if (returnType.isHLLC()) {
+                // for HLL, it will be compressed when export to bytes
+                space += returnType.getSpaceEstimate() * 0.75;
+            } else {
+                space += returnType.getSpaceEstimate();
+            }
+        }
+        bytesLength += space;
+
+        logger.info("Cuboid " + cuboidId + " has " + rowCount + " rows, each row size is " + bytesLength + " bytes.");
+        logger.info("Cuboid " + cuboidId + " total size is " + (bytesLength * rowCount / (1024L * 1024L)) + "M.");
+        return bytesLength * rowCount / (1024L * 1024L);
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new CreateHTableJob(), args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.java
new file mode 100644
index 0000000..aa37826
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.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.storage.hbase.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class CubeHFileJob extends AbstractHadoopJob {
+
+    protected static final Logger logger = LoggerFactory.getLogger(CubeHFileJob.class);
+
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_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));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            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(CubeHFileMapper.class);
+            job.setReducerClass(KeyValueSortReducer.class);
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            Configuration conf = HBaseConfiguration.create(getConf());
+            // add metadata to distributed cache
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
+            HTable htable = new HTable(conf, tableName);
+
+            //Automatic config !
+            HFileOutputFormat.configureIncrementalLoad(job, htable);
+
+            // set block replication to 3 for hfiles
+            conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, "3");
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            logger.error("error in CubeHFileJob", e);
+            printUsage(options);
+            throw e;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new CubeHFileJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper.java
new file mode 100644
index 0000000..65b56b9
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase.steps;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.metadata.measure.MeasureCodec;
+
+import com.google.common.collect.Lists;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CubeHFileMapper extends KylinMapper<Text, Text, ImmutableBytesWritable, KeyValue> {
+
+    ImmutableBytesWritable outputKey = new ImmutableBytesWritable();
+
+    String cubeName;
+    CubeDesc cubeDesc;
+
+    MeasureCodec inputCodec;
+    Object[] inputMeasures;
+    List<KeyValueCreator> keyValueCreators;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME);
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        CubeManager cubeMgr = CubeManager.getInstance(config);
+        cubeDesc = cubeMgr.getCube(cubeName).getDescriptor();
+
+        inputCodec = new MeasureCodec(cubeDesc.getMeasures());
+        inputMeasures = new Object[cubeDesc.getMeasures().size()];
+        keyValueCreators = Lists.newArrayList();
+
+        for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
+            for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
+                keyValueCreators.add(new KeyValueCreator(cubeDesc, colDesc));
+            }
+        }
+    }
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        outputKey.set(key.getBytes(), 0, key.getLength());
+        KeyValue outputValue;
+
+        int n = keyValueCreators.size();
+        if (n == 1 && keyValueCreators.get(0).isFullCopy) { // shortcut for
+                                                            // simple full copy
+
+            outputValue = keyValueCreators.get(0).create(key, value.getBytes(), 0, value.getLength());
+            context.write(outputKey, outputValue);
+
+        } else { // normal (complex) case that distributes measures to multiple HBase columns
+
+            inputCodec.decode(ByteBuffer.wrap(value.getBytes(), 0, value.getLength()), inputMeasures);
+
+            for (int i = 0; i < n; i++) {
+                outputValue = keyValueCreators.get(i).create(key, inputMeasures);
+                context.write(outputKey, outputValue);
+            }
+        }
+    }
+
+}



[02/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRawScanner.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRawScanner.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRawScanner.java
deleted file mode 100644
index 0313026..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRawScanner.java
+++ /dev/null
@@ -1,111 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.storage.gridtable.IGTStore.IGTStoreScanner;
-
-public class GTRawScanner implements IGTScanner {
-
-    final GTInfo info;
-    final IGTStoreScanner storeScanner;
-    final ImmutableBitSet selectedColBlocks;
-
-    private GTRowBlock.Reader curBlockReader;
-    private GTRecord next;
-    final private GTRecord oneRecord; // avoid instance creation
-
-    private int scannedRowCount = 0;
-    private int scannedRowBlockCount = 0;
-
-    public GTRawScanner(GTInfo info, IGTStore store, GTScanRequest req) throws IOException {
-        this.info = info;
-        this.selectedColBlocks = info.selectColumnBlocks(req.getColumns());
-        this.storeScanner = store.scan(req.getPkStart(), req.getPkEnd(), selectedColBlocks, req);
-        this.oneRecord = new GTRecord(info);
-    }
-
-    @Override
-    public GTInfo getInfo() {
-        return info;
-    }
-
-    @Override
-    public int getScannedRowCount() {
-        return scannedRowCount;
-    }
-
-    @Override
-    public int getScannedRowBlockCount() {
-        return scannedRowBlockCount;
-    }
-
-    @Override
-    public void close() throws IOException {
-        storeScanner.close();
-    }
-
-    @Override
-    public Iterator<GTRecord> iterator() {
-        return new Iterator<GTRecord>() {
-
-            @Override
-            public boolean hasNext() {
-                if (next != null)
-                    return true;
-
-                if (fetchOneRecord()) {
-                    next = oneRecord;
-                    return true;
-                } else {
-                    return false;
-                }
-            }
-
-            private boolean fetchOneRecord() {
-                while (true) {
-                    // get a block
-                    if (curBlockReader == null) {
-                        if (storeScanner.hasNext()) {
-                            curBlockReader = storeScanner.next().getReader(selectedColBlocks);
-                            scannedRowBlockCount++;
-                        } else {
-                            return false;
-                        }
-                    }
-                    // if block exhausted, try next block
-                    if (curBlockReader.hasNext() == false) {
-                        curBlockReader = null;
-                        continue;
-                    }
-                    // fetch a row
-                    curBlockReader.fetchNext(oneRecord);
-                    scannedRowCount++;
-                    return true;
-                }
-            }
-
-            @Override
-            public GTRecord next() {
-                // fetch next record
-                if (next == null) {
-                    hasNext();
-                    if (next == null)
-                        throw new NoSuchElementException();
-                }
-
-                GTRecord result = next;
-                next = null;
-                return result;
-            }
-
-            @Override
-            public void remove() {
-                throw new UnsupportedOperationException();
-            }
-        };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRecord.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRecord.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRecord.java
deleted file mode 100644
index fbb6171..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRecord.java
+++ /dev/null
@@ -1,285 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.BitSet;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.ImmutableBitSet;
-
-public class GTRecord implements Comparable<GTRecord> {
-
-    final GTInfo info;
-    final ByteArray[] cols;
-
-    private ImmutableBitSet maskForEqualHashComp;
-
-    public GTRecord(GTInfo info, ImmutableBitSet maskForEqualHashComp) {
-        this.info = info;
-        this.cols = new ByteArray[info.getColumnCount()];
-        for (int i = 0; i < cols.length; i++) {
-            if (maskForEqualHashComp.get(i)) {
-                this.cols[i] = new ByteArray();
-            }
-        }
-        this.maskForEqualHashComp = maskForEqualHashComp;
-    }
-
-    public GTRecord(GTInfo info) {
-        this(info, info.colAll);
-    }
-
-    public GTInfo getInfo() {
-        return info;
-    }
-
-    public ByteArray get(int i) {
-        return cols[i];
-    }
-
-    public void set(int i, ByteArray data) {
-        cols[i].set(data.array(), data.offset(), data.length());
-    }
-
-    /** set record to the codes of specified values, new space allocated to hold the codes */
-    public GTRecord setValues(Object... values) {
-        setValues(info.colAll, new ByteArray(info.getMaxRecordLength()), values);
-        return this;
-    }
-
-    /** set record to the codes of specified values, reuse given space to hold the codes */
-    public GTRecord setValues(ImmutableBitSet selectedCols, ByteArray space, Object... values) {
-        assert selectedCols.cardinality() == values.length;
-        
-        ByteBuffer buf = space.asBuffer();
-        int pos = buf.position();
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            info.codeSystem.encodeColumnValue(c, values[i], buf);
-            int newPos = buf.position();
-            cols[c].set(buf.array(), buf.arrayOffset() + pos, newPos - pos);
-            pos = newPos;
-        }
-        return this;
-    }
-
-    /** decode and return the values of this record */
-    public Object[] getValues() {
-        return getValues(info.colAll, new Object[info.getColumnCount()]);
-    }
-
-    /** decode and return the values of this record */
-    public Object[] getValues(ImmutableBitSet selectedCols, Object[] result) {
-        assert selectedCols.cardinality() == result.length;
-        
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            if (cols[c] == null || cols[c].array() == null) {
-                result[i] = null;
-            } else {
-                result[i] = info.codeSystem.decodeColumnValue(c, cols[c].asBuffer());
-            }
-        }
-        return result;
-    }
-
-    public Object[] getValues(int[] selectedColumns, Object[] result) {
-        assert selectedColumns.length <= result.length;
-        for (int i = 0; i < selectedColumns.length; i++) {
-            int c = selectedColumns[i];
-            if (cols[c].array() == null) {
-                result[i] = null;
-            } else {
-                result[i] = info.codeSystem.decodeColumnValue(c, cols[c].asBuffer());
-            }
-        }
-        return result;
-    }
-
-    public GTRecord copy() {
-        return copy(info.colAll);
-    }
-
-    public GTRecord copy(ImmutableBitSet selectedCols) {
-        int len = 0;
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            len += cols[c].length();
-        }
-
-        byte[] space = new byte[len];
-
-        GTRecord copy = new GTRecord(info, this.maskForEqualHashComp);
-        int pos = 0;
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            System.arraycopy(cols[c].array(), cols[c].offset(), space, pos, cols[c].length());
-            copy.cols[c].set(space, pos, cols[c].length());
-            pos += cols[c].length();
-        }
-
-        return copy;
-    }
-
-    public ImmutableBitSet maskForEqualHashComp() {
-        return maskForEqualHashComp;
-    }
-
-    public void maskForEqualHashComp(ImmutableBitSet set) {
-        this.maskForEqualHashComp = set;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-
-        GTRecord o = (GTRecord) obj;
-        if (this.info != o.info)
-            return false;
-        if (this.maskForEqualHashComp != o.maskForEqualHashComp)
-            return false;
-        for (int i = 0; i < maskForEqualHashComp.trueBitCount(); i++) {
-            int c = maskForEqualHashComp.trueBitAt(i);
-            if (this.cols[c].equals(o.cols[c]) == false) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    @Override
-    public int hashCode() {
-        int hash = 1;
-        for (int i = 0; i < maskForEqualHashComp.trueBitCount(); i++) {
-            int c = maskForEqualHashComp.trueBitAt(i);
-            hash = (31 * hash) + cols[c].hashCode();
-        }
-        return hash;
-    }
-
-    @Override
-    public int compareTo(GTRecord o) {
-        assert this.info == o.info;
-        assert this.maskForEqualHashComp == o.maskForEqualHashComp; // reference equal for performance
-        IGTComparator comparator = info.codeSystem.getComparator();
-
-        int comp = 0;
-        for (int i = 0; i < maskForEqualHashComp.trueBitCount(); i++) {
-            int c = maskForEqualHashComp.trueBitAt(i);
-            comp = comparator.compare(cols[c], o.cols[c]);
-            if (comp != 0)
-                return comp;
-        }
-        return comp;
-    }
-
-    @Override
-    public String toString() {
-        return toString(maskForEqualHashComp);
-    }
-    
-    public String toString(ImmutableBitSet selectedColumns) {
-        Object[] values = new Object[selectedColumns.cardinality()];
-        getValues(selectedColumns, values);
-        return Arrays.toString(values);
-    }
-
-    // ============================================================================
-
-    public ByteArray exportColumns(ImmutableBitSet selectedCols) {
-        int len = 0;
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            len += cols[c].length();
-        }
-
-        ByteArray buf = ByteArray.allocate(len);
-        exportColumns(info.primaryKey, buf);
-        return buf;
-    }
-
-    /** write data to given buffer, like serialize */
-    public void exportColumns(ImmutableBitSet selectedCols, ByteArray buf) {
-        int pos = 0;
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            System.arraycopy(cols[c].array(), cols[c].offset(), buf.array(), buf.offset() + pos, cols[c].length());
-            pos += cols[c].length();
-        }
-        buf.setLength(pos);
-    }
-
-    /** write data to given buffer, like serialize */
-    public void exportColumns(ImmutableBitSet selectedCols, ByteBuffer buf) {
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            buf.put(cols[c].array(), cols[c].offset(), cols[c].length());
-        }
-    }
-
-    public void exportColumns(int[] fieldIndex, ByteBuffer buf) {
-        for (int i : fieldIndex) {
-            buf.put(cols[i].array(), cols[i].offset(), cols[i].length());
-        }
-    }
-
-
-    /** write data to given buffer, like serialize */
-    public void exportColumnBlock(int c, ByteBuffer buf) {
-        exportColumns(info.colBlocks[c], buf);
-    }
-
-    /** change pointers to point to data in given buffer, UNLIKE deserialize */
-    public void loadPrimaryKey(ByteBuffer buf) {
-        loadColumns(info.primaryKey, buf);
-    }
-
-    /** change pointers to point to data in given buffer, UNLIKE deserialize */
-    public void loadCellBlock(int c, ByteBuffer buf) {
-        loadColumns(info.colBlocks[c], buf);
-    }
-
-    /** change pointers to point to data in given buffer, UNLIKE deserialize */
-    public void loadColumns(ImmutableBitSet selectedCols, ByteBuffer buf) {
-        int pos = buf.position();
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            int len = info.codeSystem.codeLength(c, buf);
-            cols[c].set(buf.array(), buf.arrayOffset() + pos, len);
-            pos += len;
-            buf.position(pos);
-        }
-    }
-
-    /** similar to export(primaryKey) but will stop at the first null value */
-    public static ByteArray exportScanKey(GTRecord rec) {
-        if (rec == null)
-            return null;
-        
-        GTInfo info = rec.getInfo();
-        
-        BitSet selectedColumns = new BitSet();
-        int len = 0;
-        for (int i = 0; i < info.primaryKey.trueBitCount(); i++) {
-            int c = info.primaryKey.trueBitAt(i);
-            if (rec.cols[c].array() == null) {
-                break;
-            }
-            selectedColumns.set(c);
-            len += rec.cols[c].length();
-        }
-        
-        if (selectedColumns.cardinality() == 0)
-            return null;
-
-        ByteArray buf = ByteArray.allocate(len);
-        rec.exportColumns(new ImmutableBitSet(selectedColumns), buf);
-        return buf;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRowBlock.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRowBlock.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRowBlock.java
deleted file mode 100644
index 6878ef1..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTRowBlock.java
+++ /dev/null
@@ -1,259 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.ImmutableBitSet;
-
-public class GTRowBlock {
-
-    /** create a row block, allocate memory, get ready for writing */
-    public static GTRowBlock allocate(GTInfo info) {
-        GTRowBlock b = new GTRowBlock(info);
-
-        byte[] array = new byte[info.getMaxColumnLength(info.primaryKey)];
-        b.primaryKey.set(array);
-
-        int maxRows = info.isRowBlockEnabled() ? info.rowBlockSize : 1;
-        for (int i = 0; i < b.cellBlocks.length; i++) {
-            array = new byte[info.getMaxColumnLength(info.colBlocks[i]) * maxRows];
-            b.cellBlocks[i].set(array);
-        }
-        return b;
-    }
-
-    final GTInfo info;
-
-    int seqId; // 0, 1, 2...
-    int nRows;
-    ByteArray primaryKey; // the primary key of the first (smallest) row
-    ByteArray[] cellBlocks; // cells for each column block
-
-    /** create a row block that has no underlying space */
-    public GTRowBlock(GTInfo info) {
-        this.info = info;
-        this.primaryKey = new ByteArray();
-        this.cellBlocks = new ByteArray[info.colBlocks.length];
-        for (int i = 0; i < this.cellBlocks.length; i++) {
-            this.cellBlocks[i] = new ByteArray();
-        }
-    }
-
-    public int getSequenceId() {
-        return seqId;
-    }
-
-    public ByteArray getPrimaryKey() {
-        return primaryKey;
-    }
-
-    public ByteArray getCellBlock(int i) {
-        return cellBlocks[i];
-    }
-
-    public Writer getWriter() {
-        return new Writer();
-    }
-
-    public class Writer {
-        ByteBuffer[] cellBlockBuffers;
-
-        Writer() {
-            cellBlockBuffers = new ByteBuffer[info.colBlocks.length];
-            for (int i = 0; i < cellBlockBuffers.length; i++) {
-                cellBlockBuffers[i] = cellBlocks[i].asBuffer();
-            }
-        }
-
-        public void copyFrom(GTRowBlock other) {
-            assert info == other.info;
-
-            seqId = other.seqId;
-            nRows = other.nRows;
-            primaryKey.copyFrom(other.primaryKey);
-            for (int i = 0; i < info.colBlocks.length; i++) {
-                cellBlockBuffers[i].clear();
-                cellBlockBuffers[i].put(other.cellBlocks[i].array(), other.cellBlocks[i].offset(), other.cellBlocks[i].length());
-            }
-        }
-
-        public void append(GTRecord r) {
-            // add record to block
-            if (isEmpty()) {
-                r.exportColumns(info.primaryKey, primaryKey);
-            }
-            for (int i = 0; i < info.colBlocks.length; i++) {
-                r.exportColumnBlock(i, cellBlockBuffers[i]);
-            }
-            nRows++;
-        }
-
-        public void readyForFlush() {
-            for (int i = 0; i < cellBlocks.length; i++) {
-                cellBlocks[i].setLength(cellBlockBuffers[i].position());
-            }
-        }
-
-        public void clearForNext() {
-            seqId++;
-            nRows = 0;
-            for (int i = 0; i < cellBlockBuffers.length; i++) {
-                cellBlockBuffers[i].clear();
-            }
-        }
-    }
-
-    public Reader getReader() {
-        return new Reader(info.colBlocksAll);
-    }
-
-    public Reader getReader(ImmutableBitSet selectedColBlocks) {
-        return new Reader(selectedColBlocks);
-    }
-
-    public class Reader {
-        int cur;
-        ByteBuffer primaryKeyBuffer;
-        ByteBuffer[] cellBlockBuffers;
-        ImmutableBitSet selectedColBlocks;
-
-        Reader(ImmutableBitSet selectedColBlocks) {
-            primaryKeyBuffer = primaryKey.asBuffer();
-            cellBlockBuffers = new ByteBuffer[info.colBlocks.length];
-            for (int i = 0; i < cellBlockBuffers.length; i++) {
-                cellBlockBuffers[i] = cellBlocks[i].asBuffer();
-            }
-            this.selectedColBlocks = selectedColBlocks;
-        }
-
-        public boolean hasNext() {
-            return cur < nRows;
-        }
-
-        public void fetchNext(GTRecord result) {
-            if (hasNext() == false)
-                throw new IllegalArgumentException();
-
-            for (int i = 0; i < selectedColBlocks.trueBitCount(); i++) {
-                int c = selectedColBlocks.trueBitAt(i);
-                result.loadCellBlock(c, cellBlockBuffers[c]);
-            }
-            cur++;
-        }
-    }
-
-    public GTRowBlock copy() {
-        GTRowBlock copy = new GTRowBlock(info);
-
-        ByteBuffer buf = ByteBuffer.allocate(this.exportLength());
-        this.export(buf);
-        buf.clear();
-        copy.load(buf);
-
-        return copy;
-    }
-
-    public boolean isEmpty() {
-        return nRows == 0;
-    }
-
-    public boolean isFull() {
-        if (info.isRowBlockEnabled())
-            return nRows >= info.rowBlockSize;
-        else
-            return nRows > 0;
-    }
-
-    public int getNumberOfRows() {
-        return nRows;
-    }
-
-    public void setNumberOfRows(int nRows) {
-        this.nRows = nRows;
-    }
-    
-    // ============================================================================
-
-    public int exportLength() {
-        int len = 4; // seq Id
-        if (info.isRowBlockEnabled())
-            len += 4; // nRows
-        len += 4 + primaryKey.length(); // PK byte array
-        for (ByteArray array : cellBlocks) {
-            len += 4 + array.length(); // cell block byte array
-        }
-        return len;
-    }
-
-    /** write data to given output stream, like serialize */
-    public void export(DataOutputStream out) throws IOException {
-        out.writeInt(seqId);
-        if (info.isRowBlockEnabled())
-            out.writeInt(nRows);
-        export(out, primaryKey);
-        for (ByteArray cb : cellBlocks) {
-            export(out, cb);
-        }
-    }
-
-    private void export(DataOutputStream out, ByteArray array) throws IOException {
-        out.writeInt(array.length());
-        out.write(array.array(), array.offset(), array.length());
-    }
-
-    /** write data to given buffer, like serialize */
-    public void export(ByteBuffer buf) {
-        buf.putInt(seqId);
-        if (info.isRowBlockEnabled())
-            buf.putInt(nRows);
-        export(primaryKey, buf);
-        for (ByteArray cb : cellBlocks) {
-            export(cb, buf);
-        }
-    }
-
-    private void export(ByteArray array, ByteBuffer buf) {
-        buf.putInt(array.length());
-        buf.put(array.array(), array.offset(), array.length());
-    }
-    
-    /** read data from given input stream, like deserialize */
-    public void importFrom(DataInputStream in) throws IOException {
-        seqId = in.readInt();
-        nRows = info.isRowBlockEnabled() ? in.readInt() : 1;
-        importFrom(in, primaryKey);
-        for (int i = 0; i < info.colBlocks.length; i++) {
-            ByteArray cb = cellBlocks[i];
-            importFrom(in, cb);
-        }
-    }
-
-    private void importFrom(DataInputStream in, ByteArray result) throws IOException {
-        byte[] data = result.array();
-        int len = in.readInt();
-        in.read(data, 0, len);
-        result.set(data, 0, len);
-    }
-
-    /** change pointers to point to data in given buffer, UNLIKE deserialize */
-    public void load(ByteBuffer buf) {
-        seqId = buf.getInt();
-        nRows = info.isRowBlockEnabled() ? buf.getInt() : 1;
-        load(primaryKey, buf);
-        for (int i = 0; i < info.colBlocks.length; i++) {
-            ByteArray cb = cellBlocks[i];
-            load(cb, buf);
-        }
-    }
-
-    private void load(ByteArray array, ByteBuffer buf) {
-        int len = buf.getInt();
-        int pos = buf.position();
-        array.set(buf.array(), buf.arrayOffset() + pos, len);
-        buf.position(pos + len);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTSampleCodeSystem.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTSampleCodeSystem.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTSampleCodeSystem.java
deleted file mode 100644
index d3fa42d..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTSampleCodeSystem.java
+++ /dev/null
@@ -1,101 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
-import org.apache.kylin.metadata.measure.serializer.DataTypeSerializer;
-
-@SuppressWarnings({ "rawtypes", "unchecked" })
-/**
- * This is just for example and is INCORRECT when numbers are encoded to bytes and compared in filter.
- * 
- * A correct implementation must ensure dimension values preserve order after encoded, e.g. by using an
- * order preserving dictionary.
- * 
- * @author yangli9
- */
-public class GTSampleCodeSystem implements IGTCodeSystem {
-
-    private GTInfo info;
-    private DataTypeSerializer[] serializers;
-    private IGTComparator comparator;
-
-    public GTSampleCodeSystem() {
-    }
-    
-    @Override
-    public void init(GTInfo info) {
-        this.info = info;
-
-        this.serializers = new DataTypeSerializer[info.getColumnCount()];
-        for (int i = 0; i < info.getColumnCount(); i++) {
-            this.serializers[i] = DataTypeSerializer.create(info.colTypes[i]);
-        }
-
-        this.comparator = new IGTComparator() {
-            @Override
-            public boolean isNull(ByteArray code) {
-                // all 0xff is null
-                byte[] array = code.array();
-                for (int i = 0, j = code.offset(), n = code.length(); i < n; i++, j++) {
-                    if (array[j] != (byte) 0xff)
-                        return false;
-                }
-                return true;
-            }
-
-            @Override
-            public int compare(ByteArray code1, ByteArray code2) {
-                return code1.compareTo(code2);
-            }
-        };
-    }
-
-    @Override
-    public int codeLength(int col, ByteBuffer buf) {
-        return serializers[col].peekLength(buf);
-    }
-
-    @Override
-    public int maxCodeLength(int col) {
-        return serializers[col].maxLength();
-    }
-
-    @Override
-    public IGTComparator getComparator() {
-        return comparator;
-    }
-
-    // ============================================================================
-
-    @Override
-    public MeasureAggregator<?>[] newMetricsAggregators(ImmutableBitSet columns, String[] aggrFunctions) {
-        assert columns.trueBitCount() == aggrFunctions.length;
-        
-        MeasureAggregator<?>[] result = new MeasureAggregator[aggrFunctions.length];
-        for (int i = 0; i < result.length; i++) {
-            int col = columns.trueBitAt(i);
-            result[i] = MeasureAggregator.create(aggrFunctions[i], info.getColumnType(col).toString());
-        }
-        return result;
-    }
-
-    @Override
-    public void encodeColumnValue(int col, Object value, ByteBuffer buf) {
-        serializers[col].serialize(value, buf);
-    }
-
-    @Override
-    public void encodeColumnValue(int col, Object value, int roundingFlag, ByteBuffer buf) {
-        // ignore rounding
-        encodeColumnValue(col, value, buf);
-    }
-
-    @Override
-    public Object decodeColumnValue(int col, ByteBuffer buf) {
-        return serializers[col].deserialize(buf);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRange.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRange.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRange.java
deleted file mode 100644
index 916fb11..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRange.java
+++ /dev/null
@@ -1,83 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.util.Collections;
-import java.util.List;
-
-public class GTScanRange {
-
-    final public GTRecord pkStart; // inclusive, record must not be null, col[pk].array() can be null to mean unbounded
-    final public GTRecord pkEnd; // inclusive, record must not be null, col[pk].array() can be null to mean unbounded
-    final public List<GTRecord> hbaseFuzzyKeys; // partial matching primary keys
-
-    public GTScanRange(GTRecord pkStart, GTRecord pkEnd) {
-        this(pkStart, pkEnd, null);
-    }
-
-    public GTScanRange(GTRecord pkStart, GTRecord pkEnd, List<GTRecord> hbaseFuzzyKeys) {
-        GTInfo info = pkStart.info;
-        assert info == pkEnd.info;
-
-        validateRangeKey(pkStart);
-        validateRangeKey(pkEnd);
-
-        this.pkStart = pkStart;
-        this.pkEnd = pkEnd;
-        this.hbaseFuzzyKeys = hbaseFuzzyKeys == null ? Collections.<GTRecord> emptyList() : hbaseFuzzyKeys;
-    }
-
-    private void validateRangeKey(GTRecord pk) {
-        pk.maskForEqualHashComp(pk.info.primaryKey);
-        boolean afterNull = false;
-        for (int i = 0; i < pk.info.primaryKey.trueBitCount(); i++) {
-            int c = pk.info.primaryKey.trueBitAt(i);
-            if (afterNull) {
-                pk.cols[c].set(null, 0, 0);
-            } else {
-                afterNull = pk.cols[c].array() == null;
-            }
-        }
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((hbaseFuzzyKeys == null) ? 0 : hbaseFuzzyKeys.hashCode());
-        result = prime * result + ((pkEnd == null) ? 0 : pkEnd.hashCode());
-        result = prime * result + ((pkStart == null) ? 0 : pkStart.hashCode());
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        GTScanRange other = (GTScanRange) obj;
-        if (hbaseFuzzyKeys == null) {
-            if (other.hbaseFuzzyKeys != null)
-                return false;
-        } else if (!hbaseFuzzyKeys.equals(other.hbaseFuzzyKeys))
-            return false;
-        if (pkEnd == null) {
-            if (other.pkEnd != null)
-                return false;
-        } else if (!pkEnd.equals(other.pkEnd))
-            return false;
-        if (pkStart == null) {
-            if (other.pkStart != null)
-                return false;
-        } else if (!pkStart.equals(other.pkStart))
-            return false;
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        return (pkStart == null ? "null" : pkStart.toString(pkStart.info.primaryKey)) //
-                + "-" + (pkEnd == null ? "null" : pkEnd.toString(pkEnd.info.primaryKey));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRangePlanner.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRangePlanner.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRangePlanner.java
deleted file mode 100644
index 234d0c3..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRangePlanner.java
+++ /dev/null
@@ -1,486 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
-public class GTScanRangePlanner {
-
-    private static final int MAX_HBASE_FUZZY_KEYS = 100;
-
-    final private GTInfo info;
-    final private ComparatorEx<ByteArray> byteUnknownIsSmaller;
-    final private ComparatorEx<ByteArray> byteUnknownIsBigger;
-    final private ComparatorEx<GTRecord> recordUnknownIsSmaller;
-    final private ComparatorEx<GTRecord> recordUnknownIsBigger;
-
-    public GTScanRangePlanner(GTInfo info) {
-        this.info = info;
-
-        IGTComparator comp = info.codeSystem.getComparator();
-        this.byteUnknownIsSmaller = byteComparatorTreatsUnknownSmaller(comp);
-        this.byteUnknownIsBigger = byteComparatorTreatsUnknownBigger(comp);
-        this.recordUnknownIsSmaller = recordComparatorTreatsUnknownSmaller(comp);
-        this.recordUnknownIsBigger = recordComparatorTreatsUnknownBigger(comp);
-    }
-
-    // return empty list meaning filter is always false
-    public List<GTScanRange> planScanRanges(TupleFilter filter) {
-        return planScanRanges(filter, Integer.MAX_VALUE);
-    }
-
-    // return empty list meaning filter is always false
-    public List<GTScanRange> planScanRanges(TupleFilter filter, int maxRanges) {
-
-        TupleFilter flatFilter = flattenToOrAndFilter(filter);
-
-        List<Collection<ColumnRange>> orAndDimRanges = translateToOrAndDimRanges(flatFilter);
-
-        List<GTScanRange> scanRanges = Lists.newArrayListWithCapacity(orAndDimRanges.size());
-        for (Collection<ColumnRange> andDimRanges : orAndDimRanges) {
-            GTScanRange scanRange = newScanRange(andDimRanges);
-            scanRanges.add(scanRange);
-        }
-
-        List<GTScanRange> mergedRanges = mergeOverlapRanges(scanRanges);
-        mergedRanges = mergeTooManyRanges(mergedRanges, maxRanges);
-
-        return mergedRanges;
-    }
-
-    private GTScanRange newScanRange(Collection<ColumnRange> andDimRanges) {
-        GTRecord pkStart = new GTRecord(info);
-        GTRecord pkEnd = new GTRecord(info);
-        List<GTRecord> hbaseFuzzyKeys = Lists.newArrayList();
-
-        for (ColumnRange range : andDimRanges) {
-            int col = range.column.getColumnDesc().getZeroBasedIndex();
-            if (info.primaryKey.get(col) == false)
-                continue;
-
-            pkStart.set(col, range.begin);
-            pkEnd.set(col, range.end);
-
-            if (range.equals != null) {
-                ImmutableBitSet fuzzyMask = new ImmutableBitSet(col);
-                for (ByteArray v : range.equals) {
-                    GTRecord fuzzy = new GTRecord(info);
-                    fuzzy.set(col, v);
-                    fuzzy.maskForEqualHashComp(fuzzyMask);
-                    hbaseFuzzyKeys.add(fuzzy);
-                }
-            }
-        }
-
-        return new GTScanRange(pkStart, pkEnd, hbaseFuzzyKeys);
-    }
-
-    private TupleFilter flattenToOrAndFilter(TupleFilter filter) {
-        if (filter == null)
-            return null;
-
-        TupleFilter flatFilter = filter.flatFilter();
-
-        // normalize to OR-AND filter
-        if (flatFilter.getOperator() == FilterOperatorEnum.AND) {
-            LogicalTupleFilter f = new LogicalTupleFilter(FilterOperatorEnum.OR);
-            f.addChild(flatFilter);
-            flatFilter = f;
-        }
-
-        if (flatFilter.getOperator() != FilterOperatorEnum.OR)
-            throw new IllegalStateException();
-
-        return flatFilter;
-    }
-
-    private List<Collection<ColumnRange>> translateToOrAndDimRanges(TupleFilter flatFilter) {
-        List<Collection<ColumnRange>> result = Lists.newArrayList();
-
-        if (flatFilter == null) {
-            result.add(Collections.<ColumnRange> emptyList());
-            return result;
-        }
-
-        for (TupleFilter andFilter : flatFilter.getChildren()) {
-            if (andFilter.getOperator() != FilterOperatorEnum.AND)
-                throw new IllegalStateException("Filter should be AND instead of " + andFilter);
-
-            Collection<ColumnRange> andRanges = translateToAndDimRanges(andFilter.getChildren());
-            result.add(andRanges);
-        }
-
-        return preEvaluateConstantConditions(result);
-    }
-
-    private Collection<ColumnRange> translateToAndDimRanges(List<? extends TupleFilter> andFilters) {
-        Map<TblColRef, ColumnRange> rangeMap = new HashMap<TblColRef, ColumnRange>();
-        for (TupleFilter filter : andFilters) {
-            if ((filter instanceof CompareTupleFilter) == false) {
-                continue;
-            }
-
-            CompareTupleFilter comp = (CompareTupleFilter) filter;
-            if (comp.getColumn() == null) {
-                continue;
-            }
-
-            @SuppressWarnings("unchecked")
-            ColumnRange newRange = new ColumnRange(comp.getColumn(), (Set<ByteArray>) comp.getValues(), comp.getOperator());
-            ColumnRange existing = rangeMap.get(newRange.column);
-            if (existing == null) {
-                rangeMap.put(newRange.column, newRange);
-            } else {
-                existing.andMerge(newRange);
-            }
-        }
-        return rangeMap.values();
-    }
-
-    private List<Collection<ColumnRange>> preEvaluateConstantConditions(List<Collection<ColumnRange>> orAndRanges) {
-        boolean globalAlwaysTrue = false;
-        Iterator<Collection<ColumnRange>> iterator = orAndRanges.iterator();
-        while (iterator.hasNext()) {
-            Collection<ColumnRange> andRanges = iterator.next();
-            Iterator<ColumnRange> iterator2 = andRanges.iterator();
-            boolean hasAlwaysFalse = false;
-            while (iterator2.hasNext()) {
-                ColumnRange range = iterator2.next();
-                if (range.satisfyAll())
-                    iterator2.remove();
-                else if (range.satisfyNone())
-                    hasAlwaysFalse = true;
-            }
-            if (hasAlwaysFalse) {
-                iterator.remove();
-            } else if (andRanges.isEmpty()) {
-                globalAlwaysTrue = true;
-                break;
-            }
-        }
-        // return empty OR list means global false
-        // return an empty AND collection inside OR list means global true
-        if (globalAlwaysTrue) {
-            orAndRanges.clear();
-            orAndRanges.add(Collections.<ColumnRange> emptyList());
-        }
-        return orAndRanges;
-    }
-
-    private List<GTScanRange> mergeOverlapRanges(List<GTScanRange> ranges) {
-        if (ranges.size() <= 1) {
-            return ranges;
-        }
-
-        // sort ranges by start key
-        Collections.sort(ranges, new Comparator<GTScanRange>() {
-            @Override
-            public int compare(GTScanRange a, GTScanRange b) {
-                return recordUnknownIsSmaller.compare(a.pkStart, b.pkStart);
-            }
-        });
-
-        // merge the overlap range
-        List<GTScanRange> mergedRanges = new ArrayList<GTScanRange>();
-        int mergeBeginIndex = 0;
-        GTRecord mergeEnd = ranges.get(0).pkEnd;
-        for (int index = 0; index < ranges.size(); index++) {
-            GTScanRange range = ranges.get(index);
-
-            // if overlap, swallow it
-            if (recordUnknownIsSmaller.min(range.pkStart, mergeEnd) == range.pkStart //
-                    || recordUnknownIsBigger.max(mergeEnd, range.pkStart) == mergeEnd) {
-                mergeEnd = recordUnknownIsBigger.max(mergeEnd, range.pkEnd);
-                continue;
-            }
-
-            // not overlap, split here
-            GTScanRange mergedRange = mergeKeyRange(ranges.subList(mergeBeginIndex, index));
-            mergedRanges.add(mergedRange);
-
-            // start new split
-            mergeBeginIndex = index;
-            mergeEnd = recordUnknownIsBigger.max(mergeEnd, range.pkEnd);
-        }
-
-        // don't miss the last range
-        GTScanRange mergedRange = mergeKeyRange(ranges.subList(mergeBeginIndex, ranges.size()));
-        mergedRanges.add(mergedRange);
-
-        return mergedRanges;
-    }
-
-    private GTScanRange mergeKeyRange(List<GTScanRange> ranges) {
-        GTScanRange first = ranges.get(0);
-        if (ranges.size() == 1)
-            return first;
-
-        GTRecord start = first.pkStart;
-        GTRecord end = first.pkEnd;
-        List<GTRecord> newFuzzyKeys = new ArrayList<GTRecord>();
-
-        boolean hasNonFuzzyRange = false;
-        for (GTScanRange range : ranges) {
-            hasNonFuzzyRange = hasNonFuzzyRange || range.hbaseFuzzyKeys.isEmpty();
-            newFuzzyKeys.addAll(range.hbaseFuzzyKeys);
-            end = recordUnknownIsBigger.max(end, range.pkEnd);
-        }
-
-        // if any range is non-fuzzy, then all fuzzy keys must be cleared
-        // also too many fuzzy keys will slow down HBase scan
-        if (hasNonFuzzyRange || newFuzzyKeys.size() > MAX_HBASE_FUZZY_KEYS) {
-            newFuzzyKeys.clear();
-        }
-
-        return new GTScanRange(start, end, newFuzzyKeys);
-    }
-
-    private List<GTScanRange> mergeTooManyRanges(List<GTScanRange> ranges, int maxRanges) {
-        if (ranges.size() <= maxRanges) {
-            return ranges;
-        }
-
-        // TODO: check the distance between range and merge the large distance range
-        List<GTScanRange> result = new ArrayList<GTScanRange>(1);
-        GTScanRange mergedRange = mergeKeyRange(ranges);
-        result.add(mergedRange);
-        return result;
-    }
-
-    private class ColumnRange {
-        private TblColRef column;
-        private ByteArray begin = ByteArray.EMPTY;
-        private ByteArray end = ByteArray.EMPTY;
-        private Set<ByteArray> equals;
-
-        public ColumnRange(TblColRef column, Set<ByteArray> values, FilterOperatorEnum op) {
-            this.column = column;
-
-            switch (op) {
-            case EQ:
-            case IN:
-                equals = new HashSet<ByteArray>(values);
-                refreshBeginEndFromEquals();
-                break;
-            case LT:
-            case LTE:
-                end = byteUnknownIsBigger.max(values);
-                break;
-            case GT:
-            case GTE:
-                begin = byteUnknownIsSmaller.min(values);
-                break;
-            case NEQ:
-            case NOTIN:
-            case ISNULL:
-            case ISNOTNULL:
-                // let Optiq filter it!
-                break;
-            default:
-                throw new UnsupportedOperationException(op.name());
-            }
-        }
-
-        void copy(TblColRef column, ByteArray beginValue, ByteArray endValue, Set<ByteArray> equalValues) {
-            this.column = column;
-            this.begin = beginValue;
-            this.end = endValue;
-            this.equals = equalValues;
-        }
-
-        private void refreshBeginEndFromEquals() {
-            if (equals.isEmpty()) {
-                begin = ByteArray.EMPTY;
-                end = ByteArray.EMPTY;
-            } else {
-                begin = byteUnknownIsSmaller.min(equals);
-                end = byteUnknownIsBigger.max(equals);
-            }
-        }
-
-        public boolean satisfyAll() {
-            return begin.array() == null && end.array() == null; // the NEQ case
-        }
-
-        public boolean satisfyNone() {
-            if (equals != null) {
-                return equals.isEmpty();
-            } else if (begin.array() != null && end.array() != null) {
-                return info.codeSystem.getComparator().compare(begin, end) > 0;
-            } else {
-                return false;
-            }
-        }
-
-        public void andMerge(ColumnRange another) {
-            assert this.column.equals(another.column);
-
-            if (another.satisfyAll()) {
-                return;
-            }
-
-            if (this.satisfyAll()) {
-                copy(another.column, another.begin, another.end, another.equals);
-                return;
-            }
-
-            if (this.equals != null && another.equals != null) {
-                this.equals.retainAll(another.equals);
-                refreshBeginEndFromEquals();
-                return;
-            }
-
-            if (this.equals != null) {
-                this.equals = filter(this.equals, another.begin, another.end);
-                refreshBeginEndFromEquals();
-                return;
-            }
-
-            if (another.equals != null) {
-                this.equals = filter(another.equals, this.begin, this.end);
-                refreshBeginEndFromEquals();
-                return;
-            }
-
-            this.begin = byteUnknownIsSmaller.max(this.begin, another.begin);
-            this.end = byteUnknownIsBigger.min(this.end, another.end);
-        }
-
-        private Set<ByteArray> filter(Set<ByteArray> equalValues, ByteArray beginValue, ByteArray endValue) {
-            Set<ByteArray> result = Sets.newHashSetWithExpectedSize(equalValues.size());
-            for (ByteArray v : equalValues) {
-                if (byteUnknownIsSmaller.compare(beginValue, v) <= 0 && byteUnknownIsBigger.compare(v, endValue) <= 0) {
-                    result.add(v);
-                }
-            }
-            return equalValues;
-        }
-
-        public String toString() {
-            if (equals == null) {
-                return column.getName() + " between " + begin + " and " + end;
-            } else {
-                return column.getName() + " in " + equals;
-            }
-        }
-    }
-
-    public static abstract class ComparatorEx<T> implements Comparator<T> {
-
-        public T min(Collection<T> v) {
-            if (v.size() <= 0) {
-                return null;
-            }
-
-            Iterator<T> iterator = v.iterator();
-            T min = iterator.next();
-            while (iterator.hasNext()) {
-                min = min(min, iterator.next());
-            }
-            return min;
-        }
-
-        public T max(Collection<T> v) {
-            if (v.size() <= 0) {
-                return null;
-            }
-
-            Iterator<T> iterator = v.iterator();
-            T max = iterator.next();
-            while (iterator.hasNext()) {
-                max = max(max, iterator.next());
-            }
-            return max;
-        }
-
-        public T min(T a, T b) {
-            return compare(a, b) <= 0 ? a : b;
-        }
-
-        public T max(T a, T b) {
-            return compare(a, b) >= 0 ? a : b;
-        }
-
-        public boolean between(T v, T start, T end) {
-            return compare(start, v) <= 0 && compare(v, end) <= 0;
-        }
-    }
-
-    public static ComparatorEx<ByteArray> byteComparatorTreatsUnknownSmaller(final IGTComparator comp) {
-        return new ComparatorEx<ByteArray>() {
-            @Override
-            public int compare(ByteArray a, ByteArray b) {
-                if (a.array() == null)
-                    return -1;
-                else if (b.array() == null)
-                    return 1;
-                else
-                    return comp.compare(a, b);
-            }
-        };
-    }
-
-    public static ComparatorEx<ByteArray> byteComparatorTreatsUnknownBigger(final IGTComparator comp) {
-        return new ComparatorEx<ByteArray>() {
-            @Override
-            public int compare(ByteArray a, ByteArray b) {
-                if (a.array() == null)
-                    return 1;
-                else if (b.array() == null)
-                    return -1;
-                else
-                    return comp.compare(a, b);
-            }
-        };
-    }
-
-    public static ComparatorEx<GTRecord> recordComparatorTreatsUnknownSmaller(IGTComparator comp) {
-        return new RecordComparator(byteComparatorTreatsUnknownSmaller(comp));
-    }
-
-    public static ComparatorEx<GTRecord> recordComparatorTreatsUnknownBigger(IGTComparator comp) {
-        return new RecordComparator(byteComparatorTreatsUnknownBigger(comp));
-    }
-
-    private static class RecordComparator extends ComparatorEx<GTRecord> {
-        final ComparatorEx<ByteArray> comparator;
-
-        RecordComparator(ComparatorEx<ByteArray> byteComparator) {
-            this.comparator = byteComparator;
-        }
-
-        @Override
-        public int compare(GTRecord a, GTRecord b) {
-            assert a.info == b.info;
-            assert a.maskForEqualHashComp() == b.maskForEqualHashComp();
-            ImmutableBitSet mask = a.maskForEqualHashComp();
-
-            int comp = 0;
-            for (int i = 0; i < mask.trueBitCount(); i++) {
-                int c = mask.trueBitAt(i);
-                comp = comparator.compare(a.cols[c], b.cols[c]);
-                if (comp != 0)
-                    return comp;
-            }
-            return 0; // equals
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRequest.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRequest.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRequest.java
deleted file mode 100644
index 1fd3c8e..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTScanRequest.java
+++ /dev/null
@@ -1,155 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.util.Arrays;
-import java.util.Set;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import com.google.common.collect.Sets;
-
-public class GTScanRequest {
-
-    // basic
-    private GTInfo info;
-    private GTScanRange range;
-    private ImmutableBitSet columns;
-
-    // optional filtering
-    private TupleFilter filterPushDown;
-
-    // optional aggregation
-    private ImmutableBitSet aggrGroupBy;
-    private ImmutableBitSet aggrMetrics;
-    private String[] aggrMetricsFuncs;
-
-    public GTScanRequest(GTInfo info) {
-        this(info, null, null, null);
-    }
-
-    public GTScanRequest(GTInfo info, GTScanRange range, ImmutableBitSet columns, TupleFilter filterPushDown) {
-        this.info = info;
-        this.range = range == null ? new GTScanRange(new GTRecord(info), new GTRecord(info)) : range;
-        this.columns = columns;
-        this.filterPushDown = filterPushDown;
-        validate();
-    }
-
-    public GTScanRequest(GTInfo info, GTScanRange range, ImmutableBitSet aggrGroupBy, ImmutableBitSet aggrMetrics, //
-            String[] aggrMetricsFuncs, TupleFilter filterPushDown) {
-        this(info, range, null, aggrGroupBy, aggrMetrics, aggrMetricsFuncs, filterPushDown);
-    }
-
-    public GTScanRequest(GTInfo info, GTScanRange range, ImmutableBitSet dimensions, ImmutableBitSet aggrGroupBy, //
-            ImmutableBitSet aggrMetrics, String[] aggrMetricsFuncs, TupleFilter filterPushDown) {
-        this.info = info;
-        this.range = range == null ? new GTScanRange(new GTRecord(info), new GTRecord(info)) : range;
-        this.columns = dimensions;
-        this.filterPushDown = filterPushDown;
-
-        this.aggrGroupBy = aggrGroupBy;
-        this.aggrMetrics = aggrMetrics;
-        this.aggrMetricsFuncs = aggrMetricsFuncs;
-
-        validate();
-    }
-
-    private void validate() {
-        if (range == null)
-            range = new GTScanRange(null, null);
-
-        if (hasAggregation()) {
-            if (aggrGroupBy.intersects(aggrMetrics))
-                throw new IllegalStateException();
-            if (aggrMetrics.cardinality() != aggrMetricsFuncs.length)
-                throw new IllegalStateException();
-
-            if (columns == null)
-                columns = ImmutableBitSet.EMPTY;
-            columns = columns.or(aggrGroupBy);
-            columns = columns.or(aggrMetrics);
-        }
-
-        if (columns == null)
-            columns = info.colAll;
-
-        if (hasFilterPushDown()) {
-            validateFilterPushDown();
-        }
-    }
-
-    private void validateFilterPushDown() {
-        if (hasFilterPushDown() == false)
-            return;
-
-        Set<TblColRef> filterColumns = Sets.newHashSet();
-        TupleFilter.collectColumns(filterPushDown, filterColumns);
-
-        for (TblColRef col : filterColumns) {
-            // filter columns must belong to the table
-            info.validateColRef(col);
-            // filter columns must be returned to satisfy upper layer evaluation (calcite)
-            columns = columns.set(col.getColumnDesc().getZeroBasedIndex());
-        }
-
-        // un-evaluatable filter must be removed
-        if (TupleFilter.isEvaluableRecursively(filterPushDown) == false) {
-            Set<TblColRef> unevaluableColumns = Sets.newHashSet();
-            filterPushDown = GTUtil.convertFilterUnevaluatable(filterPushDown, info, unevaluableColumns);
-
-            // columns in un-evaluatable filter must be returned without loss so upper layer can do final evaluation
-            if (hasAggregation()) {
-                for (TblColRef col : unevaluableColumns) {
-                    aggrGroupBy = aggrGroupBy.set(col.getColumnDesc().getZeroBasedIndex());
-                }
-            }
-        }
-    }
-
-    public boolean hasFilterPushDown() {
-        return filterPushDown != null;
-    }
-
-    public boolean hasAggregation() {
-        return aggrGroupBy != null && aggrMetrics != null && aggrMetricsFuncs != null;
-    }
-
-    public GTInfo getInfo() {
-        return info;
-    }
-
-    public GTRecord getPkStart() {
-        return range.pkStart;
-    }
-
-    public GTRecord getPkEnd() {
-        return range.pkEnd;
-    }
-
-    public ImmutableBitSet getColumns() {
-        return columns;
-    }
-
-    public TupleFilter getFilterPushDown() {
-        return filterPushDown;
-    }
-
-    public ImmutableBitSet getAggrGroupBy() {
-        return aggrGroupBy;
-    }
-
-    public ImmutableBitSet getAggrMetrics() {
-        return aggrMetrics;
-    }
-
-    public String[] getAggrMetricsFuncs() {
-        return aggrMetricsFuncs;
-    }
-
-    @Override
-    public String toString() {
-        return "GTScanRequest [range=" + range + ", columns=" + columns + ", filterPushDown=" + filterPushDown + ", aggrGroupBy=" + aggrGroupBy + ", aggrMetrics=" + aggrMetrics + ", aggrMetricsFuncs=" + Arrays.toString(aggrMetricsFuncs) + "]";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GTUtil.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTUtil.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GTUtil.java
deleted file mode 100644
index 2532659..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GTUtil.java
+++ /dev/null
@@ -1,221 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.IFilterCodeSystem;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilterSerializer;
-import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import com.google.common.collect.Sets;
-
-public class GTUtil {
-
-    static final TableDesc MOCKUP_TABLE = TableDesc.mockup("GT_MOCKUP_TABLE");
-
-    static TblColRef tblColRef(int col, String datatype) {
-        ColumnDesc desc = ColumnDesc.mockup(MOCKUP_TABLE, col + 1, "" + col, datatype);
-        return new TblColRef(desc);
-    }
-
-    public static TupleFilter convertFilterUnevaluatable(TupleFilter rootFilter, GTInfo info, //
-            Set<TblColRef> unevaluatableColumnCollector) {
-        return convertFilter(rootFilter, info, null, false, unevaluatableColumnCollector);
-    }
-
-    public static TupleFilter convertFilterConstants(TupleFilter rootFilter, GTInfo info) {
-        return convertFilter(rootFilter, info, null, true, null);
-    }
-
-    public static TupleFilter convertFilterColumnsAndConstants(TupleFilter rootFilter, GTInfo info, //
-            List<TblColRef> colMapping, Set<TblColRef> unevaluatableColumnCollector) {
-        return convertFilter(rootFilter, info, colMapping, true, unevaluatableColumnCollector);
-    }
-
-    // converts TblColRef to GridTable column, encode constants, drop unEvaluatable parts
-    private static TupleFilter convertFilter(TupleFilter rootFilter, final GTInfo info, //
-            final List<TblColRef> colMapping, final boolean encodeConstants, //
-            final Set<TblColRef> unevaluatableColumnCollector) {
-
-        IFilterCodeSystem<ByteArray> filterCodeSystem = wrap(info.codeSystem.getComparator());
-        
-        byte[] bytes = TupleFilterSerializer.serialize(rootFilter, new TupleFilterSerializer.Decorator() {
-            @Override
-            public TupleFilter onSerialize(TupleFilter filter) {
-                if (filter == null)
-                    return null;
-
-                // In case of NOT(unEvaluatableFilter), we should immediately replace it as TRUE,
-                // Otherwise, unEvaluatableFilter will later be replace with TRUE and NOT(unEvaluatableFilter)
-                // will always return FALSE.
-                if (filter.getOperator() == TupleFilter.FilterOperatorEnum.NOT && !TupleFilter.isEvaluableRecursively(filter)) {
-                    TupleFilter.collectColumns(filter, unevaluatableColumnCollector);
-                    return ConstantTupleFilter.TRUE;
-                }
-
-                // shortcut for unEvaluatable filter
-                if (filter.isEvaluable() == false) {
-                    TupleFilter.collectColumns(filter, unevaluatableColumnCollector);
-                    return ConstantTupleFilter.TRUE;
-                }
-
-                // map to column onto grid table
-                if (colMapping != null && filter instanceof ColumnTupleFilter) {
-                    ColumnTupleFilter colFilter = (ColumnTupleFilter) filter;
-                    int gtColIdx = colMapping.indexOf(colFilter.getColumn());
-                    return new ColumnTupleFilter(info.colRef(gtColIdx));
-                }
-
-                // encode constants
-                if (encodeConstants && filter instanceof CompareTupleFilter) {
-                    return encodeConstants((CompareTupleFilter) filter);
-                }
-
-                return filter;
-            }
-
-            @SuppressWarnings({ "rawtypes", "unchecked" })
-            private TupleFilter encodeConstants(CompareTupleFilter oldCompareFilter) {
-                // extract ColumnFilter & ConstantFilter
-                TblColRef externalCol = oldCompareFilter.getColumn();
-
-                if (externalCol == null) {
-                    return oldCompareFilter;
-                }
-
-                Collection constValues = oldCompareFilter.getValues();
-                if (constValues == null || constValues.isEmpty()) {
-                    return oldCompareFilter;
-                }
-
-                CompareTupleFilter newCompareFilter = new CompareTupleFilter(oldCompareFilter.getOperator());
-                newCompareFilter.addChild(new ColumnTupleFilter(externalCol));
-
-                Object firstValue = constValues.iterator().next();
-                int col = colMapping == null ? externalCol.getColumnDesc().getZeroBasedIndex() : colMapping.indexOf(externalCol);
-
-                TupleFilter result;
-                ByteArray code;
-
-                // translate constant into code
-                switch (newCompareFilter.getOperator()) {
-                case EQ:
-                case IN:
-                    Set newValues = Sets.newHashSet();
-                    for (Object value : constValues) {
-                        code = translate(col, value, 0);
-                        if (code != null)
-                            newValues.add(code);
-                    }
-                    if (newValues.isEmpty()) {
-                        result = ConstantTupleFilter.FALSE;
-                    } else {
-                        newCompareFilter.addChild(new ConstantTupleFilter(newValues));
-                        result = newCompareFilter;
-                    }
-                    break;
-                case NEQ:
-                    code = translate(col, firstValue, 0);
-                    if (code == null) {
-                        result = ConstantTupleFilter.TRUE;
-                    } else {
-                        newCompareFilter.addChild(new ConstantTupleFilter(code));
-                        result = newCompareFilter;
-                    }
-                    break;
-                case LT:
-                    code = translate(col, firstValue, 1);
-                    if (code == null) {
-                        result = ConstantTupleFilter.TRUE;
-                    } else {
-                        newCompareFilter.addChild(new ConstantTupleFilter(code));
-                        result = newCompareFilter;
-                    }
-                    break;
-                case LTE:
-                    code = translate(col, firstValue, -1);
-                    if (code == null) {
-                        result = ConstantTupleFilter.FALSE;
-                    } else {
-                        newCompareFilter.addChild(new ConstantTupleFilter(code));
-                        result = newCompareFilter;
-                    }
-                    break;
-                case GT:
-                    code = translate(col, firstValue, -1);
-                    if (code == null) {
-                        result = ConstantTupleFilter.TRUE;
-                    } else {
-                        newCompareFilter.addChild(new ConstantTupleFilter(code));
-                        result = newCompareFilter;
-                    }
-                    break;
-                case GTE:
-                    code = translate(col, firstValue, 1);
-                    if (code == null) {
-                        result = ConstantTupleFilter.FALSE;
-                    } else {
-                        newCompareFilter.addChild(new ConstantTupleFilter(code));
-                        result = newCompareFilter;
-                    }
-                    break;
-                default:
-                    throw new IllegalStateException("Cannot handle operator " + newCompareFilter.getOperator());
-                }
-                return result;
-            }
-
-            transient ByteBuffer buf = ByteBuffer.allocate(info.getMaxColumnLength());
-
-            private ByteArray translate(int col, Object value, int roundingFlag) {
-                try {
-                    buf.clear();
-                    info.codeSystem.encodeColumnValue(col, value, roundingFlag, buf);
-                    return ByteArray.copyOf(buf.array(), 0, buf.position());
-                } catch (IllegalArgumentException ex) {
-                    return null;
-                }
-            }
-        }, filterCodeSystem);
-
-        return TupleFilterSerializer.deserialize(bytes, filterCodeSystem);
-    }
-
-    public static IFilterCodeSystem<ByteArray> wrap(final IGTComparator comp) {
-        return new IFilterCodeSystem<ByteArray>() {
-
-            @Override
-            public int compare(ByteArray o1, ByteArray o2) {
-                return comp.compare(o1, o2);
-            }
-
-            @Override
-            public boolean isNull(ByteArray code) {
-                return comp.isNull(code);
-            }
-
-            @Override
-            public void serialize(ByteArray code, ByteBuffer buffer) {
-                if (code == null)
-                    BytesUtil.writeByteArray(null, 0, 0, buffer);
-                else
-                    BytesUtil.writeByteArray(code.array(), code.offset(), code.length(), buffer);
-            }
-
-            @Override
-            public ByteArray deserialize(ByteBuffer buffer) {
-                return new ByteArray(BytesUtil.readByteArray(buffer));
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/GridTable.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/GridTable.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/GridTable.java
deleted file mode 100644
index 092227b..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/GridTable.java
+++ /dev/null
@@ -1,61 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public class GridTable implements Closeable {
-
-    final GTInfo info;
-    final IGTStore store;
-
-    public GridTable(GTInfo info, IGTStore store) {
-        this.info = info;
-        this.store = store;
-    }
-
-    public GTBuilder rebuild() throws IOException {
-        assert info.isShardingEnabled() == false;
-        return rebuild(-1);
-    }
-
-    public GTBuilder rebuild(int shard) throws IOException {
-        assert shard < info.nShards;
-        return new GTBuilder(info, shard, store);
-    }
-
-    public GTBuilder append() throws IOException {
-        assert info.isShardingEnabled() == false;
-        return append(-1);
-    }
-
-    public GTBuilder append(int shard) throws IOException {
-        return new GTBuilder(info, shard, store, true);
-    }
-
-    public IGTScanner scan(GTScanRequest req) throws IOException {
-        IGTScanner result = new GTRawScanner(info, store, req);
-        
-        if (req.hasFilterPushDown()) {
-            result = new GTFilterScanner(result, req);
-        }
-        if (req.hasAggregation()) {
-            result = new GTAggregateScanner(result, req);
-        }
-        return result;
-    }
-
-    public GTInfo getInfo() {
-        return info;
-    }
-
-    public IGTStore getStore() {
-        return store;
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (store instanceof Closeable) {
-            ((Closeable) store).close();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTCodeSystem.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTCodeSystem.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTCodeSystem.java
deleted file mode 100644
index 644c94d..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTCodeSystem.java
+++ /dev/null
@@ -1,43 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
-
-public interface IGTCodeSystem {
-    
-    void init(GTInfo info);
-
-    IGTComparator getComparator();
-    
-    /** Return the length of code starting at the specified buffer, buffer position must not change after return */
-    int codeLength(int col, ByteBuffer buf);
-    
-    /** Return the max possible length of a column */
-    int maxCodeLength(int col);
-    
-    /**
-     * Encode a value into code.
-     * 
-     * @throws IllegalArgumentException if the value is not in dictionary
-     */
-    void encodeColumnValue(int col, Object value, ByteBuffer buf) throws IllegalArgumentException;
-    
-    /**
-     * Encode a value into code, with option to floor rounding -1, no rounding 0,  or ceiling rounding 1
-     * 
-     * @throws IllegalArgumentException
-     * - if rounding=0 and the value is not in dictionary
-     * - if rounding=-1 and there's no equal or smaller value in dictionary
-     * - if rounding=1 and there's no equal or bigger value in dictionary
-     */
-    void encodeColumnValue(int col, Object value, int roundingFlag, ByteBuffer buf) throws IllegalArgumentException;
-    
-    /** Decode a code into value */
-    Object decodeColumnValue(int col, ByteBuffer buf);
-    
-    /** Return aggregators for metrics */
-    MeasureAggregator<?>[] newMetricsAggregators(ImmutableBitSet columns, String[] aggrFunctions);
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTComparator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTComparator.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTComparator.java
deleted file mode 100644
index 442adcc..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTComparator.java
+++ /dev/null
@@ -1,15 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.util.Comparator;
-
-import org.apache.kylin.common.util.ByteArray;
-
-public interface IGTComparator extends Comparator<ByteArray> {
-
-    /** if given code represents the NULL value */
-    boolean isNull(ByteArray code);
-
-    /** compare two values by their codes */
-    // int compare(T code1, T code2);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTScanner.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTScanner.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTScanner.java
deleted file mode 100644
index 3d3c3c8..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTScanner.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.io.Closeable;
-
-public interface IGTScanner extends Iterable<GTRecord>, Closeable {
-    
-    GTInfo getInfo();
-    
-    int getScannedRowCount();
-    
-    int getScannedRowBlockCount();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTStore.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTStore.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTStore.java
deleted file mode 100644
index cf4a3cc..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/IGTStore.java
+++ /dev/null
@@ -1,26 +0,0 @@
-package org.apache.kylin.storage.gridtable;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-
-public interface IGTStore {
-
-    GTInfo getInfo();
-    
-    IGTStoreWriter rebuild(int shard) throws IOException;
-    
-    IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) throws IOException;
-    
-    IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) throws IOException;
-
-    interface IGTStoreWriter extends Closeable {
-        void write(GTRowBlock block) throws IOException;
-    }
-    
-    interface IGTStoreScanner extends Iterator<GTRowBlock>, Closeable {
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/UnitTestSupport.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/UnitTestSupport.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/UnitTestSupport.java
deleted file mode 100644
index b1a7180..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/UnitTestSupport.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements. See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License. You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.kylin.storage.gridtable;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.storage.gridtable.GTInfo.Builder;
-
-public class UnitTestSupport {
-
-    public static GTInfo basicInfo() {
-        Builder builder = infoBuilder();
-        GTInfo info = builder.build();
-        return info;
-    }
-
-    public static GTInfo advancedInfo() {
-        Builder builder = infoBuilder();
-        builder.enableColumnBlock(new ImmutableBitSet[] { setOf(0), setOf(1, 2), setOf(3, 4) });
-        builder.enableRowBlock(4);
-        GTInfo info = builder.build();
-        return info;
-    }
-
-    private static Builder infoBuilder() {
-        Builder builder = GTInfo.builder();
-        builder.setCodeSystem(new GTSampleCodeSystem());
-        builder.setColumns( //
-                DataType.getInstance("varchar(10)"), //
-                DataType.getInstance("varchar(10)"), //
-                DataType.getInstance("varchar(10)"), //
-                DataType.getInstance("bigint"), //
-                DataType.getInstance("decimal") //
-        );
-        builder.setPrimaryKey(setOf(0));
-        builder.setColumnPreferIndex(setOf(0));
-        return builder;
-    }
-
-    public static List<GTRecord> mockupData(GTInfo info, int nRows) {
-        List<GTRecord> result = new ArrayList<GTRecord>(nRows);
-        int round = nRows / 10;
-        for (int i = 0; i < round; i++) {
-            String d_01_14 = datePlus("2015-01-14", i * 4);
-            String d_01_15 = datePlus("2015-01-15", i * 4);
-            String d_01_16 = datePlus("2015-01-16", i * 4);
-            String d_01_17 = datePlus("2015-01-17", i * 4);
-            result.add(newRec(info, d_01_14, "Yang", "Food", new LongMutable(10), new BigDecimal("10.5")));
-            result.add(newRec(info, d_01_14, "Luke", "Food", new LongMutable(10), new BigDecimal("10.5")));
-            result.add(newRec(info, d_01_15, "Xu", "Food", new LongMutable(10), new BigDecimal("10.5")));
-            result.add(newRec(info, d_01_15, "Dong", "Food", new LongMutable(10), new BigDecimal("10.5")));
-            result.add(newRec(info, d_01_15, "Jason", "Food", new LongMutable(10), new BigDecimal("10.5")));
-            result.add(newRec(info, d_01_16, "Mahone", "Food", new LongMutable(10), new BigDecimal("10.5")));
-            result.add(newRec(info, d_01_16, "Shaofeng", "Food", new LongMutable(10), new BigDecimal("10.5")));
-            result.add(newRec(info, d_01_16, "Qianhao", "Food", new LongMutable(10), new BigDecimal("10.5")));
-            result.add(newRec(info, d_01_16, "George", "Food", new LongMutable(10), new BigDecimal("10.5")));
-            result.add(newRec(info, d_01_17, "Kejia", "Food", new LongMutable(10), new BigDecimal("10.5")));
-        }
-        return result;
-    }
-    
-    private static String datePlus(String date, int plusDays) {
-        long millis = DateFormat.stringToMillis(date);
-        millis += (1000L * 3600L * 24L) * plusDays;
-        return DateFormat.formatToDateStr(millis);
-    }
-
-    private static GTRecord newRec(GTInfo info, String date, String name, String category, LongMutable amount, BigDecimal price) {
-        GTRecord rec = new GTRecord(info);
-        return rec.setValues(date, name, category, amount, price);
-    }
-
-    private static ImmutableBitSet setOf(int... values) {
-        BitSet set = new BitSet();
-        for (int i : values)
-            set.set(i);
-        return new ImmutableBitSet(set);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/FileSystem.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/FileSystem.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/FileSystem.java
deleted file mode 100644
index db245fd..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/FileSystem.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package org.apache.kylin.storage.gridtable.diskstore;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- */
-interface FileSystem {
-
-    boolean checkExistence(String path);
-
-    boolean delete(String path);
-
-    void deleteOnExit(String path);
-
-    boolean createDirectory(String path);
-
-    boolean createFile(String path);
-
-    OutputStream getWriter(String path);
-
-    InputStream getReader(String path);
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/GTDiskStore.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/GTDiskStore.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/GTDiskStore.java
deleted file mode 100644
index 4dcacf7..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/GTDiskStore.java
+++ /dev/null
@@ -1,179 +0,0 @@
-package org.apache.kylin.storage.gridtable.diskstore;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.UUID;
-
-import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.storage.gridtable.GTInfo;
-import org.apache.kylin.storage.gridtable.GTRecord;
-import org.apache.kylin.storage.gridtable.GTRowBlock;
-import org.apache.kylin.storage.gridtable.GTScanRequest;
-import org.apache.kylin.storage.gridtable.IGTStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
-
-/**
- */
-public class GTDiskStore implements IGTStore {
-
-    private static final Logger logger = LoggerFactory.getLogger(GTDiskStore.class);
-
-    private final String identifier;
-    private final FileSystem fileSystem;
-    private final DiskStoreWriter writer;
-    private final GTInfo gtInfo;
-
-    public GTDiskStore(GTInfo gtInfo) {
-        this.gtInfo = gtInfo;
-        this.fileSystem = new LocalFileSystem();
-        this.identifier = generateIdentifier(fileSystem);
-        logger.info("disk store created, identifier:" + identifier);
-        this.writer = new DiskStoreWriter(fileSystem.getWriter(getRowBlockFile(identifier)));
-        deleteTmpFilesOnExit();
-    }
-
-    @Override
-    public GTInfo getInfo() {
-        return gtInfo;
-    }
-
-    private String generateIdentifier(FileSystem fs) {
-        int tryCount = 0;
-        while (true) {
-            //uuid may conflict
-            String identifier = UUID.randomUUID().toString();
-            final String path = getRootDirectory(identifier);
-            if (fs.createDirectory(path)) {
-                return identifier;
-            } else {
-                logger.warn("failed to create dir " + path);
-                if (++tryCount > 5) {
-                    throw new RuntimeException("failed to generateIdentifier");
-                }
-            }
-        }
-    }
-
-    private String getRootDirectory(String identifier) {
-        return "/tmp/kylin_gtdiskstore_" + identifier;
-    }
-
-    private String getRowBlockFile(String identifier) {
-        return getRootDirectory(identifier) + "/rowblock";
-    }
-
-    private class DiskStoreWriter implements IGTStoreWriter {
-
-        private final DataOutputStream outputStream;
-
-        DiskStoreWriter(OutputStream outputStream) {
-            this.outputStream = new DataOutputStream(outputStream);
-        }
-
-        @Override
-        public void write(GTRowBlock block) throws IOException {
-            final int blockSize = block.exportLength();
-            outputStream.writeInt(blockSize);
-            block.export(outputStream);
-            outputStream.flush();
-        }
-
-        @Override
-        public void close() throws IOException {
-            outputStream.close();
-        }
-    }
-
-    public long memoryUsage() {
-        return 0;
-    }
-
-    @Override
-    public IGTStoreWriter rebuild(int shard) throws IOException {
-        return writer;
-    }
-
-    @Override
-    public IGTStoreWriter append(int shard, GTRowBlock.Writer fillLast) throws IOException {
-        return writer;
-    }
-
-    private class DiskStoreScanner implements IGTStoreScanner {
-
-        private final DataInputStream inputStream;
-        private int blockSize = 0;
-
-        DiskStoreScanner(InputStream inputStream) {
-            this.inputStream = new DataInputStream(inputStream);
-        }
-
-        @Override
-        public void close() throws IOException {
-            inputStream.close();
-        }
-
-        @Override
-        public boolean hasNext() {
-            try {
-                blockSize = inputStream.readInt();
-                return blockSize > 0;
-            } catch (EOFException e) {
-                return false;
-            } catch (IOException e) {
-                logger.error("input stream fail", e);
-                throw new RuntimeException(e);
-            }
-        }
-
-        @Override
-        public GTRowBlock next() {
-            GTRowBlock block = new GTRowBlock(gtInfo);
-            ByteBuffer buffer = ByteBuffer.allocate(blockSize);
-            int count = blockSize;
-            while (count > 0) {
-                try {
-                    count -= inputStream.read(buffer.array(), buffer.position(), buffer.remaining());
-                } catch (IOException e) {
-                    logger.error("input stream fail", e);
-                    throw new RuntimeException(e);
-                }
-            }
-            Preconditions.checkArgument(count == 0, "invalid read count:" + count + " block size:" + blockSize);
-            block.load(buffer);
-            return block;
-        }
-
-        @Override
-        public void remove() {
-            throw new UnsupportedOperationException();
-        }
-    }
-
-    @Override
-    public IGTStoreScanner scan(GTRecord pkStart, GTRecord pkEnd, ImmutableBitSet selectedColBlocks, GTScanRequest additionalPushDown) throws IOException {
-        return new DiskStoreScanner(fileSystem.getReader(getRowBlockFile(identifier)));
-    }
-
-    public void drop() throws IOException {
-        try {
-            writer.close();
-        } catch (Exception e) {
-            logger.error("error to close writer", e);
-        }
-        deleteTmpFilesOnExit();
-    }
-
-    private void deleteTmpFilesOnExit() {
-        fileSystem.deleteOnExit(getRowBlockFile(identifier));
-        fileSystem.deleteOnExit(getRootDirectory(identifier));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/HadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/HadoopFileSystem.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/HadoopFileSystem.java
deleted file mode 100644
index 429a30b..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/HadoopFileSystem.java
+++ /dev/null
@@ -1,97 +0,0 @@
-package org.apache.kylin.storage.gridtable.diskstore;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.kylin.engine.mr.HadoopUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- */
-class HadoopFileSystem implements FileSystem {
-
-    private static final Logger logger = LoggerFactory.getLogger(HadoopFileSystem.class);
-
-    final org.apache.hadoop.fs.FileSystem fileSystem;
-
-    HadoopFileSystem() {
-        try {
-            fileSystem = org.apache.hadoop.fs.FileSystem.get(HadoopUtil.getCurrentConfiguration());
-        } catch (IOException e) {
-            logger.error("error construct HadoopFileSystem", e);
-            throw new RuntimeException(e);
-        }
-    }
-    @Override
-    public boolean checkExistence(String path) {
-        try {
-            return fileSystem.exists(new Path(path));
-        } catch (IOException e) {
-            logger.error("error checkExistence, path:" + path, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public boolean delete(String path) {
-        try {
-            return fileSystem.delete(new Path(path), true);
-        } catch (IOException e) {
-            logger.error("error delete, path:" + path, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public void deleteOnExit(String path) {
-        try {
-            fileSystem.deleteOnExit(new Path(path));
-        } catch (IOException e) {
-            logger.error("error deleteOnExit, path:" + path, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public boolean createDirectory(String path) {
-        try {
-            return fileSystem.mkdirs(new Path(path));
-        } catch (IOException e) {
-            logger.error("error createDirectory, path:" + path, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public boolean createFile(String path) {
-        try {
-            return fileSystem.createNewFile(new Path(path));
-        } catch (IOException e) {
-            logger.error("error createFile, path:" + path, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public OutputStream getWriter(String path) {
-        try {
-            return fileSystem.create(new Path(path));
-        } catch (IOException e) {
-            logger.error("error getWriter, path:" + path, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public InputStream getReader(String path) {
-        try {
-            return fileSystem.open(new Path(path));
-        } catch (IOException e) {
-            logger.error("error getReader, path:" + path, e);
-            throw new RuntimeException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/LocalFileSystem.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/LocalFileSystem.java b/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/LocalFileSystem.java
deleted file mode 100644
index 0d07f3b..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/gridtable/diskstore/LocalFileSystem.java
+++ /dev/null
@@ -1,67 +0,0 @@
-package org.apache.kylin.storage.gridtable.diskstore;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-
-/**
- */
-class LocalFileSystem implements FileSystem {
-
-    private static Logger logger = LoggerFactory.getLogger(LocalFileSystem.class);
-
-    LocalFileSystem(){}
-
-    @Override
-    public boolean checkExistence(String path) {
-        return new File(path).exists();
-    }
-
-    @Override
-    public boolean delete(String path) {
-        return new File(path).delete();
-    }
-
-    @Override
-    public void deleteOnExit(String path) {
-        new File(path).deleteOnExit();
-    }
-
-    @Override
-    public boolean createDirectory(String path) {
-        return new File(path).mkdirs();
-    }
-
-    @Override
-    public boolean createFile(String path) {
-        try {
-            return new File(path).createNewFile();
-        } catch (IOException e) {
-            logger.warn("create file failed:" + path, e);
-            return false;
-        }
-    }
-
-    @Override
-    public OutputStream getWriter(String path) {
-        try {
-            return new FileOutputStream(path);
-        } catch (FileNotFoundException e) {
-            //should not happen
-            logger.error("path:" + path + " out found");
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public InputStream getReader(String path) {
-        try {
-            return new FileInputStream(path);
-        } catch (FileNotFoundException e) {
-            //should not happen
-            logger.error("path:" + path + " out found");
-            throw new RuntimeException(e);
-        }
-    }
-}



[22/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java b/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
new file mode 100644
index 0000000..932da38
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.IntermediateColumnDesc;
+import org.apache.kylin.metadata.model.JoinDesc;
+import org.apache.kylin.metadata.model.LookupDesc;
+import org.apache.kylin.metadata.model.PartitionDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.w3c.dom.Document;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+
+public class JoinedFlatTable {
+
+    public static final String FACT_TABLE_ALIAS = "FACT_TABLE";
+
+    public static final String LOOKUP_TABLE_ALAIS_PREFIX = "LOOKUP_";
+
+    public static String getTableDir(IJoinedFlatTableDesc intermediateTableDesc, String storageDfsDir) {
+        return storageDfsDir + "/" + intermediateTableDesc.getTableName();
+    }
+
+    public static String generateCreateTableStatement(IJoinedFlatTableDesc intermediateTableDesc, String storageDfsDir) {
+        StringBuilder ddl = new StringBuilder();
+
+        ddl.append("CREATE EXTERNAL TABLE IF NOT EXISTS " + intermediateTableDesc.getTableName() + "\n");
+
+        ddl.append("(" + "\n");
+        for (int i = 0; i < intermediateTableDesc.getColumnList().size(); i++) {
+            IntermediateColumnDesc col = intermediateTableDesc.getColumnList().get(i);
+            if (i > 0) {
+                ddl.append(",");
+            }
+            ddl.append(colName(col.getCanonicalName()) + " " + getHiveDataType(col.getDataType()) + "\n");
+        }
+        ddl.append(")" + "\n");
+
+        ddl.append("ROW FORMAT DELIMITED FIELDS TERMINATED BY '\\177'" + "\n");
+        ddl.append("STORED AS SEQUENCEFILE" + "\n");
+        ddl.append("LOCATION '" + storageDfsDir + "/" + intermediateTableDesc.getTableName() + "';").append("\n");
+        // ddl.append("TBLPROPERTIES ('serialization.null.format'='\\\\N')" +
+        // ";\n");
+        return ddl.toString();
+    }
+
+    public static String generateDropTableStatement(IJoinedFlatTableDesc intermediateTableDesc) {
+        StringBuilder ddl = new StringBuilder();
+        ddl.append("DROP TABLE IF EXISTS " + intermediateTableDesc.getTableName() + ";").append("\n");
+        return ddl.toString();
+    }
+
+    public static String generateInsertDataStatement(IJoinedFlatTableDesc intermediateTableDesc, JobEngineConfig engineConfig) throws IOException {
+        StringBuilder sql = new StringBuilder();
+
+        File hadoopPropertiesFile = new File(engineConfig.getHiveConfFilePath());
+
+        if (hadoopPropertiesFile.exists()) {
+            DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+            DocumentBuilder builder;
+            Document doc;
+            try {
+                builder = factory.newDocumentBuilder();
+                doc = builder.parse(hadoopPropertiesFile);
+                NodeList nl = doc.getElementsByTagName("property");
+                for (int i = 0; i < nl.getLength(); i++) {
+                    String name = doc.getElementsByTagName("name").item(i).getFirstChild().getNodeValue();
+                    String value = doc.getElementsByTagName("value").item(i).getFirstChild().getNodeValue();
+                    if (name.equals("tmpjars") == false) {
+                        sql.append("SET " + name + "=" + value + ";").append("\n");
+                    }
+                }
+
+            } catch (ParserConfigurationException e) {
+                throw new IOException(e);
+            } catch (SAXException e) {
+                throw new IOException(e);
+            }
+        }
+
+        sql.append("INSERT OVERWRITE TABLE " + intermediateTableDesc.getTableName() + " " + generateSelectDataStatement(intermediateTableDesc) + ";").append("\n");
+
+        return sql.toString();
+    }
+
+    public static String generateSelectDataStatement(IJoinedFlatTableDesc intermediateTableDesc) {
+        StringBuilder sql = new StringBuilder();
+        sql.append("SELECT" + "\n");
+        String tableAlias;
+        Map<String, String> tableAliasMap = buildTableAliasMap(intermediateTableDesc.getDataModel());
+        for (int i = 0; i < intermediateTableDesc.getColumnList().size(); i++) {
+            IntermediateColumnDesc col = intermediateTableDesc.getColumnList().get(i);
+            if (i > 0) {
+                sql.append(",");
+            }
+            tableAlias = tableAliasMap.get(col.getTableName());
+            sql.append(tableAlias + "." + col.getColumnName() + "\n");
+        }
+        appendJoinStatement(intermediateTableDesc, sql, tableAliasMap);
+        appendWhereStatement(intermediateTableDesc, sql, tableAliasMap);
+        return sql.toString();
+    }
+
+    private static Map<String, String> buildTableAliasMap(DataModelDesc dataModelDesc) {
+        Map<String, String> tableAliasMap = new HashMap<String, String>();
+
+        tableAliasMap.put(dataModelDesc.getFactTable().toUpperCase(), FACT_TABLE_ALIAS);
+
+        int i = 1;
+        for (LookupDesc lookupDesc: dataModelDesc.getLookups()) {
+            JoinDesc join = lookupDesc.getJoin();
+            if (join != null) {
+                tableAliasMap.put(lookupDesc.getTable().toUpperCase(), LOOKUP_TABLE_ALAIS_PREFIX + i);
+                i++;
+            }
+
+        }
+        return tableAliasMap;
+    }
+
+    private static void appendJoinStatement(IJoinedFlatTableDesc intermediateTableDesc, StringBuilder sql, Map<String, String> tableAliasMap) {
+        Set<String> dimTableCache = new HashSet<String>();
+
+        DataModelDesc dataModelDesc = intermediateTableDesc.getDataModel();
+        String factTableName = dataModelDesc.getFactTable();
+        String factTableAlias = tableAliasMap.get(factTableName);
+        sql.append("FROM " + factTableName + " as " + factTableAlias + " \n");
+
+        for (LookupDesc lookupDesc : dataModelDesc.getLookups()) {
+            JoinDesc join = lookupDesc.getJoin();
+            if (join != null && join.getType().equals("") == false) {
+                String joinType = join.getType().toUpperCase();
+                String dimTableName = lookupDesc.getTable();
+                if (!dimTableCache.contains(dimTableName)) {
+                    TblColRef[] pk = join.getPrimaryKeyColumns();
+                    TblColRef[] fk = join.getForeignKeyColumns();
+                    if (pk.length != fk.length) {
+                        throw new RuntimeException("Invalid join condition of lookup table:" + lookupDesc);
+                    }
+                    sql.append(joinType + " JOIN " + dimTableName + " as " + tableAliasMap.get(dimTableName) + "\n");
+                    sql.append("ON ");
+                    for (int i = 0; i < pk.length; i++) {
+                        if (i > 0) {
+                            sql.append(" AND ");
+                        }
+                        sql.append(factTableAlias + "." + fk[i].getName() + " = " + tableAliasMap.get(dimTableName) + "." + pk[i].getName());
+                    }
+                    sql.append("\n");
+
+                    dimTableCache.add(dimTableName);
+                }
+            }
+        }
+    }
+
+    private static void appendWhereStatement(IJoinedFlatTableDesc intermediateTableDesc, StringBuilder sql, Map<String, String> tableAliasMap) {
+        if (!(intermediateTableDesc instanceof CubeJoinedFlatTableDesc)) {
+            return;//TODO: for now only cube segments support filter and partition
+        }
+        CubeJoinedFlatTableDesc desc = (CubeJoinedFlatTableDesc) intermediateTableDesc;
+
+        boolean hasCondition = false;
+        StringBuilder whereBuilder = new StringBuilder();
+        whereBuilder.append("WHERE");
+
+        CubeDesc cubeDesc = desc.getCubeDesc();
+        DataModelDesc model = cubeDesc.getModel();
+        
+        if (model.getFilterCondition() != null && model.getFilterCondition().equals("") == false) {
+            whereBuilder.append(" (").append(model.getFilterCondition()).append(") ");
+            hasCondition = true;
+        }
+
+        CubeSegment cubeSegment = desc.getCubeSegment();
+
+        if (null != cubeSegment) {
+            PartitionDesc partDesc = model.getPartitionDesc();
+            long dateStart = cubeSegment.getDateRangeStart();
+            long dateEnd = cubeSegment.getDateRangeEnd();
+
+            if (!(dateStart == 0 && dateEnd == Long.MAX_VALUE)) {
+                whereBuilder.append(hasCondition ? " AND (" : " (");
+                whereBuilder.append(partDesc.getPartitionConditionBuilder().buildDateRangeCondition(partDesc, dateStart, dateEnd, tableAliasMap));
+                whereBuilder.append(")\n");
+                hasCondition = true;
+            }
+        }
+
+        if (hasCondition) {
+            sql.append(whereBuilder.toString());
+        }
+    }
+
+    private static String colName(String canonicalColName) {
+        return canonicalColName.replace(".", "_");
+    }
+    
+    private static String getHiveDataType(String javaDataType) {
+        String hiveDataType = javaDataType.toLowerCase().startsWith("varchar") ? "string" : javaDataType;
+        hiveDataType = javaDataType.toLowerCase().startsWith("integer") ? "int" : hiveDataType;
+
+        return hiveDataType.toLowerCase();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/Scheduler.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/Scheduler.java b/core-job/src/main/java/org/apache/kylin/job/Scheduler.java
new file mode 100644
index 0000000..2ed2fc2
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/Scheduler.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job;
+
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.exception.SchedulerException;
+import org.apache.kylin.job.execution.Executable;
+import org.apache.kylin.job.lock.JobLock;
+
+/**
+ */
+public interface Scheduler<T extends Executable> {
+
+    void init(JobEngineConfig jobEngineConfig, JobLock jobLock) throws SchedulerException;
+
+    void shutdown() throws SchedulerException;
+
+    boolean stop(T executable) throws SchedulerException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/cmd/BaseCommandOutput.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/cmd/BaseCommandOutput.java b/core-job/src/main/java/org/apache/kylin/job/cmd/BaseCommandOutput.java
new file mode 100644
index 0000000..29b5324
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/cmd/BaseCommandOutput.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.cmd;
+
+/**
+ */
+public abstract class BaseCommandOutput implements ICommandOutput {
+
+    @Override
+    public void log(String message) {
+        this.appendOutput(message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/cmd/ICommandOutput.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/cmd/ICommandOutput.java b/core-job/src/main/java/org/apache/kylin/job/cmd/ICommandOutput.java
new file mode 100644
index 0000000..6cab6a3
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/cmd/ICommandOutput.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.cmd;
+
+import org.apache.kylin.common.util.Logger;
+import org.apache.kylin.job.constant.JobStepStatusEnum;
+
+/**
+ * @author xjiang
+ * 
+ */
+public interface ICommandOutput extends Logger {
+
+    public void setStatus(JobStepStatusEnum status);
+
+    public JobStepStatusEnum getStatus();
+
+    public void appendOutput(String message);
+
+    public String getOutput();
+
+    public void setExitCode(int exitCode);
+
+    public int getExitCode();
+
+    public void reset();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/cmd/IJobCommand.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/cmd/IJobCommand.java b/core-job/src/main/java/org/apache/kylin/job/cmd/IJobCommand.java
new file mode 100644
index 0000000..5a47173
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/cmd/IJobCommand.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job.cmd;
+
+import org.apache.kylin.job.exception.JobException;
+
+/**
+ * @author xjiang
+ * 
+ */
+public interface IJobCommand {
+
+    public ICommandOutput execute() throws JobException;
+
+    public void cancel() throws JobException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java b/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
new file mode 100644
index 0000000..6a718fc
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.cmd;
+
+import org.apache.kylin.common.util.CliCommandExecutor;
+import org.apache.kylin.job.constant.JobStepStatusEnum;
+import org.apache.kylin.job.exception.JobException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.*;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class ShellCmd implements IJobCommand {
+
+    private static Logger log = LoggerFactory.getLogger(ShellCmd.class);
+
+    private final String executeCommand;
+    private final ICommandOutput output;
+    private final boolean isAsync;
+    private final CliCommandExecutor cliCommandExecutor;
+
+    private FutureTask<Integer> future;
+
+    private ShellCmd(String executeCmd, ICommandOutput out, String host, int port, String user, String password, boolean async) {
+        this.executeCommand = executeCmd;
+        this.output = out;
+        this.cliCommandExecutor = new CliCommandExecutor();
+        this.cliCommandExecutor.setRunAtRemote(host, port, user, password);
+        this.isAsync = async;
+    }
+
+    public ShellCmd(String executeCmd, String host, int port, String user, String password, boolean async) {
+        this(executeCmd, new ShellCmdOutput(), host, port, user, password, async);
+    }
+
+    @Override
+    public ICommandOutput execute() throws JobException {
+
+        final ExecutorService executor = Executors.newSingleThreadExecutor();
+        future = new FutureTask<Integer>(new Callable<Integer>() {
+            public Integer call() throws JobException, IOException {
+                executor.shutdown();
+                return executeCommand(executeCommand);
+            }
+        });
+        executor.execute(future);
+
+        int exitCode = -1;
+        if (!isAsync) {
+            try {
+                exitCode = future.get();
+                log.info("finish executing");
+            } catch (CancellationException e) {
+                log.debug("Command is cancelled");
+                exitCode = -2;
+            } catch (Exception e) {
+                throw new JobException("Error when execute job " + executeCommand, e);
+            } finally {
+                if (exitCode == 0) {
+                    output.setStatus(JobStepStatusEnum.FINISHED);
+                } else if (exitCode == -2) {
+                    output.setStatus(JobStepStatusEnum.DISCARDED);
+                } else {
+                    output.setStatus(JobStepStatusEnum.ERROR);
+                }
+                output.setExitCode(exitCode);
+            }
+        }
+        return output;
+    }
+
+    protected int executeCommand(String command) throws JobException, IOException {
+        output.reset();
+        output.setStatus(JobStepStatusEnum.RUNNING);
+        return cliCommandExecutor.execute(command, output).getFirst();
+    }
+
+    @Override
+    public void cancel() throws JobException {
+        future.cancel(true);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java b/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
new file mode 100644
index 0000000..ebcad47
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.cmd;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.kylin.job.constant.JobStepStatusEnum;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class ShellCmdOutput extends BaseCommandOutput implements ICommandOutput {
+
+    protected static final Logger log = LoggerFactory.getLogger(ShellCmdOutput.class);
+
+    protected StringBuilder output;
+    protected int exitCode;
+    protected JobStepStatusEnum status;
+
+    public ShellCmdOutput() {
+        init();
+    }
+
+    private void init() {
+        output = new StringBuilder();
+        exitCode = -1;
+        status = JobStepStatusEnum.NEW;
+    }
+
+    @Override
+    public JobStepStatusEnum getStatus() {
+        return status;
+    }
+
+    @Override
+    public void setStatus(JobStepStatusEnum s) {
+        this.status = s;
+    }
+
+    @Override
+    public String getOutput() {
+        return output.toString();
+    }
+
+    @Override
+    public void appendOutput(String message) {
+        output.append(message).append(System.getProperty("line.separator"));
+        log.debug(message);
+    }
+
+    @Override
+    public int getExitCode() {
+        return exitCode;
+    }
+
+    @Override
+    public void setExitCode(int code) {
+        exitCode = code;
+    }
+
+    @Override
+    public void reset() {
+        init();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/common/OptionsHelper.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/common/OptionsHelper.java b/core-job/src/main/java/org/apache/kylin/job/common/OptionsHelper.java
new file mode 100644
index 0000000..1cda348
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/common/OptionsHelper.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.common;
+
+import java.io.File;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+
+/**
+ */
+public class OptionsHelper {
+    private CommandLine commandLine;
+
+    public void parseOptions(Options options, String[] args) throws ParseException {
+        CommandLineParser parser = new GnuParser();
+        commandLine = parser.parse(options, args);
+    }
+
+    public Option[] getOptions() {
+        return commandLine.getOptions();
+    }
+
+    public String getOptionsAsString() {
+        StringBuilder buf = new StringBuilder();
+        for (Option option : commandLine.getOptions()) {
+            buf.append(" ");
+            buf.append(option.getOpt());
+            if (option.hasArg()) {
+                buf.append("=");
+                buf.append(option.getValue());
+            }
+        }
+        return buf.toString();
+    }
+
+    public String getOptionValue(Option option) {
+        return commandLine.getOptionValue(option.getOpt());
+    }
+
+    public boolean hasOption(Option option) {
+        return commandLine.hasOption(option.getOpt());
+    }
+
+    public void printUsage(String programName, Options options) {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp(programName, options);
+    }
+
+    public static String convertToFileURL(String path) {
+        if (File.separatorChar != '/') {
+            path = path.replace(File.separatorChar, '/');
+        }
+
+        return path;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/common/ShellExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/common/ShellExecutable.java b/core-job/src/main/java/org/apache/kylin/job/common/ShellExecutable.java
new file mode 100644
index 0000000..786698e
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/common/ShellExecutable.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.common;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.kylin.common.util.Pair;
+
+import com.google.common.collect.Maps;
+import org.apache.kylin.common.util.Logger;
+import org.apache.kylin.job.constant.ExecutableConstants;
+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;
+
+/**
+ */
+public class ShellExecutable extends AbstractExecutable {
+
+    private static final String CMD = "cmd";
+
+    public ShellExecutable() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        try {
+            logger.info("executing:" + getCmd());
+            final ShellExecutableLogger logger = new ShellExecutableLogger();
+            final Pair<Integer, String> result = context.getConfig().getCliCommandExecutor().execute(getCmd(), logger);
+            executableManager.addJobInfo(getId(), logger.getInfo());
+            return new ExecuteResult(result.getFirst() == 0? ExecuteResult.State.SUCCEED: ExecuteResult.State.FAILED, result.getSecond());
+        } catch (IOException e) {
+            logger.error("job:" + getId() + " execute finished with exception", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+
+    public void setCmd(String cmd) {
+        setParam(CMD, cmd);
+    }
+
+    public String getCmd() {
+        return getParam(CMD);
+    }
+
+    private static class ShellExecutableLogger implements Logger {
+
+        private final Map<String, String> info = Maps.newHashMap();
+
+        private static final Pattern PATTERN_APP_ID = Pattern.compile("Submitted application (.*?) to ResourceManager");
+        private static final Pattern PATTERN_APP_URL = Pattern.compile("The url to track the job: (.*)");
+        private static final Pattern PATTERN_JOB_ID = Pattern.compile("Running job: (.*)");
+        private static final Pattern PATTERN_HDFS_BYTES_WRITTEN = Pattern.compile("(?:HD|MAPR)FS: Number of bytes written=(\\d+)");
+        private static final Pattern PATTERN_SOURCE_RECORDS_COUNT = Pattern.compile("Map input records=(\\d+)");
+        private static final Pattern PATTERN_SOURCE_RECORDS_SIZE = Pattern.compile("(?:HD|MAPR)FS Read: (\\d+) HDFS Write");
+
+        // hive
+        private static final Pattern PATTERN_HIVE_APP_ID_URL = Pattern.compile("Starting Job = (.*?), Tracking URL = (.*)");
+        private static final Pattern PATTERN_HIVE_BYTES_WRITTEN = Pattern.compile("(?:HD|MAPR)FS Read: (\\d+) HDFS Write: (\\d+) SUCCESS");
+
+        @Override
+        public void log(String message) {
+            Matcher matcher = PATTERN_APP_ID.matcher(message);
+            if (matcher.find()) {
+                String appId = matcher.group(1);
+                info.put(ExecutableConstants.YARN_APP_ID, appId);
+            }
+
+            matcher = PATTERN_APP_URL.matcher(message);
+            if (matcher.find()) {
+                String appTrackingUrl = matcher.group(1);
+                info.put(ExecutableConstants.YARN_APP_URL, appTrackingUrl);
+            }
+
+            matcher = PATTERN_JOB_ID.matcher(message);
+            if (matcher.find()) {
+                String mrJobID = matcher.group(1);
+                info.put(ExecutableConstants.MR_JOB_ID, mrJobID);
+            }
+
+            matcher = PATTERN_HDFS_BYTES_WRITTEN.matcher(message);
+            if (matcher.find()) {
+                String hdfsWritten = matcher.group(1);
+                info.put(ExecutableConstants.HDFS_BYTES_WRITTEN, hdfsWritten);
+            }
+
+            matcher = PATTERN_SOURCE_RECORDS_COUNT.matcher(message);
+            if (matcher.find()) {
+                String sourceCount = matcher.group(1);
+                info.put(ExecutableConstants.SOURCE_RECORDS_COUNT, sourceCount);
+            }
+
+            matcher = PATTERN_SOURCE_RECORDS_SIZE.matcher(message);
+            if (matcher.find()) {
+                String sourceSize = matcher.group(1);
+                info.put(ExecutableConstants.SOURCE_RECORDS_SIZE, sourceSize);
+            }
+
+            // hive
+            matcher = PATTERN_HIVE_APP_ID_URL.matcher(message);
+            if (matcher.find()) {
+                String jobId = matcher.group(1);
+                String trackingUrl = matcher.group(2);
+                info.put(ExecutableConstants.MR_JOB_ID, jobId);
+                info.put(ExecutableConstants.YARN_APP_URL, trackingUrl);
+            }
+
+            matcher = PATTERN_HIVE_BYTES_WRITTEN.matcher(message);
+            if (matcher.find()) {
+                // String hdfsRead = matcher.group(1);
+                String hdfsWritten = matcher.group(2);
+                info.put(ExecutableConstants.HDFS_BYTES_WRITTEN, hdfsWritten);
+            }
+        }
+
+        Map<String, String> getInfo() {
+            return info;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/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
new file mode 100644
index 0000000..fdcfdbe
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.constant;
+
+/**
+ */
+public final class ExecutableConstants {
+
+    private ExecutableConstants(){}
+
+    public static final String YARN_APP_ID = "yarn_application_id";
+
+    public static final String YARN_APP_URL = "yarn_application_tracking_url";
+    public static final String MR_JOB_ID = "mr_job_id";
+    public static final String HDFS_BYTES_WRITTEN = "hdfs_bytes_written";
+    public static final String SOURCE_RECORDS_COUNT = "source_records_count";
+    public static final String SOURCE_RECORDS_SIZE = "source_records_size";
+    public static final String GLOBAL_LISTENER_NAME = "ChainListener";
+
+
+
+
+    public static final int DEFAULT_SCHEDULER_INTERVAL_SECONDS = 60;
+
+    public static final String CUBE_JOB_GROUP_NAME = "cube_job_group";
+
+    public static final String DAEMON_JOB_GROUP_NAME = "daemon_job_group";
+    public static final String STEP_NAME_BUILD_DICTIONARY = "Build Dimension Dictionary";
+
+    public static final String STEP_NAME_CREATE_FLAT_HIVE_TABLE = "Create Intermediate Flat Hive Table";
+    public static final String STEP_NAME_FACT_DISTINCT_COLUMNS = "Extract Fact Table Distinct Columns";
+    public static final String STEP_NAME_BUILD_BASE_CUBOID = "Build Base Cuboid Data";
+    public static final String STEP_NAME_BUILD_IN_MEM_CUBE = "Build Cube";
+    public static final String STEP_NAME_BUILD_N_D_CUBOID = "Build N-Dimension Cuboid Data";
+    public static final String STEP_NAME_GET_CUBOID_KEY_DISTRIBUTION = "Calculate HTable Region Splits";
+    public static final String STEP_NAME_CREATE_HBASE_TABLE = "Create HTable";
+    public static final String STEP_NAME_CONVERT_CUBOID_TO_HFILE = "Convert Cuboid Data to HFile";
+    public static final String STEP_NAME_BULK_LOAD_HFILE = "Load HFile to HBase Table";
+    public static final String STEP_NAME_MERGE_DICTIONARY = "Merge Cuboid Dictionary";
+    public static final String STEP_NAME_MERGE_STATISTICS = "Merge Cuboid Statistics";
+    public static final String STEP_NAME_SAVE_STATISTICS = "Save Cuboid Statistics";
+    public static final String STEP_NAME_MERGE_CUBOID = "Merge Cuboid Data";
+    public static final String STEP_NAME_UPDATE_CUBE_INFO = "Update Cube Info";
+    public static final String STEP_NAME_GARBAGE_COLLECTION = "Garbage Collection";
+    
+    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 PROP_ENGINE_CONTEXT = "jobengineConfig";
+    public static final String PROP_JOB_FLOW = "jobFlow";
+    public static final String PROP_JOBINSTANCE_UUID = "jobInstanceUuid";
+    public static final String PROP_JOBSTEP_SEQ_ID = "jobStepSequenceID";
+    public static final String PROP_COMMAND = "command";
+    // public static final String PROP_STORAGE_LOCATION =
+    // "storageLocationIdentifier";
+    public static final String PROP_JOB_ASYNC = "jobAsync";
+    public static final String PROP_JOB_CMD_EXECUTOR = "jobCmdExecutor";
+    public static final String PROP_JOB_CMD_OUTPUT = "jobCmdOutput";
+    public static final String PROP_JOB_KILLED = "jobKilled";
+    public static final String PROP_JOB_RUNTIME_FLOWS = "jobFlows";
+
+    public static final String NOTIFY_EMAIL_TEMPLATE = "<div><b>Build Result of Job ${job_name}</b><pre><ul>" + "<li>Build Result: <b>${result}</b></li>" + "<li>Job Engine: ${job_engine}</li>" + "<li>Cube Name: ${cube_name}</li>" + "<li>Start Time: ${start_time}</li>" + "<li>Duration: ${duration}</li>" + "<li>MR Waiting: ${mr_waiting}</li>" + "<li>Last Update Time: ${last_update_time}</li>" + "<li>Submitter: ${submitter}</li>" + "<li>Error Log: ${error_log}</li>" + "</ul></pre><div/>";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/constant/JobStatusEnum.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/constant/JobStatusEnum.java b/core-job/src/main/java/org/apache/kylin/job/constant/JobStatusEnum.java
new file mode 100644
index 0000000..a4ef564
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/constant/JobStatusEnum.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.constant;
+
+public enum JobStatusEnum {
+
+    NEW(0), PENDING(1), RUNNING(2), FINISHED(4), ERROR(8), DISCARDED(16);
+
+    private final int code;
+
+    private JobStatusEnum(int statusCode) {
+        this.code = statusCode;
+    }
+
+    public static JobStatusEnum getByCode(int statusCode) {
+        for (JobStatusEnum status : values()) {
+            if (status.getCode() == statusCode) {
+                return status;
+            }
+        }
+
+        return null;
+    }
+
+    public int getCode() {
+        return this.code;
+    }
+
+    public boolean isComplete() {
+        return code == JobStatusEnum.FINISHED.getCode() || code == JobStatusEnum.ERROR.getCode() || code == JobStatusEnum.DISCARDED.getCode();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/constant/JobStepCmdTypeEnum.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/constant/JobStepCmdTypeEnum.java b/core-job/src/main/java/org/apache/kylin/job/constant/JobStepCmdTypeEnum.java
new file mode 100644
index 0000000..02b40a3
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/constant/JobStepCmdTypeEnum.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.constant;
+
+/**
+ * @author xduo, ysong1
+ * 
+ */
+public enum JobStepCmdTypeEnum {
+    SHELL_CMD, SHELL_CMD_HADOOP, JAVA_CMD_HADOOP_FACTDISTINCT, JAVA_CMD_HADOOP_BASECUBOID, JAVA_CMD_HADOOP_NDCUBOID, JAVA_CMD_HADOOP_RANGEKEYDISTRIBUTION, JAVA_CMD_HADOOP_CONVERTHFILE, JAVA_CMD_HADOOP_MERGECUBOID, JAVA_CMD_HADOOP_NO_MR_DICTIONARY, JAVA_CMD_HADDOP_NO_MR_CREATEHTABLE, JAVA_CMD_HADOOP_NO_MR_BULKLOAD
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/constant/JobStepStatusEnum.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/constant/JobStepStatusEnum.java b/core-job/src/main/java/org/apache/kylin/job/constant/JobStepStatusEnum.java
new file mode 100644
index 0000000..08ee79a
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/constant/JobStepStatusEnum.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.constant;
+
+public enum JobStepStatusEnum {
+    NEW(0), PENDING(1), RUNNING(2), FINISHED(4), ERROR(8), DISCARDED(16), WAITING(32), KILLED(64);
+
+    private final int code;
+
+    private JobStepStatusEnum(int statusCode) {
+        this.code = statusCode;
+    }
+
+    public static JobStepStatusEnum getByCode(int statusCode) {
+        for (JobStepStatusEnum status : values()) {
+            if (status.getCode() == statusCode) {
+                return status;
+            }
+        }
+
+        return null;
+    }
+
+    public int getCode() {
+        return this.code;
+    }
+
+    public boolean isComplete() {
+        return code == JobStepStatusEnum.FINISHED.getCode() || code == JobStepStatusEnum.ERROR.getCode() || code == JobStepStatusEnum.DISCARDED.getCode();
+    }
+
+    public boolean isRunable() {
+        return code == JobStepStatusEnum.PENDING.getCode() || code == JobStepStatusEnum.ERROR.getCode();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
new file mode 100644
index 0000000..4862bb1
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.dao;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.io.IOUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.JsonSerializer;
+import org.apache.kylin.common.persistence.RawResource;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.persistence.Serializer;
+import org.apache.kylin.job.exception.PersistentException;
+import org.apache.kylin.metadata.MetadataManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ */
+public class ExecutableDao {
+
+    private static final Serializer<ExecutablePO> JOB_SERIALIZER = new JsonSerializer<ExecutablePO>(ExecutablePO.class);
+    private static final Serializer<ExecutableOutputPO> JOB_OUTPUT_SERIALIZER = new JsonSerializer<ExecutableOutputPO>(ExecutableOutputPO.class);
+    private static final Logger logger = LoggerFactory.getLogger(ExecutableDao.class);
+    private static final ConcurrentHashMap<KylinConfig, ExecutableDao> CACHE = new ConcurrentHashMap<KylinConfig, ExecutableDao>();
+    public static final String JOB_PATH_ROOT = "/execute";
+    public static final String JOB_OUTPUT_ROOT = "/execute_output";
+
+    private ResourceStore store;
+
+    public static ExecutableDao getInstance(KylinConfig config) {
+        ExecutableDao r = CACHE.get(config);
+        if (r == null) {
+            r = new ExecutableDao(config);
+            CACHE.put(config, r);
+            if (CACHE.size() > 1) {
+                logger.warn("More than one singleton exist");
+            }
+
+        }
+        return r;
+    }
+
+    private ExecutableDao(KylinConfig config) {
+        logger.info("Using metadata url: " + config);
+        this.store = MetadataManager.getInstance(config).getStore();
+    }
+
+    private String pathOfJob(ExecutablePO job) {
+        return pathOfJob(job.getUuid());
+    }
+    private String pathOfJob(String uuid) {
+        return JOB_PATH_ROOT + "/" + uuid;
+    }
+
+    private String pathOfJobOutput(String uuid) {
+        return JOB_OUTPUT_ROOT + "/" + uuid;
+    }
+
+    private ExecutablePO readJobResource(String path) throws IOException {
+        return store.getResource(path, ExecutablePO.class, JOB_SERIALIZER);
+    }
+
+    private void writeJobResource(String path, ExecutablePO job) throws IOException {
+        store.putResource(path, job, JOB_SERIALIZER);
+    }
+
+    private ExecutableOutputPO readJobOutputResource(String path) throws IOException {
+        return store.getResource(path, ExecutableOutputPO.class, JOB_OUTPUT_SERIALIZER);
+    }
+
+    private long writeJobOutputResource(String path, ExecutableOutputPO output) throws IOException {
+        return store.putResource(path, output, JOB_OUTPUT_SERIALIZER);
+    }
+
+    public List<ExecutableOutputPO> getJobOutputs() throws PersistentException {
+        try {
+            ArrayList<String> resources = store.listResources(JOB_OUTPUT_ROOT);
+            if (resources == null || resources.isEmpty()) {
+                return Collections.emptyList();
+            }
+            Collections.sort(resources);
+            String rangeStart = resources.get(0);
+            String rangeEnd = resources.get(resources.size() - 1);
+            return store.getAllResources(rangeStart, rangeEnd, ExecutableOutputPO.class, JOB_OUTPUT_SERIALIZER);
+        } catch (IOException e) {
+            logger.error("error get all Jobs:", e);
+            throw new PersistentException(e);
+        }
+    }
+
+    public List<ExecutablePO> getJobs() throws PersistentException {
+        try {
+            final List<String> jobIds = store.listResources(JOB_PATH_ROOT);
+            if (jobIds == null || jobIds.isEmpty()) {
+                return Collections.emptyList();
+            }
+            Collections.sort(jobIds);
+            String rangeStart = jobIds.get(0);
+            String rangeEnd = jobIds.get(jobIds.size() - 1);
+            return store.getAllResources(rangeStart, rangeEnd, ExecutablePO.class, JOB_SERIALIZER);
+        } catch (IOException e) {
+            logger.error("error get all Jobs:", e);
+            throw new PersistentException(e);
+        }
+    }
+
+    public List<String> getJobIds() throws PersistentException {
+        try {
+            ArrayList<String> resources = store.listResources(JOB_PATH_ROOT);
+            if (resources == null) {
+                return Collections.emptyList();
+            }
+            ArrayList<String> result = Lists.newArrayListWithExpectedSize(resources.size());
+            for (String path : resources) {
+                result.add(path.substring(path.lastIndexOf("/") + 1));
+            }
+            return result;
+        } catch (IOException e) {
+            logger.error("error get all Jobs:", e);
+            throw new PersistentException(e);
+        }
+    }
+
+    public ExecutablePO getJob(String uuid) throws PersistentException {
+        try {
+            return readJobResource(pathOfJob(uuid));
+        } catch (IOException e) {
+            logger.error("error get job:" + uuid, e);
+            throw new PersistentException(e);
+        }
+    }
+
+    public ExecutablePO addJob(ExecutablePO job) throws PersistentException {
+        try {
+            if (getJob(job.getUuid()) != null) {
+                throw new IllegalArgumentException("job id:" + job.getUuid() + " already exists");
+            }
+            writeJobResource(pathOfJob(job), job);
+            return job;
+        } catch (IOException e) {
+            logger.error("error save job:" + job.getUuid(), e);
+            throw new PersistentException(e);
+        }
+    }
+
+    public void deleteJob(String uuid) throws PersistentException {
+        try {
+            store.deleteResource(pathOfJob(uuid));
+        } catch (IOException e) {
+            logger.error("error delete job:" + uuid, e);
+            throw new PersistentException(e);
+        }
+    }
+
+    public ExecutableOutputPO getJobOutput(String uuid) throws PersistentException {
+        try {
+            ExecutableOutputPO result = readJobOutputResource(pathOfJobOutput(uuid));
+            if (result == null) {
+                result = new ExecutableOutputPO();
+                result.setUuid(uuid);
+                return result;
+            }
+            return result;
+        } catch (IOException e) {
+            logger.error("error get job output id:" + uuid, e);
+            throw new PersistentException(e);
+        }
+    }
+
+    public void addJobOutput(ExecutableOutputPO output) throws PersistentException {
+        try {
+            output.setLastModified(0);
+            writeJobOutputResource(pathOfJobOutput(output.getUuid()), output);
+        } catch (IOException e) {
+            logger.error("error update job output id:" + output.getUuid(), e);
+            throw new PersistentException(e);
+        }
+    }
+
+    public void updateJobOutput(ExecutableOutputPO output) throws PersistentException {
+        try {
+            final long ts = writeJobOutputResource(pathOfJobOutput(output.getUuid()), output);
+            output.setLastModified(ts);
+        } catch (IOException e) {
+            logger.error("error update job output id:" + output.getUuid(), e);
+            throw new PersistentException(e);
+        }
+    }
+
+    public void deleteJobOutput(String uuid) throws PersistentException {
+        try {
+            store.deleteResource(pathOfJobOutput(uuid));
+        } catch (IOException e) {
+            logger.error("error delete job:" + uuid, e);
+            throw new PersistentException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableOutputPO.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableOutputPO.java b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableOutputPO.java
new file mode 100644
index 0000000..4dacd8a
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableOutputPO.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.dao;
+
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Maps;
+import org.apache.kylin.common.persistence.RootPersistentEntity;
+
+/**
+ */
+@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE)
+public class ExecutableOutputPO extends RootPersistentEntity {
+
+    @JsonProperty("content")
+    private String content;
+
+    @JsonProperty("status")
+    private String status = "READY";
+
+    @JsonProperty("info")
+    private Map<String, String> info = Maps.newHashMap();
+
+    public String getContent() {
+        return content;
+    }
+
+    public void setContent(String content) {
+        this.content = content;
+    }
+
+    public String getStatus() {
+        return status;
+    }
+
+    public void setStatus(String status) {
+        this.status = status;
+    }
+
+    public Map<String, String> getInfo() {
+        return info;
+    }
+
+    public void setInfo(Map<String, String> info) {
+        this.info = info;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
new file mode 100644
index 0000000..6a17b29
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.dao;
+
+import java.util.List;
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Maps;
+import org.apache.kylin.common.persistence.RootPersistentEntity;
+
+/**
+ */
+@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE)
+public class ExecutablePO extends RootPersistentEntity {
+
+    @JsonProperty("name")
+    private String name;
+
+    @JsonProperty("tasks")
+    private List<ExecutablePO> tasks;
+
+    @JsonProperty("type")
+    private String type;
+
+    @JsonProperty("params")
+    private Map<String, String> params = Maps.newHashMap();
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public List<ExecutablePO> getTasks() {
+        return tasks;
+    }
+
+    public void setTasks(List<ExecutablePO> tasks) {
+        this.tasks = tasks;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    public Map<String, String> getParams() {
+        return params;
+    }
+
+    public void setParams(Map<String, String> params) {
+        this.params = params;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java b/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
new file mode 100644
index 0000000..4d4d944
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.engine;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.job.common.OptionsHelper;
+import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author ysong1
+ */
+public class JobEngineConfig {
+    private static final Logger logger = LoggerFactory.getLogger(JobEngineConfig.class);
+    public static String HADOOP_JOB_CONF_FILENAME = "kylin_job_conf";
+    public static String HIVE_CONF_FILENAME = "kylin_hive_conf";
+
+    private static File getJobConfig(String fileName) {
+        String path = System.getProperty(KylinConfig.KYLIN_CONF);
+        if (StringUtils.isNotEmpty(path)) {
+            return new File(path, fileName);
+        }
+
+        path = KylinConfig.getKylinHome();
+        if (StringUtils.isNotEmpty(path)) {
+            return new File(path + File.separator + "conf", fileName);
+        }
+        return null;
+    }
+
+    private String getHadoopJobConfFilePath(RealizationCapacity capaticy, boolean appendSuffix) throws IOException {
+        String hadoopJobConfFile;
+        if (appendSuffix) {
+            hadoopJobConfFile = (HADOOP_JOB_CONF_FILENAME + "_" + capaticy.toString().toLowerCase() + ".xml");
+        } else {
+            hadoopJobConfFile = (HADOOP_JOB_CONF_FILENAME + ".xml");
+        }
+
+        File jobConfig = getJobConfig(hadoopJobConfFile);
+        if (jobConfig == null || !jobConfig.exists()) {
+            logger.warn("fail to locate " + hadoopJobConfFile + ", trying to locate " + HADOOP_JOB_CONF_FILENAME + ".xml");
+            jobConfig = getJobConfig(HADOOP_JOB_CONF_FILENAME + ".xml");
+            if (jobConfig == null || !jobConfig.exists()) {
+                logger.error("fail to locate " + HADOOP_JOB_CONF_FILENAME + ".xml");
+                throw new RuntimeException("fail to locate " + HADOOP_JOB_CONF_FILENAME + ".xml");
+            }
+        }
+        return OptionsHelper.convertToFileURL(jobConfig.getAbsolutePath());
+    }
+
+    public String getHadoopJobConfFilePath(RealizationCapacity capaticy) throws IOException {
+        String path = getHadoopJobConfFilePath(capaticy, true);
+        if (!StringUtils.isEmpty(path)) {
+            logger.info("Chosen job conf is : " + path);
+            return path;
+        } else {
+            path = getHadoopJobConfFilePath(capaticy, false);
+            if (!StringUtils.isEmpty(path)) {
+                logger.info("Chosen job conf is : " + path);
+                return path;
+            }
+        }
+        return "";
+    }
+
+
+    public String getHiveConfFilePath() throws IOException {
+        String hiveConfFile = (HIVE_CONF_FILENAME + ".xml");
+
+        File jobConfig = getJobConfig(hiveConfFile);
+        if (jobConfig == null || !jobConfig.exists()) {
+
+            logger.error("fail to locate " + HIVE_CONF_FILENAME + ".xml");
+            throw new RuntimeException("fail to locate " + HIVE_CONF_FILENAME + ".xml");
+        }
+        return OptionsHelper.convertToFileURL(jobConfig.getAbsolutePath());
+    }
+
+    // there should be no setters
+    private final KylinConfig config;
+
+    public JobEngineConfig(KylinConfig config) {
+        this.config = config;
+    }
+
+    public KylinConfig getConfig() {
+        return config;
+    }
+
+    public String getHdfsWorkingDirectory() {
+        return config.getHdfsWorkingDirectory();
+    }
+    
+    /**
+     * @return the maxConcurrentJobLimit
+     */
+    public int getMaxConcurrentJobLimit() {
+        return config.getMaxConcurrentJobLimit();
+    }
+
+    /**
+     * @return the timeZone
+     */
+    public String getTimeZone() {
+        return config.getTimeZone();
+    }
+
+    /**
+     * @return the adminDls
+     */
+    public String getAdminDls() {
+        return config.getAdminDls();
+    }
+
+    /**
+     * @return the jobStepTimeout
+     */
+    public long getJobStepTimeout() {
+        return config.getJobStepTimeout();
+    }
+
+    /**
+     * @return the asyncJobCheckInterval
+     */
+    public int getAsyncJobCheckInterval() {
+        return config.getYarnStatusCheckIntervalSeconds();
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see java.lang.Object#hashCode()
+     */
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((config == null) ? 0 : config.hashCode());
+        return result;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see java.lang.Object#equals(java.lang.Object)
+     */
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        JobEngineConfig other = (JobEngineConfig) obj;
+        if (config == null) {
+            if (other.config != null)
+                return false;
+        } else if (!config.equals(other.config))
+            return false;
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/exception/ExecuteException.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/exception/ExecuteException.java b/core-job/src/main/java/org/apache/kylin/job/exception/ExecuteException.java
new file mode 100644
index 0000000..8544fff
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/exception/ExecuteException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.exception;
+
+/**
+ */
+public class ExecuteException extends Exception {
+
+    private static final long serialVersionUID = 5677121412192984281L;
+
+    public ExecuteException() {
+    }
+
+    public ExecuteException(String message) {
+        super(message);
+    }
+
+    public ExecuteException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ExecuteException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/exception/IllegalStateTranferException.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/exception/IllegalStateTranferException.java b/core-job/src/main/java/org/apache/kylin/job/exception/IllegalStateTranferException.java
new file mode 100644
index 0000000..f19b0ca
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/exception/IllegalStateTranferException.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.exception;
+
+/**
+ */
+public class IllegalStateTranferException extends RuntimeException {
+
+    private static final long serialVersionUID = 8466551519300132702L;
+
+    public IllegalStateTranferException() {
+    }
+
+    public IllegalStateTranferException(String message) {
+        super(message);
+    }
+
+    public IllegalStateTranferException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public IllegalStateTranferException(Throwable cause) {
+        super(cause);
+    }
+
+    public IllegalStateTranferException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/exception/JobException.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/exception/JobException.java b/core-job/src/main/java/org/apache/kylin/job/exception/JobException.java
new file mode 100644
index 0000000..ba4c52a
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/exception/JobException.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.job.exception;
+
+/**
+ * @author xduo
+ * 
+ */
+public class JobException extends Exception {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * 
+     */
+    public JobException() {
+        super();
+    }
+
+    /**
+     * @param message
+     * @param cause
+     */
+    public JobException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    /**
+     * @param message
+     */
+    public JobException(String message) {
+        super(message);
+    }
+
+    /**
+     * @param cause
+     */
+    public JobException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/exception/LockException.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/exception/LockException.java b/core-job/src/main/java/org/apache/kylin/job/exception/LockException.java
new file mode 100644
index 0000000..cf43ac9
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/exception/LockException.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.exception;
+
+/**
+ */
+public class LockException extends Exception {
+    private static final long serialVersionUID = 2072745879281754945L;
+
+    public LockException() {
+    }
+
+    public LockException(String message) {
+        super(message);
+    }
+
+    public LockException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public LockException(Throwable cause) {
+        super(cause);
+    }
+
+    public LockException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/exception/PersistentException.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/exception/PersistentException.java b/core-job/src/main/java/org/apache/kylin/job/exception/PersistentException.java
new file mode 100644
index 0000000..8507a53
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/exception/PersistentException.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.exception;
+
+/**
+ */
+public class PersistentException extends Exception {
+    private static final long serialVersionUID = -4239863858506718998L;
+
+    public PersistentException() {
+    }
+
+    public PersistentException(String message) {
+        super(message);
+    }
+
+    public PersistentException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public PersistentException(Throwable cause) {
+        super(cause);
+    }
+
+    public PersistentException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/core-job/src/main/java/org/apache/kylin/job/exception/SchedulerException.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/exception/SchedulerException.java b/core-job/src/main/java/org/apache/kylin/job/exception/SchedulerException.java
new file mode 100644
index 0000000..057bd4a
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/exception/SchedulerException.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.exception;
+
+/**
+ */
+public class SchedulerException extends Exception {
+    private static final long serialVersionUID = 349041244824274861L;
+
+    public SchedulerException() {
+    }
+
+    public SchedulerException(String message) {
+        super(message);
+    }
+
+    public SchedulerException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public SchedulerException(Throwable cause) {
+        super(cause);
+    }
+
+    public SchedulerException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}


[12/28] incubator-kylin git commit: KYLIN-875 Split job module into 'core-job', 'engine-mr', 'source-hive', 'storage-hbase'. The old job remains as an assembly project.

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
index 48553d7..6f8959b 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
@@ -37,9 +37,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
-
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJob.java
deleted file mode 100644
index 06046c5..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJob.java
+++ /dev/null
@@ -1,39 +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.cube;
-
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * @author honma
- * 
- */
-
-public class BaseCuboidJob extends CuboidJob {
-    public BaseCuboidJob() {
-        this.setMapperClass(HiveToBaseCuboidMapper.class);
-    }
-
-    public static void main(String[] args) throws Exception {
-        CuboidJob job = new BaseCuboidJob();
-        int exitCode = ToolRunner.run(job, args);
-        System.exit(exitCode);
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidMapperBase.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidMapperBase.java
deleted file mode 100644
index dc4c746..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/BaseCuboidMapperBase.java
+++ /dev/null
@@ -1,205 +0,0 @@
-package org.apache.kylin.job.hadoop.cube;
-
-import com.google.common.collect.Lists;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.hadoop.io.Text;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.BytesSplitter;
-import org.apache.kylin.common.util.SplittedBytes;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-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.SegmentStatusEnum;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-
-/**
- */
-public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VALUEIN, Text, Text> {
-    protected static final Logger logger = LoggerFactory.getLogger(HiveToBaseCuboidMapper.class);
-    public static final byte[] HIVE_NULL = Bytes.toBytes("\\N");
-    public static final byte[] ONE = Bytes.toBytes("1");
-    protected String cubeName;
-    protected String segmentName;
-    protected Cuboid baseCuboid;
-    protected CubeInstance cube;
-    protected CubeDesc cubeDesc;
-    protected CubeSegment cubeSegment;
-    protected List<byte[]> nullBytes;
-    protected CubeJoinedFlatTableDesc intermediateTableDesc;
-    protected String intermediateTableRowDelimiter;
-    protected byte byteRowDelimiter;
-    protected int counter;
-    protected Object[] measures;
-    protected byte[][] keyBytesBuf;
-    protected BytesSplitter bytesSplitter;
-    protected AbstractRowKeyEncoder rowKeyEncoder;
-    protected MeasureCodec measureCodec;
-    private int errorRecordCounter;
-    private Text outputKey = new Text();
-    private Text outputValue = new Text();
-    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
-        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME);
-        intermediateTableRowDelimiter = context.getConfiguration().get(BatchConstants.CFG_CUBE_INTERMEDIATE_TABLE_ROW_DELIMITER, Character.toString(BatchConstants.INTERMEDIATE_TABLE_ROW_DELIMITER));
-        if (Bytes.toBytes(intermediateTableRowDelimiter).length > 1) {
-            throw new RuntimeException("Expected delimiter byte length is 1, but got " + Bytes.toBytes(intermediateTableRowDelimiter).length);
-        }
-
-        byteRowDelimiter = Bytes.toBytes(intermediateTableRowDelimiter)[0];
-
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-
-        cube = CubeManager.getInstance(config).getCube(cubeName);
-        cubeDesc = cube.getDescriptor();
-        cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
-
-        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
-
-        intermediateTableDesc = new CubeJoinedFlatTableDesc(cube.getDescriptor(), cubeSegment);
-
-        bytesSplitter = new BytesSplitter(200, 4096);
-        rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid);
-
-        measureCodec = new MeasureCodec(cubeDesc.getMeasures());
-        measures = new Object[cubeDesc.getMeasures().size()];
-
-        int colCount = cubeDesc.getRowkey().getRowKeyColumns().length;
-        keyBytesBuf = new byte[colCount][];
-
-        initNullBytes();
-    }
-
-    private void initNullBytes() {
-        nullBytes = Lists.newArrayList();
-        nullBytes.add(HIVE_NULL);
-        String[] nullStrings = cubeDesc.getNullStrings();
-        if (nullStrings != null) {
-            for (String s : nullStrings) {
-                nullBytes.add(Bytes.toBytes(s));
-            }
-        }
-    }
-
-    private boolean isNull(byte[] v) {
-        for (byte[] nullByte : nullBytes) {
-            if (Bytes.equals(v, nullByte))
-                return true;
-        }
-        return false;
-    }
-
-    private byte[] buildKey(SplittedBytes[] splitBuffers) {
-        int[] rowKeyColumnIndexes = intermediateTableDesc.getRowKeyColumnIndexes();
-        for (int i = 0; i < baseCuboid.getColumns().size(); i++) {
-            int index = rowKeyColumnIndexes[i];
-            keyBytesBuf[i] = Arrays.copyOf(splitBuffers[index].value, splitBuffers[index].length);
-            if (isNull(keyBytesBuf[i])) {
-                keyBytesBuf[i] = null;
-            }
-        }
-        return rowKeyEncoder.encode(keyBytesBuf);
-    }
-
-    private void buildValue(SplittedBytes[] splitBuffers) {
-
-        for (int i = 0; i < measures.length; i++) {
-            byte[] valueBytes = getValueBytes(splitBuffers, i);
-            measures[i] = measureCodec.getSerializer(i).valueOf(valueBytes);
-        }
-
-        valueBuf.clear();
-        measureCodec.encode(measures, valueBuf);
-    }
-
-    private byte[] getValueBytes(SplittedBytes[] splitBuffers, int measureIdx) {
-        MeasureDesc desc = cubeDesc.getMeasures().get(measureIdx);
-        FunctionDesc func = desc.getFunction();
-        ParameterDesc paramDesc = func.getParameter();
-        int[] flatTableIdx = intermediateTableDesc.getMeasureColumnIndexes()[measureIdx];
-
-        byte[] result = null;
-
-        // constant
-        if (flatTableIdx == null) {
-            result = Bytes.toBytes(paramDesc.getValue());
-        }
-        // column values
-        else {
-            // for multiple columns, their values are joined
-            for (int i = 0; i < flatTableIdx.length; i++) {
-                SplittedBytes split = splitBuffers[flatTableIdx[i]];
-                if (result == null) {
-                    result = Arrays.copyOf(split.value, split.length);
-                } else {
-                    byte[] newResult = new byte[result.length + split.length];
-                    System.arraycopy(result, 0, newResult, 0, result.length);
-                    System.arraycopy(split.value, 0, newResult, result.length, split.length);
-                    result = newResult;
-                }
-            }
-        }
-
-        if (func.isCount() || func.isHolisticCountDistinct()) {
-            // note for holistic count distinct, this value will be ignored
-            result = ONE;
-        }
-
-        if (isNull(result)) {
-            result = null;
-        }
-
-        return result;
-    }
-
-    protected void outputKV(Context context) throws IOException, InterruptedException {
-        intermediateTableDesc.sanityCheck(bytesSplitter);
-
-        byte[] rowKey = buildKey(bytesSplitter.getSplitBuffers());
-        outputKey.set(rowKey, 0, rowKey.length);
-
-        buildValue(bytesSplitter.getSplitBuffers());
-        outputValue.set(valueBuf.array(), 0, valueBuf.position());
-        context.write(outputKey, outputValue);
-    }
-
-    protected void handleErrorRecord(BytesSplitter bytesSplitter, Exception ex) throws IOException {
-
-        System.err.println("Insane record: " + bytesSplitter);
-        ex.printStackTrace(System.err);
-
-        errorRecordCounter++;
-        if (errorRecordCounter > BatchConstants.ERROR_RECORD_THRESHOLD) {
-            if (ex instanceof IOException)
-                throw (IOException) ex;
-            else if (ex instanceof RuntimeException)
-                throw (RuntimeException) ex;
-            else
-                throw new RuntimeException("", ex);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
deleted file mode 100644
index 918795e..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
-import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author George Song (ysong1)
- */
-public class CubeHFileJob extends AbstractHadoopJob {
-
-    protected static final Logger logger = LoggerFactory.getLogger(CubeHFileJob.class);
-
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_CUBE_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));
-            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
-
-            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
-
-            CubeInstance cube = cubeMgr.getCube(cubeName);
-            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(CubeHFileMapper.class);
-            job.setReducerClass(KeyValueSortReducer.class);
-
-            // set job configuration
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
-            Configuration conf = HBaseConfiguration.create(getConf());
-            // add metadata to distributed cache
-            attachKylinPropsAndMetadata(cube, job.getConfiguration());
-
-            String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
-            HTable htable = new HTable(conf, tableName);
-
-            //Automatic config !
-            HFileOutputFormat.configureIncrementalLoad(job, htable);
-
-            // set block replication to 3 for hfiles
-            conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, "3");
-
-            this.deletePath(job.getConfiguration(), output);
-
-            return waitForCompletion(job);
-        } catch (Exception e) {
-            logger.error("error in CubeHFileJob", e);
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        int exitCode = ToolRunner.run(new CubeHFileJob(), args);
-        System.exit(exitCode);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper.java
deleted file mode 100644
index f6e6182..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileMapper.java
+++ /dev/null
@@ -1,99 +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.cube;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-
-import com.google.common.collect.Lists;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-public class CubeHFileMapper extends KylinMapper<Text, Text, ImmutableBytesWritable, KeyValue> {
-
-    ImmutableBytesWritable outputKey = new ImmutableBytesWritable();
-
-    String cubeName;
-    CubeDesc cubeDesc;
-
-    MeasureCodec inputCodec;
-    Object[] inputMeasures;
-    List<KeyValueCreator> keyValueCreators;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME);
-
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-
-        CubeManager cubeMgr = CubeManager.getInstance(config);
-        cubeDesc = cubeMgr.getCube(cubeName).getDescriptor();
-
-        inputCodec = new MeasureCodec(cubeDesc.getMeasures());
-        inputMeasures = new Object[cubeDesc.getMeasures().size()];
-        keyValueCreators = Lists.newArrayList();
-
-        for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
-            for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
-                keyValueCreators.add(new KeyValueCreator(cubeDesc, colDesc));
-            }
-        }
-    }
-
-    @Override
-    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
-        outputKey.set(key.getBytes(), 0, key.getLength());
-        KeyValue outputValue;
-
-        int n = keyValueCreators.size();
-        if (n == 1 && keyValueCreators.get(0).isFullCopy) { // shortcut for
-                                                            // simple full copy
-
-            outputValue = keyValueCreators.get(0).create(key, value.getBytes(), 0, value.getLength());
-            context.write(outputKey, outputValue);
-
-        } else { // normal (complex) case that distributes measures to multiple HBase columns
-
-            inputCodec.decode(ByteBuffer.wrap(value.getBytes(), 0, value.getLength()), inputMeasures);
-
-            for (int i = 0; i < n; i++) {
-                outputValue = keyValueCreators.get(i).create(key, inputMeasures);
-                context.write(outputKey, outputValue);
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidJob.java
deleted file mode 100644
index 46bb9fc..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidJob.java
+++ /dev/null
@@ -1,200 +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.cube;
-
-import org.apache.commons.cli.Options;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-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.cube.cuboid.CuboidCLI;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.exception.JobException;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * @author ysong1
- */
-public class CuboidJob extends AbstractHadoopJob {
-
-    protected static final Logger logger = LoggerFactory.getLogger(CuboidJob.class);
-    private static final String MAPRED_REDUCE_TASKS = "mapred.reduce.tasks";
-
-    @SuppressWarnings("rawtypes")
-    private Class<? extends Mapper> mapperClass;
-
-    @Override
-    public int run(String[] args) throws Exception {
-        if (this.mapperClass == null)
-            throw new Exception("Mapper class is not set!");
-        
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_CUBE_NAME);
-            options.addOption(OPTION_SEGMENT_NAME);
-            options.addOption(OPTION_INPUT_PATH);
-            options.addOption(OPTION_OUTPUT_PATH);
-            options.addOption(OPTION_NCUBOID_LEVEL);
-            options.addOption(OPTION_INPUT_FORMAT);
-            parseOptions(options, args);
-
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
-            int nCuboidLevel = Integer.parseInt(getOptionValue(OPTION_NCUBOID_LEVEL));
-            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
-
-            KylinConfig config = KylinConfig.getInstanceFromEnv();
-            CubeManager cubeMgr = CubeManager.getInstance(config);
-            CubeInstance cube = cubeMgr.getCube(cubeName);
-
-            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
-            logger.info("Starting: " + job.getJobName());
-
-            setJobClasspath(job);
-
-            // Mapper
-            configureMapperInputFormat(cube.getSegment(segmentName, SegmentStatusEnum.NEW));
-            job.setMapperClass(this.mapperClass);
-            job.setMapOutputKeyClass(Text.class);
-            job.setMapOutputValueClass(Text.class);
-            job.setCombinerClass(CuboidReducer.class); // for base cuboid shuffle skew, some rowkey aggregates far more records than others
-
-            // Reducer
-            job.setReducerClass(CuboidReducer.class);
-            job.setOutputFormatClass(SequenceFileOutputFormat.class);
-            job.setOutputKeyClass(Text.class);
-            job.setOutputValueClass(Text.class);
-
-            FileOutputFormat.setOutputPath(job, output);
-
-            // set job configuration
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
-            // add metadata to distributed cache
-            attachKylinPropsAndMetadata(cube, job.getConfiguration());
-
-            setReduceTaskNum(job, config, cubeName, nCuboidLevel);
-
-            this.deletePath(job.getConfiguration(), output);
-
-            return waitForCompletion(job);
-        } catch (Exception e) {
-            logger.error("error in CuboidJob", e);
-            printUsage(options);
-            throw e;
-        }
-    }
-
-    private void configureMapperInputFormat(CubeSegment cubeSeg) throws IOException {
-        String input = getOptionValue(OPTION_INPUT_PATH);
-        
-        if (StringUtils.isBlank(input)) {
-            // base cuboid case
-            IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSeg).getFlatTableInputFormat();
-            flatTableInputFormat.configureJob(job);
-        }
-        else {
-            // n-dimension cuboid case
-            FileInputFormat.setInputPaths(job, new Path(input));
-            if (hasOption(OPTION_INPUT_FORMAT) && ("textinputformat".equalsIgnoreCase(getOptionValue(OPTION_INPUT_FORMAT)))) {
-                job.setInputFormatClass(TextInputFormat.class);
-            } else {
-                job.setInputFormatClass(SequenceFileInputFormat.class);
-            }
-        }
-    }
-
-    protected void setReduceTaskNum(Job job, KylinConfig config, String cubeName, int level) throws ClassNotFoundException, IOException, InterruptedException, JobException {
-        Configuration jobConf = job.getConfiguration();
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-
-        CubeDesc cubeDesc = CubeManager.getInstance(config).getCube(cubeName).getDescriptor();
-
-        double perReduceInputMB = kylinConfig.getDefaultHadoopJobReducerInputMB();
-        double reduceCountRatio = kylinConfig.getDefaultHadoopJobReducerCountRatio();
-
-        // total map input MB
-        double totalMapInputMB = this.getTotalMapInputMB();
-
-        // output / input ratio
-        int preLevelCuboids, thisLevelCuboids;
-        if (level == 0) { // base cuboid
-            preLevelCuboids = thisLevelCuboids = 1;
-        } else { // n-cuboid
-            int[] allLevelCount = CuboidCLI.calculateAllLevelCount(cubeDesc);
-            preLevelCuboids = allLevelCount[level - 1];
-            thisLevelCuboids = allLevelCount[level];
-        }
-
-        // total reduce input MB
-        double totalReduceInputMB = totalMapInputMB * thisLevelCuboids / preLevelCuboids;
-
-        // number of reduce tasks
-        int numReduceTasks = (int) Math.round(totalReduceInputMB / perReduceInputMB * reduceCountRatio);
-
-        // adjust reducer number for cube which has DISTINCT_COUNT measures for
-        // better performance
-        if (cubeDesc.hasHolisticCountDistinctMeasures()) {
-            numReduceTasks = numReduceTasks * 4;
-        }
-
-        // at least 1 reducer
-        numReduceTasks = Math.max(1, numReduceTasks);
-        // no more than 5000 reducer by default
-        numReduceTasks = Math.min(kylinConfig.getHadoopJobMaxReducerNumber(), numReduceTasks);
-
-        jobConf.setInt(MAPRED_REDUCE_TASKS, numReduceTasks);
-
-        logger.info("Having total map input MB " + Math.round(totalMapInputMB));
-        logger.info("Having level " + level + ", pre-level cuboids " + preLevelCuboids + ", this level cuboids " + thisLevelCuboids);
-        logger.info("Having per reduce MB " + perReduceInputMB + ", reduce count ratio " + reduceCountRatio);
-        logger.info("Setting " + MAPRED_REDUCE_TASKS + "=" + numReduceTasks);
-    }
-
-    /**
-     * @param mapperClass
-     *            the mapperClass to set
-     */
-    @SuppressWarnings("rawtypes")
-    public void setMapperClass(Class<? extends Mapper> mapperClass) {
-        this.mapperClass = mapperClass;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidReducer.java
deleted file mode 100644
index 7f41b8d..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CuboidReducer.java
+++ /dev/null
@@ -1,101 +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.cube;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.metadata.measure.MeasureAggregators;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.engine.mr.KylinReducer;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-
-/**
- * @author George Song (ysong1)
- * 
- */
-public class CuboidReducer extends KylinReducer<Text, Text, Text, Text> {
-
-    private static final Logger logger = LoggerFactory.getLogger(CuboidReducer.class);
-
-    private String cubeName;
-    private CubeDesc cubeDesc;
-    private List<MeasureDesc> measuresDescs;
-
-    private MeasureCodec codec;
-    private MeasureAggregators aggs;
-
-    private int counter;
-    private Object[] input;
-    private Object[] result;
-
-    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-    private Text outputValue = new Text();
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
-
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-
-        cubeDesc = CubeManager.getInstance(config).getCube(cubeName).getDescriptor();
-        measuresDescs = cubeDesc.getMeasures();
-
-        codec = new MeasureCodec(measuresDescs);
-        aggs = new MeasureAggregators(measuresDescs);
-
-        input = new Object[measuresDescs.size()];
-        result = new Object[measuresDescs.size()];
-    }
-
-    @Override
-    public void reduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
-
-        aggs.reset();
-
-        for (Text value : values) {
-            codec.decode(ByteBuffer.wrap(value.getBytes(), 0, value.getLength()), input);
-            aggs.aggregate(input);
-        }
-        aggs.collectStates(result);
-
-        valueBuf.clear();
-        codec.encode(result, valueBuf);
-
-        outputValue.set(valueBuf.array(), 0, valueBuf.position());
-        context.write(key, outputValue);
-
-        counter++;
-        if (counter % BatchConstants.COUNTER_MAX == 0) {
-            logger.info("Handled " + counter + " records!");
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsCombiner.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsCombiner.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsCombiner.java
deleted file mode 100644
index 12c8713..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsCombiner.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import java.io.IOException;
-import java.util.HashSet;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.hadoop.io.Text;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.engine.mr.KylinReducer;
-
-/**
- * @author yangli9
- */
-public class FactDistinctColumnsCombiner extends KylinReducer<LongWritable, Text, LongWritable, Text> {
-
-    private Text outputValue = new Text();
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-    }
-
-    @Override
-    public void reduce(LongWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
-
-        if(key.get() >= 0) {
-            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);
-            }
-        } else {
-            // for hll, each key only has one output, no need to do local combine;
-            outputValue.set(values.iterator().next().getBytes());
-            context.write(key, outputValue);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsJob.java
deleted file mode 100644
index 1697339..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsJob.java
+++ /dev/null
@@ -1,132 +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.cube;
-
-import java.io.IOException;
-
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-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.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.IMRInput.IMRTableInputFormat;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-public class FactDistinctColumnsJob extends AbstractHadoopJob {
-    protected static final Logger log = LoggerFactory.getLogger(FactDistinctColumnsJob.class);
-
-    @Override
-    public int run(String[] args) throws Exception {
-        Options options = new Options();
-
-        try {
-            options.addOption(OPTION_JOB_NAME);
-            options.addOption(OPTION_CUBE_NAME);
-            options.addOption(OPTION_OUTPUT_PATH);
-            options.addOption(OPTION_SEGMENT_NAME);
-            options.addOption(OPTION_STATISTICS_ENABLED);
-            options.addOption(OPTION_STATISTICS_OUTPUT);
-            options.addOption(OPTION_STATISTICS_SAMPLING_PERCENT);
-            parseOptions(options, args);
-
-            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
-            String cubeName = getOptionValue(OPTION_CUBE_NAME);
-            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-            
-            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
-            String statistics_enabled = getOptionValue(OPTION_STATISTICS_ENABLED);
-            String statistics_output = getOptionValue(OPTION_STATISTICS_OUTPUT);
-            String statistics_sampling_percent = getOptionValue(OPTION_STATISTICS_SAMPLING_PERCENT);
-
-            // ----------------------------------------------------------------------------
-            // add metadata to distributed cache
-            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
-            CubeInstance cube = cubeMgr.getCube(cubeName);
-
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
-            job.getConfiguration().set(BatchConstants.CFG_STATISTICS_ENABLED, statistics_enabled);
-            job.getConfiguration().set(BatchConstants.CFG_STATISTICS_OUTPUT, statistics_output);
-            job.getConfiguration().set(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT, statistics_sampling_percent);
-            log.info("Starting: " + job.getJobName());
-
-            setJobClasspath(job);
-
-            setupMapper(cube.getSegment(segmentName, SegmentStatusEnum.NEW));
-            setupReducer(output);
-
-            // CubeSegment seg = cubeMgr.getCube(cubeName).getTheOnlySegment();
-            attachKylinPropsAndMetadata(cube, job.getConfiguration());
-
-            return waitForCompletion(job);
-
-        } catch (Exception e) {
-            logger.error("error in FactDistinctColumnsJob", e);
-            printUsage(options);
-            throw e;
-        }
-
-    }
-
-    private void setupMapper(CubeSegment cubeSeg) throws IOException {
-        IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSeg).getFlatTableInputFormat();
-        flatTableInputFormat.configureJob(job);
-
-        job.setMapperClass(FactDistinctHiveColumnsMapper.class);
-        job.setCombinerClass(FactDistinctColumnsCombiner.class);
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setMapOutputValueClass(Text.class);
-    }
-
-    private void setupReducer(Path output) throws IOException {
-        job.setReducerClass(FactDistinctColumnsReducer.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 {
-        FactDistinctColumnsJob job = new FactDistinctColumnsJob();
-        int exitCode = ToolRunner.run(job, args);
-        System.exit(exitCode);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsMapperBase.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsMapperBase.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsMapperBase.java
deleted file mode 100644
index d30cfc0..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsMapperBase.java
+++ /dev/null
@@ -1,89 +0,0 @@
-package org.apache.kylin.job.hadoop.cube;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.RowKeyDesc;
-import org.apache.kylin.dict.DictionaryManager;
-import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
-import org.apache.kylin.engine.mr.KylinMapper;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.kylin.metadata.model.TblColRef;
-
-/**
- */
-public class FactDistinctColumnsMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VALUEIN, LongWritable, Text> {
-
-    protected String cubeName;
-    protected CubeInstance cube;
-    protected CubeSegment cubeSeg;
-    protected CubeDesc cubeDesc;
-    protected long baseCuboidId;
-    protected List<TblColRef> columns;
-    protected ArrayList<Integer> factDictCols;
-    protected IMRTableInputFormat flatTableInputFormat;
-
-    protected LongWritable outputKey = new LongWritable();
-    protected Text outputValue = new Text();
-    protected int errorRecordCounter = 0;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        Configuration conf = context.getConfiguration();
-        bindCurrentConfiguration(conf);
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-
-        cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
-        cube = CubeManager.getInstance(config).getCube(cubeName);
-        cubeSeg = cube.getSegment(conf.get(BatchConstants.CFG_CUBE_SEGMENT_NAME), SegmentStatusEnum.NEW);
-        cubeDesc = cube.getDescriptor();
-        baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        columns = Cuboid.findById(cubeDesc, baseCuboidId).getColumns();
-
-        factDictCols = new ArrayList<Integer>();
-        RowKeyDesc rowKey = cubeDesc.getRowkey();
-        DictionaryManager dictMgr = DictionaryManager.getInstance(config);
-        for (int i = 0; i < columns.size(); i++) {
-            TblColRef col = columns.get(i);
-            if (!rowKey.isUseDictionary(col))
-                continue;
-
-            String scanTable = (String) dictMgr.decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0];
-            if (cubeDesc.getModel().isFactTable(scanTable)) {
-                factDictCols.add(i);
-            }
-        }
-        
-        flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSeg).getFlatTableInputFormat();
-    }
-
-    protected void handleErrorRecord(String[] record, Exception ex) throws IOException {
-
-        System.err.println("Insane record: " + Arrays.toString(record));
-        ex.printStackTrace(System.err);
-
-        errorRecordCounter++;
-        if (errorRecordCounter > BatchConstants.ERROR_RECORD_THRESHOLD) {
-            if (ex instanceof IOException)
-                throw (IOException) ex;
-            else if (ex instanceof RuntimeException)
-                throw (RuntimeException) ex;
-            else
-                throw new RuntimeException("", ex);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsReducer.java
deleted file mode 100644
index 4b73739..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctColumnsReducer.java
+++ /dev/null
@@ -1,224 +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.cube;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.*;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.engine.mr.KylinReducer;
-import org.apache.kylin.job.constant.BatchConstants;
-import org.apache.kylin.job.hadoop.AbstractHadoopJob;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-/**
- * @author yangli9
- */
-public class FactDistinctColumnsReducer extends KylinReducer<LongWritable, Text, NullWritable, Text> {
-
-    private List<TblColRef> columnList = new ArrayList<TblColRef>();
-    private boolean collectStatistics = false;
-    private String statisticsOutput = null;
-    private List<Long> baseCuboidRowCountInMappers;
-    protected Map<Long, HyperLogLogPlusCounter> cuboidHLLMap = null;
-    protected long baseCuboidId;
-    protected CubeDesc cubeDesc;
-    private long totalRowsBeforeMerge = 0;
-    private int SAMPING_PERCENTAGE = 5;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
-
-        Configuration conf = context.getConfiguration();
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-        String cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
-        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
-        cubeDesc = cube.getDescriptor();
-
-        baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
-        columnList = baseCuboid.getColumns();
-        collectStatistics = Boolean.parseBoolean(conf.get(BatchConstants.CFG_STATISTICS_ENABLED));
-        statisticsOutput = conf.get(BatchConstants.CFG_STATISTICS_OUTPUT);
-
-        if (collectStatistics) {
-            baseCuboidRowCountInMappers = Lists.newArrayList();
-            cuboidHLLMap = Maps.newHashMap();
-            SAMPING_PERCENTAGE = Integer.parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT, "5"));
-        }
-    }
-
-    @Override
-    public void reduce(LongWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
-
-        if (key.get() >= 0) {
-            TblColRef col = columnList.get((int) 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, col.getName()));
-
-            try {
-                for (ByteArray value : set) {
-                    out.write(value.array(), value.offset(), value.length());
-                    out.write('\n');
-                }
-            } finally {
-                out.close();
-            }
-        } else {
-            // for hll
-            long cuboidId = 0 - key.get();
-
-            for (Text value : values) {
-                HyperLogLogPlusCounter hll = new HyperLogLogPlusCounter(14);
-                ByteArray byteArray = new ByteArray(value.getBytes());
-                hll.readRegisters(byteArray.asBuffer());
-
-                totalRowsBeforeMerge += hll.getCountEstimate();
-
-                if (cuboidId == baseCuboidId) {
-                    baseCuboidRowCountInMappers.add(hll.getCountEstimate());
-                }
-
-                if (cuboidHLLMap.get(cuboidId) != null) {
-                    cuboidHLLMap.get(cuboidId).merge(hll);
-                } else {
-                    cuboidHLLMap.put(cuboidId, hll);
-                }
-            }
-        }
-
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-
-        //output the hll info;
-        if (collectStatistics) {
-            writeMapperAndCuboidStatistics(context); // for human check
-            writeCuboidStatistics(context.getConfiguration(), new Path(statisticsOutput), cuboidHLLMap, SAMPING_PERCENTAGE); // for CreateHTableJob
-        }
-    }
-
-    private void writeMapperAndCuboidStatistics(Context context) throws IOException {
-        Configuration conf = context.getConfiguration();
-        FileSystem fs = FileSystem.get(conf);
-        FSDataOutputStream out = fs.create(new Path(statisticsOutput, BatchConstants.CFG_STATISTICS_CUBE_ESTIMATION));
-
-        try {
-            String msg;
-
-            List<Long> allCuboids = new ArrayList<Long>();
-            allCuboids.addAll(cuboidHLLMap.keySet());
-            Collections.sort(allCuboids);
-
-            msg = "Total cuboid number: \t" + allCuboids.size();
-            writeLine(out, msg);
-            msg = "Samping percentage: \t" + SAMPING_PERCENTAGE;
-            writeLine(out, msg);
-
-            writeLine(out, "The following statistics are collected based sampling data.");
-            for (int i = 0; i < baseCuboidRowCountInMappers.size(); i++) {
-                if (baseCuboidRowCountInMappers.get(i) > 0) {
-                    msg = "Base Cuboid in Mapper " + i + " row count: \t " + baseCuboidRowCountInMappers.get(i);
-                    writeLine(out, msg);
-                }
-            }
-
-            long grantTotal = 0;
-            for (long i : allCuboids) {
-                grantTotal += cuboidHLLMap.get(i).getCountEstimate();
-                msg = "Cuboid " + i + " row count is: \t " + cuboidHLLMap.get(i).getCountEstimate();
-                writeLine(out, msg);
-            }
-
-            msg = "Sum of all the cube segments (before merge) is: \t " + totalRowsBeforeMerge;
-            writeLine(out, msg);
-
-            msg = "After merge, the cube has row count: \t " + grantTotal;
-            writeLine(out, msg);
-
-            if (grantTotal > 0) {
-                msg = "The compaction factor is: \t" + totalRowsBeforeMerge / grantTotal;
-                writeLine(out, msg);
-            }
-
-        } finally {
-            out.close();
-        }
-    }
-
-    private void writeLine(FSDataOutputStream out, String msg) throws IOException {
-        out.write(msg.getBytes());
-        out.write('\n');
-
-    }
-
-    public static void writeCuboidStatistics(Configuration conf, Path outputPath, Map<Long, HyperLogLogPlusCounter> cuboidHLLMap, int samplingPercentage) throws IOException {
-        Path seqFilePath = new Path(outputPath, BatchConstants.CFG_STATISTICS_CUBOID_ESTIMATION);
-        SequenceFile.Writer writer = SequenceFile.createWriter(conf,
-                SequenceFile.Writer.file(seqFilePath), SequenceFile.Writer.keyClass(LongWritable.class),
-                SequenceFile.Writer.valueClass(BytesWritable.class));
-
-        List<Long> allCuboids = new ArrayList<Long>();
-        allCuboids.addAll(cuboidHLLMap.keySet());
-        Collections.sort(allCuboids);
-
-        // persist the sample percentage with key 0
-        writer.append(new LongWritable(0l), new BytesWritable(Bytes.toBytes(samplingPercentage)));
-        ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-        try {
-            for (long i : allCuboids) {
-                valueBuf.clear();
-                cuboidHLLMap.get(i).writeRegisters(valueBuf);
-                valueBuf.flip();
-                writer.append(new LongWritable(i), new BytesWritable(valueBuf.array(), valueBuf.limit()));
-            }
-        } finally {
-            writer.close();
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctHiveColumnsMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctHiveColumnsMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctHiveColumnsMapper.java
deleted file mode 100644
index c19af69..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/FactDistinctHiveColumnsMapper.java
+++ /dev/null
@@ -1,176 +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.cube;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.BitSet;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.cube.cuboid.CuboidScheduler;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.job.constant.BatchConstants;
-
-import com.google.common.collect.Lists;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hasher;
-import com.google.common.hash.Hashing;
-
-/**
- * @author yangli9
- */
-public class FactDistinctHiveColumnsMapper<KEYIN> extends FactDistinctColumnsMapperBase<KEYIN, Object> {
-
-    private CubeJoinedFlatTableDesc intermediateTableDesc;
-
-    protected boolean collectStatistics = false;
-    protected CuboidScheduler cuboidScheduler = null;
-    protected int nRowKey;
-    private Integer[][] allCuboidsBitSet = null;
-    private HyperLogLogPlusCounter[] allCuboidsHLL = null;
-    private Long[] cuboidIds;
-    private HashFunction hf = null;
-    private int rowCount = 0;
-    private int SAMPING_PERCENTAGE = 5;
-    private ByteArray[] row_hashcodes = null;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.setup(context);
-        intermediateTableDesc = new CubeJoinedFlatTableDesc(cubeDesc, null);
-        collectStatistics = Boolean.parseBoolean(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_ENABLED));
-        if (collectStatistics) {
-            SAMPING_PERCENTAGE = Integer.parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT, "5"));
-            cuboidScheduler = new CuboidScheduler(cubeDesc);
-            nRowKey = cubeDesc.getRowkey().getRowKeyColumns().length;
-
-            List<Long> cuboidIdList = Lists.newArrayList();
-            List<Integer[]> allCuboidsBitSetList = Lists.newArrayList();
-            addCuboidBitSet(baseCuboidId, allCuboidsBitSetList, cuboidIdList);
-
-            allCuboidsBitSet = allCuboidsBitSetList.toArray(new Integer[cuboidIdList.size()][]);
-            cuboidIds = cuboidIdList.toArray(new Long[cuboidIdList.size()]);
-
-            allCuboidsHLL = new HyperLogLogPlusCounter[cuboidIds.length];
-            for (int i = 0; i < cuboidIds.length; i++) {
-                allCuboidsHLL[i] = new HyperLogLogPlusCounter(14);
-            }
-
-            hf = Hashing.murmur3_32();
-            row_hashcodes = new ByteArray[nRowKey];
-            for (int i = 0; i < nRowKey; i++) {
-                row_hashcodes[i] = new ByteArray();
-            }
-        }
-    }
-
-    private void addCuboidBitSet(long cuboidId, List<Integer[]> allCuboidsBitSet, List<Long> allCuboids) {
-        allCuboids.add(cuboidId);
-        BitSet bitSet = BitSet.valueOf(new long[]{cuboidId});
-        Integer[] indice = new Integer[bitSet.cardinality()];
-
-        long mask = Long.highestOneBit(baseCuboidId);
-        int position = 0;
-        for (int i = 0; i < nRowKey; i++) {
-            if ((mask & cuboidId) > 0) {
-                indice[position] = i;
-                position++;
-            }
-            mask = mask >> 1;
-        }
-
-        allCuboidsBitSet.add(indice);
-        Collection<Long> children = cuboidScheduler.getSpanningCuboid(cuboidId);
-        for (Long childId : children) {
-            addCuboidBitSet(childId, allCuboidsBitSet, allCuboids);
-        }
-    }
-
-    @Override
-    public void map(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
-        String[] row = flatTableInputFormat.parseMapperInput(record);
-        try {
-            for (int i : factDictCols) {
-                outputKey.set((long) i);
-                String fieldValue = row[intermediateTableDesc.getRowKeyColumnIndexes()[i]];
-                if (fieldValue == null)
-                    continue;
-                byte[] bytes = Bytes.toBytes(fieldValue);
-                outputValue.set(bytes, 0, bytes.length);
-                context.write(outputKey, outputValue);
-            }
-        } catch (Exception ex) {
-            handleErrorRecord(row, ex);
-        }
-
-        if (collectStatistics && rowCount < SAMPING_PERCENTAGE) {
-            putRowKeyToHLL(row);
-        }
-
-        if (rowCount++ == 100)
-            rowCount = 0;
-    }
-
-    private void putRowKeyToHLL(String[] row) {
-
-        //generate hash for each row key column
-        for (int i = 0; i < nRowKey; i++) {
-            Hasher hc = hf.newHasher();
-            String colValue = row[intermediateTableDesc.getRowKeyColumnIndexes()[i]];
-            if (colValue != null) {
-                row_hashcodes[i].set(hc.putString(colValue).hash().asBytes());
-            } else {
-                row_hashcodes[i].set(hc.putInt(0).hash().asBytes());
-            }
-        }
-
-        // user the row key column hash to get a consolidated hash for each cuboid
-        for (int i = 0, n = allCuboidsBitSet.length; i < n; i++) {
-            Hasher hc = hf.newHasher();
-            for (int position = 0; position < allCuboidsBitSet[i].length; position++) {
-                hc.putBytes(row_hashcodes[allCuboidsBitSet[i][position]].array());
-            }
-
-            allCuboidsHLL[i].add(hc.hash().asBytes());
-        }
-    }
-
-    @Override
-    protected void cleanup(Context context) throws IOException, InterruptedException {
-        if (collectStatistics) {
-            ByteBuffer hllBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-            // output each cuboid's hll to reducer, key is 0 - cuboidId
-            HyperLogLogPlusCounter hll;
-            for (int i = 0; i < cuboidIds.length; i++) {
-                hll = allCuboidsHLL[i];
-                outputKey.set(0 - cuboidIds[i]);
-                hllBuf.clear();
-                hll.writeRegisters(hllBuf);
-                outputValue.set(hllBuf.array(), 0, hllBuf.position());
-                context.write(outputKey, outputValue);
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/HiveToBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/HiveToBaseCuboidMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/HiveToBaseCuboidMapper.java
deleted file mode 100644
index 9fa1159..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/HiveToBaseCuboidMapper.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job.hadoop.cube;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-
-import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
-import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.job.constant.BatchConstants;
-
-/**
- * @author George Song (ysong1)
- */
-public class HiveToBaseCuboidMapper<KEYIN> extends BaseCuboidMapperBase<KEYIN, Object> {
-    
-    private IMRTableInputFormat flatTableInputFormat;
-
-    @Override
-    protected void setup(Context context) throws IOException {
-        super.setup(context);
-        flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSegment).getFlatTableInputFormat();
-    }
-
-    @Override
-    public void map(KEYIN key, Object value, Context context) throws IOException, InterruptedException {
-        counter++;
-        if (counter % BatchConstants.COUNTER_MAX == 0) {
-            logger.info("Handled " + counter + " records!");
-        }
-        
-
-        try {
-            //put a record into the shared bytesSplitter
-            String[] row = flatTableInputFormat.parseMapperInput(value);
-            bytesSplitter.setBuffers(convertUTF8Bytes(row));
-            //take care of the data in bytesSplitter
-            outputKV(context);
-
-        } catch (Exception ex) {
-            handleErrorRecord(bytesSplitter, ex);
-        }
-    }
-
-    private byte[][] convertUTF8Bytes(String[] row) throws UnsupportedEncodingException {
-        byte[][] result = new byte[row.length][];
-        for (int i = 0; i < row.length; i++) {
-            result[i] = row[i].getBytes("UTF-8");
-        }
-        return result;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJob.java
deleted file mode 100644
index 39efd8f..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJob.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.cube;
-
-///*
-// * Copyright 2013-2014 eBay Software Foundation
-// *
-// * Licensed under the Apache License, Version 2.0 (the "License");
-// * you may not use this file except in compliance with the License.
-// * You may obtain a copy of the License at
-// *
-// *   http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT 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.index.cube;
-//
-//import org.apache.commons.cli.Options;
-//import org.apache.hadoop.fs.FileSystem;
-//import org.apache.hadoop.fs.Path;
-//import org.apache.hadoop.io.LongWritable;
-//import org.apache.hadoop.io.Text;
-//import org.apache.hadoop.mapreduce.Job;
-//import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-//import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-//import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-//import org.apache.hadoop.util.ToolRunner;
-//
-//import org.apache.kylin.cube.CubeInstance;
-//import org.apache.kylin.cube.CubeManager;
-//import org.apache.kylin.cube.cuboid.Cuboid;
-//import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
-//import org.apache.kylin.cube.kv.RowKeyEncoder;
-//import org.apache.kylin.index.AbstractHadoopJob;
-//import org.apache.kylin.metadata.model.cube.CubeDesc;
-//
-///**
-// * @author xjiang
-// *
-// */
-//
-//public class KeyDistributionJob extends AbstractHadoopJob {
-//
-//    public static final String JOB_TITLE = "Kylin Row Key Distribution Job";
-//    public static final String KEY_HEADER_LENGTH = "key_header_length";
-//    public static final String KEY_COLUMN_PERCENTAGE = "key_column_percentage";
-//    public static final String KEY_SPLIT_NUMBER = "key_split_number";
-//
-//    /* (non-Javadoc)
-//     * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
-//     */
-//    @Override
-//    public int run(String[] args) throws Exception {
-//        Options options = new Options();
-//
-//        try {
-//            options.addOption(OPTION_INPUT_PATH);
-//            options.addOption(OPTION_OUTPUT_PATH);
-//            options.addOption(OPTION_METADATA_URL);
-//            options.addOption(OPTION_CUBE_NAME);
-//            options.addOption(OPTION_KEY_COLUMN_PERCENTAGE);
-//            options.addOption(OPTION_KEY_SPLIT_NUMBER);
-//            parseOptions(options, args);
-//
-//            // start job
-//            String jobName = JOB_TITLE + getOptionsAsString();
-//            System.out.println("Starting: " + jobName);
-//            Job job = Job.getInstanceFromEnv(getConf(), jobName);
-//
-//            // set job configuration - basic 
-//            setJobClasspath(job);
-//            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
-//
-//            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
-//            FileOutputFormat.setOutputPath(job, output);
-//            //job.getConfiguration().set("dfs.block.size", "67108864");
-//
-//            // set job configuration - key prefix size & key split number
-//            String keyColumnPercentage = getOptionValue(OPTION_KEY_COLUMN_PERCENTAGE);
-//            job.getConfiguration().set(KEY_COLUMN_PERCENTAGE, keyColumnPercentage);
-//            String metadataUrl = validateMetadataUrl(getOptionValue(OPTION_METADATA_URL));
-//            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
-//            int keyHeaderLen = getKeyHeaderLength(metadataUrl, cubeName);
-//            job.getConfiguration().set(KEY_HEADER_LENGTH, String.valueOf(keyHeaderLen));
-//            job.getConfiguration().set(KEY_SPLIT_NUMBER, getOptionValue(OPTION_KEY_SPLIT_NUMBER));
-//
-//            // Mapper
-//            job.setInputFormatClass(SequenceFileInputFormat.class);
-//            job.setMapperClass(KeyDistributionMapper.class);
-//            job.setMapOutputKeyClass(Text.class);
-//            job.setMapOutputValueClass(LongWritable.class);
-//
-//            // Combiner, not needed any more as mapper now does the groping
-//            //job.setCombinerClass(KeyDistributionCombiner.class);
-//
-//            // Reducer - only one
-//            job.setReducerClass(KeyDistributionReducer.class);
-//            // use sequence file as output
-//            job.setOutputFormatClass(SequenceFileOutputFormat.class);
-//            // key is text
-//            job.setOutputKeyClass(Text.class);
-//            // value is long
-//            job.setOutputValueClass(LongWritable.class);
-//            job.setNumReduceTasks(1);
-//
-//            FileSystem fs = FileSystem.get(job.getConfiguration());
-//            if (fs.exists(output))
-//                fs.delete(output, true);
-//
-//            return waitForCompletion(job);
-//        } catch (Exception e) {
-//            printUsage(options);
-//            e.printStackTrace(System.err);
-//            return 2;
-//        }
-//    }
-//
-//    private int getKeyHeaderLength(String metadataUrl, String cubeName) {
-//        CubeManager cubeMgr = CubeManager.getInstanceFromEnv(metadataUrl);
-//        CubeInstance cubeInstance = cubeMgr.getCube(cubeName);
-//        CubeDesc cubeDesc = cubeInstance.getDescriptor();
-//        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-//        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
-//        RowKeyEncoder rowKeyEncoder =
-//                (RowKeyEncoder) AbstractRowKeyEncoder.createInstance(cubeInstance.getTheOnlySegment(),
-//                        baseCuboid);
-//
-//        return rowKeyEncoder.getHeaderLength();
-//
-//    }
-//
-//    public static void main(String[] args) throws Exception {
-//        int exitCode = ToolRunner.run(new KeyDistributionJob(), args);
-//        System.exit(exitCode);
-//    }
-// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionMapper.java
deleted file mode 100644
index c5d0e2b..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionMapper.java
+++ /dev/null
@@ -1,124 +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.cube;
-
-///*
-// * Copyright 2013-2014 eBay Software Foundation
-// *
-// * Licensed under the Apache License, Version 2.0 (the "License");
-// * you may not use this file except in compliance with the License.
-// * You may obtain a copy of the License at
-// *
-// *   http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT 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.index.cube;
-//
-//import java.io.IOException;
-//
-//import org.apache.hadoop.io.LongWritable;
-//import org.apache.hadoop.io.Text;
-//import org.apache.hadoop.mapreduce.Mapper;
-//
-///**
-// * @author xjiang
-// *
-// */
-//public class KeyDistributionMapper extends KylinMapper<Text, Text, Text, LongWritable> {
-//
-//    private int headerLength;
-//
-//    private Text currentKey;
-//    private long outputLong;
-//    private Text outputKey;
-//    private LongWritable outputValue;
-//    private int columnPercentage;
-//    private int allRowCount;
-//
-//    @Override
-//    protected void setup(Context context) throws IOException {
-//super.publishConfiguration(context.getConfiguration());
-
-//        String percentStr = context.getConfiguration().get(KeyDistributionJob.KEY_COLUMN_PERCENTAGE);
-//        this.columnPercentage = Integer.valueOf(percentStr).intValue();
-//        if (this.columnPercentage <= 0 || this.columnPercentage >= 100) {
-//            this.columnPercentage = 20;
-//        }
-//        String headerLenStr = context.getConfiguration().get(KeyDistributionJob.KEY_HEADER_LENGTH);
-//        this.headerLength = Integer.valueOf(headerLenStr).intValue();
-//
-//        currentKey = new Text();
-//        outputLong = 0;
-//        outputKey = new Text();
-//        outputValue = new LongWritable(1);
-//        allRowCount = 0;
-//    }
-//
-//    @Override
-//    protected void cleanup(Context context) throws IOException, InterruptedException {
-//        emit(context); // emit the last holding record
-//
-//        byte[] zerokey = new byte[] { 0 };
-//        outputKey.set(zerokey);
-//        outputValue.set(allRowCount);
-//        context.write(outputKey, outputValue);
-//    }
-//
-//    @Override
-//    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
-//        byte[] bytes = key.getBytes();
-//        int columnLength = bytes.length - this.headerLength;
-//        int columnPrefixLen = columnLength * this.columnPercentage / 100;
-//        if (columnPrefixLen == 0 && columnLength > 0) {
-//            columnPrefixLen = 1;
-//        }
-//        if (columnPrefixLen > 0) {
-//            currentKey.set(bytes, 0, this.headerLength + columnPrefixLen);
-//        } else {
-//            currentKey.set(bytes);
-//        }
-//
-//        allRowCount++;
-//
-//        if (outputKey.getLength() == 0) { // first record
-//            outputKey.set(currentKey);
-//            outputLong = 1;
-//        } else if (outputKey.equals(currentKey)) { // same key, note input is sorted
-//            outputLong++;
-//        } else { // the next key
-//            emit(context);
-//            outputKey.set(currentKey);
-//            outputLong = 1;
-//        }
-//    }
-//
-//    private void emit(Context context) throws IOException, InterruptedException {
-//        if (outputLong == 0)
-//            return;
-//
-//        outputValue.set(outputLong);
-//        context.write(outputKey, outputValue);
-//    }
-// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionReducer.java
deleted file mode 100644
index 97e1055..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyDistributionReducer.java
+++ /dev/null
@@ -1,113 +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.cube;
-
-///*
-// * Copyright 2013-2014 eBay Software Foundation
-// *
-// * Licensed under the Apache License, Version 2.0 (the "License");
-// * you may not use this file except in compliance with the License.
-// * You may obtain a copy of the License at
-// *
-// *   http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT 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.index.cube;
-//
-//import java.io.IOException;
-//
-//import org.apache.hadoop.io.LongWritable;
-//import org.apache.hadoop.io.Text;
-//import org.apache.hadoop.mapreduce.Reducer;
-//import org.apache.hadoop.util.StringUtils;
-//import org.slf4j.Logger;
-//import org.slf4j.LoggerFactory;
-//
-///**
-// * @author xjiang
-// *
-// */
-//public class KeyDistributionReducer extends KylinReducer<Text, LongWritable, Text, LongWritable> {
-//
-//    private static final Logger logger = LoggerFactory.getLogger(KeyDistributionReducer.class);
-//
-//    private LongWritable outputValue;
-//    private boolean isTotalCount;
-//    private long totalCount;
-//    private int splitNumber;
-//    private long splitQuota;
-//    private long splitRemain;
-//
-//    @Override
-//    protected void setup(Context context) throws IOException, InterruptedException {
-//        super.publishConfiguration(context.getConfiguration());
-
-//        String splitStr = context.getConfiguration().get(KeyDistributionJob.KEY_SPLIT_NUMBER);
-//        splitNumber = Integer.valueOf(splitStr).intValue();
-//        outputValue = new LongWritable();
-//        isTotalCount = true;
-//        totalCount = 0;
-//        splitQuota = 0;
-//        splitRemain = 0;
-//    }
-//
-//    @Override
-//    protected void cleanup(Context context) throws IOException, InterruptedException {
-//        logger.info("---------------");
-//        long splitCount = splitQuota - splitRemain;
-//        logger.info("Total Count = " + totalCount + ", Left Count = " + splitCount);
-//    }
-//
-//    @Override
-//    public void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException,
-//            InterruptedException {
-//
-//        // calculate split quota
-//        if (isTotalCount) {
-//            for (LongWritable count : values) {
-//                totalCount += count.get();
-//            }
-//            splitQuota = totalCount / splitNumber;
-//            splitRemain = splitQuota;
-//            isTotalCount = false;
-//            return;
-//        }
-//
-//        // output key when split quota is used up 
-//        for (LongWritable count : values) {
-//            splitRemain -= count.get();
-//        }
-//        if (splitRemain <= 0) {
-//            long splitCount = splitQuota - splitRemain;
-//            String hexKey = StringUtils.byteToHexString(key.getBytes());
-//            logger.info(hexKey + "\t\t" + splitCount);
-//
-//            outputValue.set(splitCount);
-//            context.write(key, outputValue);
-//            splitRemain = splitQuota;
-//        }
-//
-//    }
-// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2ef9831e/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyValueCreator.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyValueCreator.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyValueCreator.java
deleted file mode 100644
index 0318a8b..0000000
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/KeyValueCreator.java
+++ /dev/null
@@ -1,104 +0,0 @@
-package org.apache.kylin.job.hadoop.cube;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.hadoop.io.Text;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.metadata.measure.MeasureCodec;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * @author George Song (ysong1)
- */
-public class KeyValueCreator {
-    byte[] cfBytes;
-    byte[] qBytes;
-    long timestamp;
-
-    int[] refIndex;
-    MeasureDesc[] refMeasures;
-
-    MeasureCodec codec;
-    Object[] colValues;
-    ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-
-    public boolean isFullCopy;
-
-    public KeyValueCreator(CubeDesc cubeDesc, HBaseColumnDesc colDesc) {
-
-        cfBytes = Bytes.toBytes(colDesc.getColumnFamilyName());
-        qBytes = Bytes.toBytes(colDesc.getQualifier());
-        timestamp = System.currentTimeMillis();
-
-        List<MeasureDesc> measures = cubeDesc.getMeasures();
-        String[] measureNames = getMeasureNames(cubeDesc);
-        String[] refs = colDesc.getMeasureRefs();
-
-        refIndex = new int[refs.length];
-        refMeasures = new MeasureDesc[refs.length];
-        for (int i = 0; i < refs.length; i++) {
-            refIndex[i] = indexOf(measureNames, refs[i]);
-            refMeasures[i] = measures.get(refIndex[i]);
-        }
-
-        codec = new MeasureCodec(refMeasures);
-        colValues = new Object[refs.length];
-
-        isFullCopy = true;
-        for (int i = 0; i < measures.size(); i++) {
-            if (refIndex.length <= i || refIndex[i] != i)
-                isFullCopy = false;
-        }
-    }
-
-    public KeyValue create(Text key, Object[] measureValues) {
-        return create(key.getBytes(), 0, key.getLength(), measureValues);
-    }
-
-    public KeyValue create(byte[] keyBytes, int keyOffset, int keyLength, Object[] measureValues) {
-        for (int i = 0; i < colValues.length; i++) {
-            colValues[i] = measureValues[refIndex[i]];
-        }
-
-        valueBuf.clear();
-        codec.encode(colValues, valueBuf);
-
-        return create(keyBytes, keyOffset, keyLength, valueBuf.array(), 0, valueBuf.position());
-    }
-
-
-    public KeyValue create(byte[] keyBytes, int keyOffset, int keyLength, byte[] value, int voffset, int vlen) {
-        return new KeyValue(keyBytes, keyOffset, keyLength, //
-                cfBytes, 0, cfBytes.length, //
-                qBytes, 0, qBytes.length, //
-                timestamp, KeyValue.Type.Put, //
-                value, voffset, vlen);
-    }
-
-    public KeyValue create(Text key, byte[] value, int voffset, int vlen) {
-        return create(key.getBytes(), 0, key.getLength(), value, voffset, vlen);
-    }
-
-    private int indexOf(String[] measureNames, String ref) {
-        for (int i = 0; i < measureNames.length; i++)
-            if (measureNames[i].equalsIgnoreCase(ref))
-                return i;
-
-        throw new IllegalArgumentException("Measure '" + ref + "' not found in " + Arrays.toString(measureNames));
-    }
-
-    private String[] getMeasureNames(CubeDesc cubeDesc) {
-        List<MeasureDesc> measures = cubeDesc.getMeasures();
-        String[] result = new String[measures.size()];
-        for (int i = 0; i < measures.size(); i++)
-            result[i] = measures.get(i).getName();
-        return result;
-    }
-
-}