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/11/27 10:00:56 UTC

[01/13] incubator-kylin git commit: KYLIN-976 very initial

Repository: incubator-kylin
Updated Branches:
  refs/heads/KYLIN-976 [created] dac248804


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java
index 440dcc4..1a6d944 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java
@@ -20,8 +20,8 @@ package org.apache.kylin.storage.hbase.common.coprocessor;
 
 import java.util.Map;
 
+import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.common.util.MemoryBudgetController;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
 
 import com.google.common.collect.Maps;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
index fc1d967..7a75bbc 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
+import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorBehavior;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorFilter;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
index b7b12d1..b75510f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
+import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggregationCache;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
index 634e7d3..c9343fb 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
@@ -25,13 +25,13 @@ import java.util.Comparator;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
-import org.apache.kylin.metadata.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorConstants;
 import org.apache.kylin.storage.hbase.steps.RowValueDecoder;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
index dc33b27..a9d3b10 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
@@ -21,7 +21,7 @@ package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.kylin.metadata.measure.MeasureAggregator;
+import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggregationCache;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
index 95cca0d..8f0b17b 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
@@ -22,15 +22,15 @@ import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.invertedindex.index.RawTableRecord;
 import org.apache.kylin.invertedindex.index.TableRecordInfo;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
-import org.apache.kylin.metadata.measure.fixedlen.FixedLenMeasureCodec;
+import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
 import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorConstants;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
index e4360a2..046d8c1 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.CompressionUtils;
@@ -47,7 +48,6 @@ import org.apache.kylin.invertedindex.index.Slice;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
 import org.apache.kylin.invertedindex.model.IIDesc;
 import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorConstants;
@@ -65,6 +65,7 @@ import com.google.protobuf.HBaseZeroCopyByteString;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
+
 import it.uniroma3.mat.extendedset.intset.ConciseSet;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 9f97b0e..26876ec 100644
--- 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
@@ -25,6 +25,7 @@ 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.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.CubeDesc;
@@ -33,7 +34,6 @@ 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
index e7c4cf5..d93b108 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 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.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeSegment;
@@ -58,7 +59,6 @@ import org.apache.kylin.engine.mr.IMROutput2;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.steps.MergeCuboidMapper;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
-import org.apache.kylin.metadata.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 7f40259..4037f90 100644
--- 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
@@ -5,11 +5,11 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.io.Text;
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.util.Bytes;
 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;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 50c2fac..54217ff 100644
--- 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
@@ -23,11 +23,11 @@ import java.util.BitSet;
 import java.util.Collection;
 
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.DoubleMutable;
+import org.apache.kylin.common.util.LongMutable;
 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
index a5aba2c..08d4a49 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
@@ -36,8 +36,8 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.metadata.measure.LongMutable;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
index 8b56605..8e665f7 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
@@ -19,14 +19,15 @@
 package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint;
 
 import com.google.common.collect.Lists;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
-import org.apache.kylin.metadata.measure.fixedlen.FixedLenMeasureCodec;
+import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 270928f..cf706e4 100644
--- 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
@@ -29,13 +29,13 @@ 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.aggregation.MeasureCodec;
 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.engine.mr.HadoopUtil;
-import org.apache.kylin.metadata.measure.MeasureCodec;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
index ba79305..4662406 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
@@ -25,14 +25,14 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 
 import org.apache.hadoop.io.LongWritable;
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.cube.CubeManager;
 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.MetadataManager;
-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.junit.After;


[12/13] incubator-kylin git commit: KYLIN-976 Add ingester; Build part done, in-mem cube test pass

Posted by li...@apache.org.
KYLIN-976 Add ingester; Build part done, in-mem cube test pass


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

Branch: refs/heads/KYLIN-976
Commit: ce61309ac860c1ee82acb08f6525a419d422eaa5
Parents: 8f2a56c
Author: Li, Yang <ya...@ebay.com>
Authored: Thu Nov 26 18:20:48 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Fri Nov 27 14:49:10 2015 +0800

----------------------------------------------------------------------
 .../kylin/job/dataGen/FactTableGenerator.java   |   2 +-
 .../streaming/StreamingTableDataGenerator.java  |   2 +-
 .../common/datatype/BigDecimalSerializer.java   | 109 -------
 .../apache/kylin/common/datatype/DataType.java  | 289 -------------------
 .../common/datatype/DataTypeSerializer.java     | 108 -------
 .../common/datatype/DateTimeSerializer.java     |  62 ----
 .../kylin/common/datatype/DoubleMutable.java    |  68 -----
 .../kylin/common/datatype/DoubleSerializer.java |  81 ------
 .../kylin/common/datatype/LongMutable.java      |  70 -----
 .../kylin/common/datatype/LongSerializer.java   |  88 ------
 .../kylin/common/datatype/StringSerializer.java |  54 ----
 .../apache/kylin/common/util/Dictionary.java    | 232 +++++++++++++++
 .../datatype/BigDecimalSerializerTest.java      |  51 ----
 .../kylin/aggregation/AggregationType.java      |  97 -------
 .../kylin/aggregation/IAggregationFactory.java  |   6 -
 .../kylin/aggregation/MeasureAggregator.java    |  76 -----
 .../kylin/aggregation/MeasureAggregators.java   |  81 ------
 .../apache/kylin/aggregation/MeasureCodec.java  |  79 -----
 .../aggregation/basic/BasicAggregation.java     | 128 --------
 .../basic/BasicAggregationFactory.java          |  31 --
 .../basic/BigDecimalMaxAggregator.java          |  54 ----
 .../basic/BigDecimalMinAggregator.java          |  55 ----
 .../basic/BigDecimalSumAggregator.java          |  51 ----
 .../aggregation/basic/DoubleMaxAggregator.java  |  54 ----
 .../aggregation/basic/DoubleMinAggregator.java  |  54 ----
 .../aggregation/basic/DoubleSumAggregator.java  |  51 ----
 .../aggregation/basic/LongMaxAggregator.java    |  54 ----
 .../aggregation/basic/LongMinAggregator.java    |  54 ----
 .../aggregation/basic/LongSumAggregator.java    |  51 ----
 .../kylin/aggregation/hllc/HLLCAggregation.java |  78 -----
 .../hllc/HLLCAggregationFactory.java            |  35 ---
 .../kylin/aggregation/hllc/HLLCAggregator.java  |  64 ----
 .../kylin/aggregation/hllc/HLLCSerializer.java  |  98 -------
 .../kylin/aggregation/hllc/LDCAggregator.java   |  63 ----
 .../kylin/aggregation/topn/TopNAggregation.java |  76 -----
 .../topn/TopNAggregationFactory.java            |  35 ---
 .../kylin/aggregation/topn/TopNAggregator.java  |  66 -----
 .../aggregation/topn/TopNCounterSerializer.java | 117 --------
 .../java/org/apache/kylin/cube/CubeManager.java |   7 +-
 .../java/org/apache/kylin/cube/CubeSegment.java |  19 +-
 .../kylin/cube/gridtable/CubeCodeSystem.java    |  30 +-
 .../kylin/cube/gridtable/CubeGridTable.java     |  13 +-
 .../gridtable/CuboidToGridTableMapping.java     |   2 +-
 .../gridtable/NotEnoughGTInfoException.java     |   1 +
 .../cube/gridtable/TrimmedCubeCodeSystem.java   |   4 +-
 .../inmemcubing/AbstractInMemCubeBuilder.java   |   6 +-
 .../cube/inmemcubing/DoggedCubeBuilder.java     |   6 +-
 .../cube/inmemcubing/InMemCubeBuilder.java      |  18 +-
 .../InMemCubeBuilderInputConverter.java         | 105 +++----
 .../cube/inmemcubing/InMemCubeBuilderUtils.java |   3 +-
 .../kylin/cube/kv/AbstractRowKeyEncoder.java    |   2 +-
 .../apache/kylin/cube/kv/RowKeyColumnIO.java    |   2 +-
 .../apache/kylin/cube/kv/RowKeyColumnOrder.java |   2 +-
 .../org/apache/kylin/cube/model/CubeDesc.java   |   4 +-
 .../model/validation/rule/FunctionRule.java     |   2 +-
 .../org/apache/kylin/cube/util/CubingUtils.java |  19 +-
 .../kylin/gridtable/DefaultGTComparator.java    |   2 +-
 .../kylin/gridtable/GTAggregateScanner.java     |   2 +-
 .../java/org/apache/kylin/gridtable/GTInfo.java |   2 +-
 .../kylin/gridtable/GTSampleCodeSystem.java     |   4 +-
 .../apache/kylin/gridtable/IGTCodeSystem.java   |   2 +-
 .../apache/kylin/gridtable/UnitTestSupport.java |   4 +-
 .../topn/TopNCounterSerializerTest.java         |  60 ----
 .../kylin/cube/DictionaryManagerTest.java       |   2 +-
 .../DoggedCubeBuilderStressTest.java            |   4 +-
 .../cube/inmemcubing/DoggedCubeBuilderTest.java |   4 +-
 .../cube/inmemcubing/InMemCubeBuilderTest.java  |  12 +-
 .../gridtable/AggregationCacheMemSizeTest.java  |  14 +-
 .../kylin/gridtable/DictGridTableTest.java      |   6 +-
 .../kylin/gridtable/SimpleGridTableTest.java    |   2 +-
 .../gridtable/SimpleInvertedIndexTest.java      |   6 +-
 .../metadata/measure/MeasureCodecTest.java      |   6 +-
 .../apache/kylin/dict/DateStrDictionary.java    |   1 +
 .../org/apache/kylin/dict/DictCodeSystem.java   |   1 +
 .../java/org/apache/kylin/dict/Dictionary.java  | 233 ---------------
 .../apache/kylin/dict/DictionaryGenerator.java  |   9 +-
 .../org/apache/kylin/dict/DictionaryInfo.java   |   1 +
 .../kylin/dict/DictionaryInfoSerializer.java    |   1 +
 .../apache/kylin/dict/DictionaryManager.java    |   3 +-
 .../apache/kylin/dict/DictionarySerializer.java |   1 +
 .../org/apache/kylin/dict/IDictionaryAware.java |   1 +
 .../dict/MultipleDictionaryValueEnumerator.java |   2 +
 .../apache/kylin/dict/TimeStrDictionary.java    |   1 +
 .../org/apache/kylin/dict/TrieDictionary.java   |   2 +
 .../apache/kylin/dict/lookup/SnapshotTable.java |   3 +-
 .../apache/kylin/dict/NumberDictionaryTest.java |   3 +-
 .../apache/kylin/measure/IMeasureFactory.java   |  24 ++
 .../apache/kylin/measure/MeasureAggregator.java |  76 +++++
 .../kylin/measure/MeasureAggregators.java       |  81 ++++++
 .../org/apache/kylin/measure/MeasureCodec.java  |  79 +++++
 .../apache/kylin/measure/MeasureIngester.java   |  44 +++
 .../org/apache/kylin/measure/MeasureType.java   | 103 +++++++
 .../measure/basic/BasicMeasureFactory.java      |  32 ++
 .../kylin/measure/basic/BasicMeasureType.java   | 135 +++++++++
 .../kylin/measure/basic/BigDecimalIngester.java |  40 +++
 .../measure/basic/BigDecimalMaxAggregator.java  |  54 ++++
 .../measure/basic/BigDecimalMinAggregator.java  |  55 ++++
 .../measure/basic/BigDecimalSumAggregator.java  |  51 ++++
 .../kylin/measure/basic/DoubleIngester.java     |  45 +++
 .../measure/basic/DoubleMaxAggregator.java      |  54 ++++
 .../measure/basic/DoubleMinAggregator.java      |  54 ++++
 .../measure/basic/DoubleSumAggregator.java      |  51 ++++
 .../kylin/measure/basic/LongIngester.java       |  45 +++
 .../kylin/measure/basic/LongMaxAggregator.java  |  54 ++++
 .../kylin/measure/basic/LongMinAggregator.java  |  54 ++++
 .../kylin/measure/basic/LongSumAggregator.java  |  51 ++++
 .../measure/hllc/HLLCAggregationFactory.java    |  36 +++
 .../kylin/measure/hllc/HLLCAggregator.java      |  64 ++++
 .../kylin/measure/hllc/HLLCMeasureType.java     | 100 +++++++
 .../kylin/measure/hllc/HLLCSerializer.java      |  87 ++++++
 .../kylin/measure/hllc/LDCAggregator.java       |  63 ++++
 .../kylin/measure/topn/TopNAggregator.java      |  66 +++++
 .../measure/topn/TopNCounterSerializer.java     | 101 +++++++
 .../kylin/measure/topn/TopNMeasureFactory.java  |  36 +++
 .../kylin/measure/topn/TopNMeasureType.java     | 111 +++++++
 .../metadata/datatype/BigDecimalSerializer.java | 100 +++++++
 .../kylin/metadata/datatype/DataType.java       | 278 ++++++++++++++++++
 .../metadata/datatype/DataTypeSerializer.java   |  99 +++++++
 .../metadata/datatype/DateTimeSerializer.java   |  49 ++++
 .../kylin/metadata/datatype/DoubleMutable.java  |  68 +++++
 .../metadata/datatype/DoubleSerializer.java     |  69 +++++
 .../kylin/metadata/datatype/LongMutable.java    |  70 +++++
 .../kylin/metadata/datatype/LongSerializer.java |  77 +++++
 .../metadata/datatype/StringSerializer.java     |  48 +++
 .../filter/TimeConditionLiteralsReplacer.java   |   2 +-
 .../apache/kylin/metadata/model/ColumnDesc.java |   2 +-
 .../kylin/metadata/model/FunctionDesc.java      |  10 +-
 .../kylin/metadata/model/MeasureDesc.java       |  12 -
 .../apache/kylin/metadata/model/TblColRef.java  |   2 +-
 .../metadata/realization/SQLDigestUtil.java     |   2 +-
 .../topn/TopNCounterSerializerTest.java         |  46 +++
 .../datatype/BigDecimalSerializerTest.java      |  53 ++++
 .../storage/translate/ColumnValueRange.java     |   2 +-
 .../org/apache/kylin/storage/tuple/Tuple.java   |   4 +-
 .../storage/translate/ColumnValueRangeTest.java |   2 +-
 .../engine/mr/steps/BaseCuboidMapperBase.java   | 123 +++-----
 .../kylin/engine/mr/steps/CuboidReducer.java    |   4 +-
 .../engine/mr/steps/InMemCuboidMapper.java      |   6 +-
 .../engine/mr/steps/InMemCuboidReducer.java     |   4 +-
 .../mr/steps/MergeCuboidFromStorageMapper.java  |   4 +-
 .../engine/mr/steps/MergeCuboidMapper.java      |   4 +-
 .../engine/mr/steps/MergeDictionaryStep.java    |   1 -
 .../kylin/engine/mr/steps/CubeReducerTest.java  |   4 +-
 .../engine/mr/steps/MergeCuboidMapperTest.java  |   4 +-
 .../apache/kylin/engine/spark/SparkCubing.java  |  12 +-
 .../streaming/OneOffStreamingBuilder.java       |   4 +-
 .../engine/streaming/StreamingBatchBuilder.java |   6 +-
 .../streaming/cube/StreamingCubeBuilder.java    |  10 +-
 .../apache/kylin/invertedindex/IISegment.java   |   3 +-
 .../index/CompressedValueContainer.java         |   3 +-
 .../invertedindex/index/RawTableRecord.java     |   4 +-
 .../apache/kylin/invertedindex/index/Slice.java |   3 +-
 .../kylin/invertedindex/index/SliceBuilder.java |   4 +-
 .../kylin/invertedindex/index/TableRecord.java  |   4 +-
 .../invertedindex/index/TableRecordInfo.java    |   4 +-
 .../index/TableRecordInfoDigest.java            |   4 +-
 .../invertedindex/measure/FixedHLLCodec.java    |   2 +-
 .../measure/FixedLenMeasureCodec.java           |   2 +-
 .../measure/FixedPointLongCodec.java            |   4 +-
 .../invertedindex/model/IIKeyValueCodec.java    |   4 +-
 .../invertedindex/util/IIDictionaryBuilder.java |   2 +-
 .../kylin/invertedindex/IIInstanceTest.java     |   2 +-
 .../invertedindex/InvertedIndexLocalTest.java   |   2 +-
 .../measure/FixedPointLongCodecTest.java        |   2 +-
 .../common/coprocessor/AggregationCache.java    |   2 +-
 .../common/coprocessor/FilterDecorator.java     |   2 +-
 .../storage/hbase/cube/v1/CubeStorageQuery.java |   2 +-
 .../hbase/cube/v1/CubeTupleConverter.java       |   2 +-
 .../observer/AggregationScanner.java            |   2 +-
 .../observer/ObserverAggregationCache.java      |   2 +-
 .../observer/ObserverAggregators.java           |   4 +-
 .../v1/coprocessor/observer/ObserverTuple.java  |   2 +-
 .../hbase/cube/v2/CubeTupleConverter.java       |   2 +-
 .../endpoint/BitMapFilterEvaluator.java         |   2 +-
 .../endpoint/ClearTextDictionary.java           |   2 +-
 .../endpoint/EndpointAggregationCache.java      |   2 +-
 .../endpoint/EndpointAggregators.java           |   6 +-
 .../ii/coprocessor/endpoint/IIEndpoint.java     |   4 +-
 .../coprocessor/endpoint/LocalDictionary.java   |   2 +-
 .../storage/hbase/steps/CreateHTableJob.java    |   2 +-
 .../storage/hbase/steps/CubeHFileMapper.java    |   2 +-
 .../hbase/steps/HBaseMROutput2Transition.java   |   2 +-
 .../storage/hbase/steps/KeyValueCreator.java    |   2 +-
 .../storage/hbase/steps/RowValueDecoder.java    |   6 +-
 .../observer/AggregateRegionObserverTest.java   |   2 +-
 .../endpoint/BitMapFilterEvaluatorTest.java     |   3 +-
 .../endpoint/EndpointAggregationTest.java       |   4 +-
 .../hbase/steps/CubeHFileMapper2Test.java       |   2 +-
 .../hbase/steps/RowValueDecoderTest.java        |   4 +-
 189 files changed, 3346 insertions(+), 3260 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java b/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
index 705c175..a85d471 100644
--- a/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
+++ b/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
@@ -37,7 +37,6 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.cube.CubeInstance;
@@ -45,6 +44,7 @@ import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.DimensionDesc;
 import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java b/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java
index 04a5b5b..4ce223e 100644
--- a/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java
+++ b/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java
@@ -5,8 +5,8 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/datatype/BigDecimalSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/BigDecimalSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/BigDecimalSerializer.java
deleted file mode 100644
index f19ce14..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/datatype/BigDecimalSerializer.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.common.datatype;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author yangli9
- * 
- */
-public class BigDecimalSerializer extends DataTypeSerializer<BigDecimal> {
-
-    private static final Logger logger = LoggerFactory.getLogger(BigDecimalSerializer.class);
-
-    final DataType type;
-    final int maxLength;
-
-    int avoidVerbose = 0;
-
-    public BigDecimalSerializer(DataType type) {
-        this.type = type;
-        // see serialize(): 1 byte scale, 1 byte length, assume every 2 digits takes 1 byte
-        this.maxLength = 1 + 1 + (type.getPrecision() + 1) / 2;
-    }
-
-    @Override
-    public void serialize(BigDecimal value, ByteBuffer out) {
-        if (value.scale() > type.getScale()) {
-            if (avoidVerbose % 10000 == 0) {
-                logger.warn("value's scale has exceeded the " + type.getScale() + ", cut it off, to ensure encoded value do not exceed maxLength " + maxLength + " times:" + (avoidVerbose++));
-            }
-            value = value.setScale(type.getScale(), BigDecimal.ROUND_HALF_EVEN);
-        }
-        byte[] bytes = value.unscaledValue().toByteArray();
-        if (bytes.length + 2 > maxLength) {
-            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
-        }
-
-        BytesUtil.writeVInt(value.scale(), out);
-        BytesUtil.writeVInt(bytes.length, out);
-        out.put(bytes);
-    }
-
-    @Override
-    public BigDecimal deserialize(ByteBuffer in) {
-        int scale = BytesUtil.readVInt(in);
-        int n = BytesUtil.readVInt(in);
-
-        byte[] bytes = new byte[n];
-        in.get(bytes);
-
-        return new BigDecimal(new BigInteger(bytes), scale);
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-
-        @SuppressWarnings("unused")
-        int scale = BytesUtil.readVInt(in);
-        int n = BytesUtil.readVInt(in);
-        int len = in.position() - mark + n;
-
-        in.position(mark);
-        return len;
-    }
-
-    @Override
-    public int maxLength() {
-        return maxLength;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 8;
-    }
-
-    @Override
-    public BigDecimal valueOf(byte[] value) {
-        if (value == null)
-            return new BigDecimal(0);
-        else
-            return new BigDecimal(Bytes.toString(value));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/datatype/DataType.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DataType.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DataType.java
deleted file mode 100644
index 36fad00..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/datatype/DataType.java
+++ /dev/null
@@ -1,289 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.common.datatype;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-/**
- */
-@SuppressWarnings("serial")
-public class DataType implements Serializable {
-
-    // standard sql types, ref: http://www.w3schools.com/sql/sql_datatypes_general.asp
-    public static final String VALID_TYPES_STRING = "any|char|varchar|boolean|binary" //
-            + "|integer|tinyint|smallint|bigint|decimal|numeric|float|real|double" //
-            + "|date|time|datetime|timestamp|byte|int|short|long|string";
-
-    private static final String TYPE_PATTEN_TAIL = "\\s*" //
-            + "(?:" + "[(]" + "([\\d\\s,]+)" + "[)]" + ")?";
-
-    private static final Pattern TYPE_PATTERN = Pattern.compile( //
-            "(" + VALID_TYPES_STRING + ")" + TYPE_PATTEN_TAIL, Pattern.CASE_INSENSITIVE);
-
-    private static final Pattern CUSTOM_TYPE_PATTERN = Pattern.compile( //
-            "(" + ".*?" + ")" + TYPE_PATTEN_TAIL, Pattern.CASE_INSENSITIVE);
-    
-    public static final Set<String> INTEGER_FAMILY = new HashSet<String>();
-    public static final Set<String> NUMBER_FAMILY = new HashSet<String>();
-    public static final Set<String> DATETIME_FAMILY = new HashSet<String>();
-    public static final Set<String> STRING_FAMILY = new HashSet<String>();
-    private static final Map<String, String> LEGACY_TYPE_MAP = new HashMap<String, String>();
-    static {
-        INTEGER_FAMILY.add("tinyint");
-        INTEGER_FAMILY.add("smallint");
-        INTEGER_FAMILY.add("integer");
-        INTEGER_FAMILY.add("bigint");
-
-        NUMBER_FAMILY.addAll(INTEGER_FAMILY);
-        NUMBER_FAMILY.add("float");
-        NUMBER_FAMILY.add("double");
-        NUMBER_FAMILY.add("decimal");
-        NUMBER_FAMILY.add("real");
-        NUMBER_FAMILY.add("numeric");
-
-        DATETIME_FAMILY.add("date");
-        DATETIME_FAMILY.add("time");
-        DATETIME_FAMILY.add("datetime");
-        DATETIME_FAMILY.add("timestamp");
-
-        STRING_FAMILY.add("varchar");
-        STRING_FAMILY.add("char");
-
-        LEGACY_TYPE_MAP.put("byte", "tinyint");
-        LEGACY_TYPE_MAP.put("int", "integer");
-        LEGACY_TYPE_MAP.put("short", "smallint");
-        LEGACY_TYPE_MAP.put("long", "bigint");
-        LEGACY_TYPE_MAP.put("string", "varchar");
-        LEGACY_TYPE_MAP.put("hllc10", "hllc(10)");
-        LEGACY_TYPE_MAP.put("hllc12", "hllc(12)");
-        LEGACY_TYPE_MAP.put("hllc14", "hllc(14)");
-        LEGACY_TYPE_MAP.put("hllc15", "hllc(15)");
-        LEGACY_TYPE_MAP.put("hllc16", "hllc(16)");
-    }
-
-    private static final ConcurrentMap<DataType, DataType> CACHE = new ConcurrentHashMap<DataType, DataType>();
-
-    public static final DataType ANY = DataType.getType("any");
-
-    public static DataType getType(String type) {
-        return getTypeInner(type, false);
-    }
-    
-    public static DataType getCustomType(String type) {
-        return getTypeInner(type, true);
-    }
-    
-    private static DataType getTypeInner(String type, boolean isCustom) {
-        if (type == null)
-            return null;
-
-        DataType dataType = new DataType(type, isCustom);
-        DataType cached = CACHE.get(dataType);
-        if (cached == null) {
-            CACHE.put(dataType, dataType);
-            cached = dataType;
-        }
-        return cached;
-    }
-
-    // ============================================================================
-
-    private String name;
-    private int precision;
-    private int scale;
-
-    DataType(String datatype, boolean isCustom) {
-        datatype = datatype.trim().toLowerCase();
-        datatype = replaceLegacy(datatype);
-
-        Pattern pattern = isCustom ? CUSTOM_TYPE_PATTERN : TYPE_PATTERN;
-        Matcher m = pattern.matcher(datatype);
-        if (m.matches() == false)
-            throw new IllegalArgumentException("bad data type -- " + datatype + ", does not match " + pattern);
-
-        name = replaceLegacy(m.group(1));
-        precision = -1;
-        scale = -1;
-
-        String leftover = m.group(2);
-        if (leftover != null) {
-            String[] parts = leftover.split("\\s*,\\s*");
-            for (int i = 0; i < parts.length; i++) {
-                int n;
-                try {
-                    n = Integer.parseInt(parts[i]);
-                } catch (NumberFormatException e) {
-                    throw new IllegalArgumentException("bad data type -- " + datatype + ", precision/scale not numeric");
-                }
-                if (i == 0)
-                    precision = n;
-                else if (i == 1)
-                    scale = n;
-                else
-                    throw new IllegalArgumentException("bad data type -- " + datatype + ", too many precision/scale parts");
-            }
-        }
-
-        // FIXME 256 for unknown string precision
-        if ((name.equals("char") || name.equals("varchar")) && precision == -1) {
-            precision = 256; // to save memory at frontend, e.g. tableau will
-                             // allocate memory according to this
-        }
-
-        // FIXME (19,4) for unknown decimal precision
-        if ((name.equals("decimal") || name.equals("numeric")) && precision == -1) {
-            precision = 19;
-            scale = 4;
-        }
-    }
-
-    private String replaceLegacy(String str) {
-        String replace = LEGACY_TYPE_MAP.get(str);
-        return replace == null ? str : replace;
-    }
-
-    public int getStorageBytesEstimate() {
-        return DataTypeSerializer.create(this).getStorageBytesEstimate();
-    }
-
-    public boolean isStringFamily() {
-        return STRING_FAMILY.contains(name);
-    }
-
-    public boolean isIntegerFamily() {
-        return INTEGER_FAMILY.contains(name);
-    }
-
-    public boolean isNumberFamily() {
-        return NUMBER_FAMILY.contains(name);
-    }
-
-    public boolean isDateTimeFamily() {
-        return DATETIME_FAMILY.contains(name);
-    }
-
-    public boolean isDate() {
-        return name.equals("date");
-    }
-
-    public boolean isTime() {
-        return name.equals("time");
-    }
-
-    public boolean isTimestamp() {
-        return name.equals("timestamp");
-    }
-
-    public boolean isDatetime() {
-        return name.equals("datetime");
-    }
-
-    public boolean isTinyInt() {
-        return name.equals("tinyint");
-    }
-
-    public boolean isSmallInt() {
-        return name.equals("smallint");
-    }
-
-    public boolean isInt() {
-        return name.equals("integer");
-    }
-
-    public boolean isBigInt() {
-        return name.equals("bigint");
-    }
-
-    public boolean isFloat() {
-        return name.equals("float");
-    }
-
-    public boolean isDouble() {
-        return name.equals("double");
-    }
-
-    public boolean isDecimal() {
-        return name.equals("decimal");
-    }
-
-    public boolean isHLLC() {
-        return name.equals("hllc");
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public int getPrecision() {
-        return precision;
-    }
-
-    public int getScale() {
-        return scale;
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((name == null) ? 0 : name.hashCode());
-        result = prime * result + precision;
-        result = prime * result + scale;
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        DataType other = (DataType) obj;
-        if (name == null) {
-            if (other.name != null)
-                return false;
-        } else if (!name.equals(other.name))
-            return false;
-        if (precision != other.precision)
-            return false;
-        if (scale != other.scale)
-            return false;
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        if (precision < 0 && scale < 0)
-            return name;
-        else if (scale < 0)
-            return name + "(" + precision + ")";
-        else
-            return name + "(" + precision + "," + scale + ")";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/datatype/DataTypeSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DataTypeSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DataTypeSerializer.java
deleted file mode 100644
index 99fa3fd..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/datatype/DataTypeSerializer.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.common.datatype;
-
-import java.io.UnsupportedEncodingException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.kylin.common.util.BytesSerializer;
-
-import com.google.common.collect.Maps;
-
-/**
- * @author yangli9
- * 
- * Note: the implementations MUST be thread-safe.
- */
-abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
-
-    final static Map<String, Class<?>> implementations;
-    static {
-        HashMap<String, Class<?>> impl = Maps.newHashMap();
-        impl.put("varchar", StringSerializer.class);
-        impl.put("decimal", BigDecimalSerializer.class);
-        impl.put("double", DoubleSerializer.class);
-        impl.put("float", DoubleSerializer.class);
-        impl.put("bigint", LongSerializer.class);
-        impl.put("long", LongSerializer.class);
-        impl.put("integer", LongSerializer.class);
-        impl.put("int", LongSerializer.class);
-        impl.put("smallint", LongSerializer.class);
-        impl.put("date", DateTimeSerializer.class);
-        impl.put("datetime", DateTimeSerializer.class);
-        impl.put("timestamp", DateTimeSerializer.class);
-        implementations = Collections.unmodifiableMap(impl);
-    }
-    
-    public static boolean hasRegistered(String dataTypeName) {
-        return implementations.containsKey(dataTypeName);
-    }
-    
-    public static void register(String dataTypeName, Class<? extends DataTypeSerializer<?>> impl) {
-        implementations.put(dataTypeName, impl);
-    }
-
-    public static DataTypeSerializer<?> create(String dataType) {
-        return create(DataType.getType(dataType));
-    }
-
-    public static DataTypeSerializer<?> create(DataType type) {
-        Class<?> clz = implementations.get(type.getName());
-        if (clz == null)
-            throw new RuntimeException("No DataTypeSerializer for type " + type);
-
-        try {
-            return (DataTypeSerializer<?>) clz.getConstructor(DataType.class).newInstance(type);
-        } catch (Exception e) {
-            throw new RuntimeException(e); // never happen
-        }
-    }
-    
-    /** peek into buffer and return the length of serialization */
-    abstract public int peekLength(ByteBuffer in);
-
-    /** return the max number of bytes to the longest serialization */
-    abstract public int maxLength();
-
-    /** get an estimate of size in bytes of the serialized data */
-    abstract public int getStorageBytesEstimate();
-
-    /** convert from String to obj (string often come as byte[] in mapred) */
-    abstract public T valueOf(byte[] value);
-
-    /** convert from String to obj */
-    public T valueOf(String value) {
-        try {
-            return valueOf(value.getBytes("UTF-8"));
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException(e); // never happen
-        }
-    }
-
-    /** convert from obj to string */
-    public String toString(T value) {
-        if (value == null)
-            return "NULL";
-        else
-            return value.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/datatype/DateTimeSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DateTimeSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DateTimeSerializer.java
deleted file mode 100644
index d3ef7cd..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/datatype/DateTimeSerializer.java
+++ /dev/null
@@ -1,62 +0,0 @@
-package org.apache.kylin.common.datatype;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.DateFormat;
-
-public class DateTimeSerializer extends DataTypeSerializer<LongMutable> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
-
-    public DateTimeSerializer(DataType type) {
-    }
-
-    @Override
-    public void serialize(LongMutable value, ByteBuffer out) {
-        out.putLong(value.get());
-    }
-
-    private LongMutable current() {
-        LongMutable l = current.get();
-        if (l == null) {
-            l = new LongMutable();
-            current.set(l);
-        }
-        return l;
-    }
-
-    @Override
-    public LongMutable deserialize(ByteBuffer in) {
-        LongMutable l = current();
-        l.set(in.getLong());
-        return l;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return 8;
-    }
-
-    @Override
-    public int maxLength() {
-        return 8;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 8;
-    }
-
-    @Override
-    public LongMutable valueOf(byte[] value) {
-        LongMutable l = current();
-        if (value == null)
-            l.set(0L);
-        else
-            l.set(DateFormat.stringToMillis(Bytes.toString(value)));
-        return l;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleMutable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleMutable.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleMutable.java
deleted file mode 100644
index a32fbd8..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleMutable.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.common.datatype;
-
-import java.io.Serializable;
-
-@SuppressWarnings("serial")
-public class DoubleMutable implements Comparable<DoubleMutable>, Serializable {
-
-    private double v;
-
-    public DoubleMutable() {
-        this(0);
-    }
-
-    public DoubleMutable(double v) {
-        set(v);
-    }
-
-    public double get() {
-        return v;
-    }
-
-    public void set(double v) {
-        this.v = v;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof DoubleMutable)) {
-            return false;
-        }
-        DoubleMutable other = (DoubleMutable) o;
-        return this.v == other.v;
-    }
-
-    @Override
-    public int hashCode() {
-        return (int) Double.doubleToLongBits(v);
-    }
-
-    @Override
-    public int compareTo(DoubleMutable o) {
-        return (v < o.v ? -1 : (v == o.v ? 0 : 1));
-    }
-
-    @Override
-    public String toString() {
-        return Double.toString(v);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleSerializer.java
deleted file mode 100644
index f128576..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleSerializer.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.common.datatype;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-
-/**
- */
-public class DoubleSerializer extends DataTypeSerializer<DoubleMutable> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<DoubleMutable> current = new ThreadLocal<DoubleMutable>();
-
-    public DoubleSerializer(DataType type) {
-    }
-
-    @Override
-    public void serialize(DoubleMutable value, ByteBuffer out) {
-        out.putDouble(value.get());
-    }
-
-    private DoubleMutable current() {
-        DoubleMutable d = current.get();
-        if (d == null) {
-            d = new DoubleMutable();
-            current.set(d);
-        }
-        return d;
-    }
-
-    @Override
-    public DoubleMutable deserialize(ByteBuffer in) {
-        DoubleMutable d = current();
-        d.set(in.getDouble());
-        return d;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return 8;
-    }
-
-    @Override
-    public int maxLength() {
-        return 8;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 8;
-    }
-
-    @Override
-    public DoubleMutable valueOf(byte[] value) {
-        DoubleMutable d = current();
-        if (value == null)
-            d.set(0d);
-        else
-            d.set(Double.parseDouble(Bytes.toString(value)));
-        return d;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/datatype/LongMutable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/LongMutable.java b/core-common/src/main/java/org/apache/kylin/common/datatype/LongMutable.java
deleted file mode 100644
index 72b540c..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/datatype/LongMutable.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.common.datatype;
-
-import java.io.Serializable;
-
-@SuppressWarnings("serial")
-public class LongMutable implements Comparable<LongMutable>, Serializable {
-
-    private long v;
-
-    public LongMutable() {
-        this(0);
-    }
-
-    public LongMutable(long v) {
-        set(v);
-    }
-
-    public long get() {
-        return v;
-    }
-
-    public void set(long v) {
-        this.v = v;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof LongMutable)) {
-            return false;
-        }
-        LongMutable other = (LongMutable) o;
-        return this.v == other.v;
-    }
-
-    @Override
-    public int hashCode() {
-        return (int) v;
-    }
-
-    @Override
-    public int compareTo(LongMutable o) {
-        long thisValue = this.v;
-        long thatValue = o.v;
-        return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
-    }
-
-    @Override
-    public String toString() {
-        return Long.toString(v);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/datatype/LongSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/LongSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/LongSerializer.java
deleted file mode 100644
index 5c5e10f..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/datatype/LongSerializer.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.common.datatype;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-
-/**
- */
-public class LongSerializer extends DataTypeSerializer<LongMutable> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
-
-    public LongSerializer(DataType type) {
-    }
-
-    @Override
-    public void serialize(LongMutable value, ByteBuffer out) {
-        BytesUtil.writeVLong(value.get(), out);
-    }
-
-    private LongMutable current() {
-        LongMutable l = current.get();
-        if (l == null) {
-            l = new LongMutable();
-            current.set(l);
-        }
-        return l;
-    }
-
-    @Override
-    public LongMutable deserialize(ByteBuffer in) {
-        LongMutable l = current();
-        l.set(BytesUtil.readVLong(in));
-        return l;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-
-        BytesUtil.readVLong(in);
-        int len = in.position() - mark;
-
-        in.position(mark);
-        return len;
-    }
-
-    @Override
-    public int maxLength() {
-        return 9; // vlong: 1 + 8
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 5;
-    }
-
-    @Override
-    public LongMutable valueOf(byte[] value) {
-        LongMutable l = current();
-        if (value == null)
-            l.set(0L);
-        else
-            l.set(Long.parseLong(Bytes.toString(value)));
-        return l;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/datatype/StringSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/StringSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/StringSerializer.java
deleted file mode 100644
index b3562eb..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/datatype/StringSerializer.java
+++ /dev/null
@@ -1,54 +0,0 @@
-package org.apache.kylin.common.datatype;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-
-public class StringSerializer extends DataTypeSerializer<String> {
-
-    final DataType type;
-    final int maxLength;
-
-    public StringSerializer(DataType type) {
-        this.type = type;
-        // see serialize(): 2 byte length, rest is String.toBytes()
-        this.maxLength = 2 + type.getPrecision();
-    }
-
-    @Override
-    public void serialize(String value, ByteBuffer out) {
-        int start = out.position();
-
-        BytesUtil.writeUTFString(value, out);
-
-        if (out.position() - start > maxLength)
-            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
-    }
-
-    @Override
-    public String deserialize(ByteBuffer in) {
-        return BytesUtil.readUTFString(in);
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return BytesUtil.peekByteArrayLength(in);
-    }
-
-    @Override
-    public int maxLength() {
-        return maxLength;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return maxLength;
-    }
-
-    @Override
-    public String valueOf(byte[] value) {
-        return Bytes.toString(value);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java b/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
new file mode 100644
index 0000000..6d3fa62
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.common.util;
+
+import java.io.PrintStream;
+import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
+
+import org.apache.kylin.common.persistence.Writable;
+
+/**
+ * A bi-way dictionary that maps from dimension/column values to IDs and vice
+ * versa. By storing IDs instead of real values, the size of cube is
+ * significantly reduced.
+ * 
+ * - IDs are smallest integers possible for the cardinality of a column, for the
+ * purpose of minimal storage space - IDs preserve ordering of values, such that
+ * range query can be applied to IDs directly
+ * 
+ * A dictionary once built, is immutable. This allows optimal memory footprint
+ * by e.g. flatten the Trie structure into a byte array, replacing node pointers
+ * with array offsets.
+ * 
+ * @author yangli9
+ */
+@SuppressWarnings("serial")
+abstract public class Dictionary<T> implements Writable, Serializable {
+
+    public static final byte NULL = (byte) 0xff;
+
+    // ID with all bit-1 (0xff e.g.) reserved for NULL value
+    public static final int NULL_ID[] = new int[] { 0, 0xff, 0xffff, 0xffffff, 0xffffffff };
+
+    abstract public int getMinId();
+
+    abstract public int getMaxId();
+
+    public int getSize() {
+        return getMaxId() - getMinId() + 1;
+    }
+
+    /**
+     * @return the size of an ID in bytes, determined by the cardinality of column
+     */
+    abstract public int getSizeOfId();
+
+    /**
+     * @return the (maximum) size of value in bytes, determined by the longest value
+     */
+    abstract public int getSizeOfValue();
+
+    /**
+     * @return true if each entry of this dict is contained by the dict in param
+     */
+    abstract public boolean contains(Dictionary<?> another);
+
+    /**
+     * Convenient form of <code>getIdFromValue(value, 0)</code>
+     */
+    final public int getIdFromValue(T value) throws IllegalArgumentException {
+        return getIdFromValue(value, 0);
+    }
+
+    /**
+     * Returns the ID integer of given value. In case of not found
+     * <p>
+     * - if roundingFlag=0, throw IllegalArgumentException; <br>
+     * - if roundingFlag<0, the closest smaller ID integer if exist; <br>
+     * - if roundingFlag>0, the closest bigger ID integer if exist. <br>
+     * <p>
+     * The implementation often has cache, thus faster than the byte[] version getIdFromValueBytes()
+     * 
+     * @throws IllegalArgumentException
+     *             if value is not found in dictionary and rounding is off;
+     *             or if rounding cannot find a smaller or bigger ID
+     */
+    final public int getIdFromValue(T value, int roundingFlag) throws IllegalArgumentException {
+        if (isNullObjectForm(value))
+            return nullId();
+        else
+            return getIdFromValueImpl(value, roundingFlag);
+    }
+
+    final public boolean containsValue(T value) throws IllegalArgumentException {
+        if (isNullObjectForm(value)) {
+            return true;
+        } else {
+            try {
+                //if no key found, it will throw exception
+                getIdFromValueImpl(value, 0);
+            } catch (IllegalArgumentException e) {
+                return false;
+            }
+            return true;
+        }
+    }
+
+    protected boolean isNullObjectForm(T value) {
+        return value == null;
+    }
+
+    abstract protected int getIdFromValueImpl(T value, int roundingFlag);
+
+    /**
+     * @return the value corresponds to the given ID
+     * @throws IllegalArgumentException
+     *             if ID is not found in dictionary
+     */
+    final public T getValueFromId(int id) throws IllegalArgumentException {
+        if (isNullId(id))
+            return null;
+        else
+            return getValueFromIdImpl(id);
+    }
+
+    abstract protected T getValueFromIdImpl(int id);
+
+    /**
+     * Convenient form of
+     * <code>getIdFromValueBytes(value, offset, len, 0)</code>
+     */
+    final public int getIdFromValueBytes(byte[] value, int offset, int len) throws IllegalArgumentException {
+        return getIdFromValueBytes(value, offset, len, 0);
+    }
+
+    /**
+     * A lower level API, return ID integer from raw value bytes. In case of not found 
+     * <p>
+     * - if roundingFlag=0, throw IllegalArgumentException; <br>
+     * - if roundingFlag<0, the closest smaller ID integer if exist; <br>
+     * - if roundingFlag>0, the closest bigger ID integer if exist. <br>
+     * <p>
+     * Bypassing the cache layer, this could be significantly slower than getIdFromValue(T value).
+     * 
+     * @throws IllegalArgumentException
+     *             if value is not found in dictionary and rounding is off;
+     *             or if rounding cannot find a smaller or bigger ID
+     */
+    final public int getIdFromValueBytes(byte[] value, int offset, int len, int roundingFlag) throws IllegalArgumentException {
+        if (isNullByteForm(value, offset, len))
+            return nullId();
+        else {
+            int id = getIdFromValueBytesImpl(value, offset, len, roundingFlag);
+            if (id < 0)
+                throw new IllegalArgumentException("Value not exists!");
+            return id;
+        }
+    }
+
+    protected boolean isNullByteForm(byte[] value, int offset, int len) {
+        return value == null;
+    }
+
+    abstract protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag);
+
+    final public byte[] getValueBytesFromId(int id) {
+        if (isNullId(id))
+            return BytesUtil.EMPTY_BYTE_ARRAY;
+        else
+            return getValueBytesFromIdImpl(id);
+    }
+
+    abstract protected byte[] getValueBytesFromIdImpl(int id);
+
+    /**
+     * A lower level API, get byte values from ID, return the number of bytes
+     * written. Bypassing the cache layer, this could be significantly slower
+     * than getIdFromValue(T value).
+     *
+     * @return size of value bytes, 0 if empty string, -1 if null
+     *
+     * @throws IllegalArgumentException
+     *             if ID is not found in dictionary
+     */
+    final public int getValueBytesFromId(int id, byte[] returnValue, int offset) throws IllegalArgumentException {
+        if (isNullId(id))
+            return -1;
+        else
+            return getValueBytesFromIdImpl(id, returnValue, offset);
+    }
+
+    abstract protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset);
+
+    abstract public void dump(PrintStream out);
+
+    public int nullId() {
+        return NULL_ID[getSizeOfId()];
+    }
+
+    public boolean isNullId(int id) {
+        int nullId = NULL_ID[getSizeOfId()];
+        return (nullId & id) == nullId;
+    }
+
+    /** utility that converts a dictionary ID to string, preserving order */
+    public static String dictIdToString(byte[] idBytes, int offset, int length) {
+        try {
+            return new String(idBytes, offset, length, "ISO-8859-1");
+        } catch (UnsupportedEncodingException e) {
+            // never happen
+            return null;
+        }
+    }
+
+    /** the reverse of dictIdToString(), returns integer ID */
+    public static int stringToDictId(String str) {
+        try {
+            byte[] bytes = str.getBytes("ISO-8859-1");
+            return BytesUtil.readUnsigned(bytes, 0, bytes.length);
+        } catch (UnsupportedEncodingException e) {
+            // never happen
+            return 0;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-common/src/test/java/org/apache/kylin/common/datatype/BigDecimalSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/datatype/BigDecimalSerializerTest.java b/core-common/src/test/java/org/apache/kylin/common/datatype/BigDecimalSerializerTest.java
deleted file mode 100644
index cba6795..0000000
--- a/core-common/src/test/java/org/apache/kylin/common/datatype/BigDecimalSerializerTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-package org.apache.kylin.common.datatype;
-
-import static org.junit.Assert.*;
-
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- */
-public class BigDecimalSerializerTest {
-
-    private static BigDecimalSerializer bigDecimalSerializer;
-
-    @BeforeClass
-    public static void beforeClass() {
-        bigDecimalSerializer = new BigDecimalSerializer(DataType.getType("decimal"));
-    }
-
-    @Test
-    public void testNormal() {
-        BigDecimal input = new BigDecimal("1234.1234");
-        ByteBuffer buffer = ByteBuffer.allocate(256);
-        buffer.mark();
-        bigDecimalSerializer.serialize(input, buffer);
-        buffer.reset();
-        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
-        assertEquals(input, output);
-    }
-
-    @Test
-    public void testScaleOutOfRange() {
-        BigDecimal input = new BigDecimal("1234.1234567890");
-        ByteBuffer buffer = ByteBuffer.allocate(256);
-        buffer.mark();
-        bigDecimalSerializer.serialize(input, buffer);
-        buffer.reset();
-        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
-        assertEquals(input.setScale(bigDecimalSerializer.type.getScale(), BigDecimal.ROUND_HALF_EVEN), output);
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testOutOfPrecision() {
-        BigDecimal input = new BigDecimal("66855344214907231736.4924");
-        ByteBuffer buffer = ByteBuffer.allocate(256);
-        bigDecimalSerializer.serialize(input, buffer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/AggregationType.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/AggregationType.java b/core-cube/src/main/java/org/apache/kylin/aggregation/AggregationType.java
deleted file mode 100644
index 81345b7..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/AggregationType.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.aggregation;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.kylin.aggregation.basic.BasicAggregationFactory;
-import org.apache.kylin.aggregation.hllc.HLLCAggregationFactory;
-import org.apache.kylin.aggregation.topn.TopNAggregationFactory;
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.DataTypeSerializer;
-import org.apache.kylin.dict.Dictionary;
-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.Maps;
-
-abstract public class AggregationType {
-    
-    private static final Map<String, IAggregationFactory> factoryRegistry = Maps.newConcurrentMap();
-    private static final IAggregationFactory defaultFactory = new BasicAggregationFactory();
-    
-    static {
-        factoryRegistry.put(FunctionDesc.FUNC_COUNT_DISTINCT, new HLLCAggregationFactory());
-        factoryRegistry.put(FunctionDesc.FUNC_TOP_N, new TopNAggregationFactory());
-    }
-    
-    public static AggregationType create(String funcName, String dataType) {
-        funcName = funcName.toUpperCase();
-        dataType = dataType.toLowerCase();
-        
-        IAggregationFactory factory = factoryRegistry.get(funcName);
-        if (factory == null)
-            factory = defaultFactory;
-        
-        AggregationType result = factory.createAggregationType(funcName, dataType);
-        
-        // register serializer for aggr data type
-        DataType aggregationDataType = result.getAggregationDataType();
-        if (DataTypeSerializer.hasRegistered(aggregationDataType.getName()) == false) {
-            DataTypeSerializer.register(aggregationDataType.getName(), result.getAggregationDataSeralizer());
-        }
-        
-        return result;
-    }
-    
-    /* ============================================================================
-     * Define
-     * ---------------------------------------------------------------------------- */
-    
-    abstract public DataType getAggregationDataType();
-    
-    abstract public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer();
-    
-    abstract public void validate(MeasureDesc measureDesc) throws IllegalArgumentException;
-    
-    /* ============================================================================
-     * Build
-     * ---------------------------------------------------------------------------- */
-    
-    abstract public MeasureAggregator<?> newAggregator();
- 
-    abstract public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc);
-    
-    abstract public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts);
-
-    /* ============================================================================
-     * Cube Selection
-     * ---------------------------------------------------------------------------- */
-    
-    /* ============================================================================
-     * Query
-     * ---------------------------------------------------------------------------- */
-    
-    /* ============================================================================
-     * Storage
-     * ---------------------------------------------------------------------------- */
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/IAggregationFactory.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/IAggregationFactory.java b/core-cube/src/main/java/org/apache/kylin/aggregation/IAggregationFactory.java
deleted file mode 100644
index cbdd9da..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/IAggregationFactory.java
+++ /dev/null
@@ -1,6 +0,0 @@
-package org.apache.kylin.aggregation;
-
-public interface IAggregationFactory {
-
-    public AggregationType createAggregationType(String funcName, String dataType);
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
deleted file mode 100644
index 6968a9d..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.aggregation;
-
-import java.io.Serializable;
-
-/**
- */
-@SuppressWarnings("serial")
-abstract public class MeasureAggregator<V> implements Serializable {
-
-    public static MeasureAggregator<?> create(String funcName, String dataType) {
-        return AggregationType.create(funcName, dataType).newAggregator();
-    }
-
-    public static int guessBigDecimalMemBytes() {
-        // 116 returned by AggregationCacheMemSizeTest
-        return 8 // aggregator obj shell
-        + 8 // ref to BigDecimal
-        + 8 // BigDecimal obj shell
-        + 100; // guess of BigDecimal internal
-    }
-
-    public static int guessDoubleMemBytes() {
-        // 29 to 44 returned by AggregationCacheMemSizeTest
-        return 44;
-        /*
-        return 8 // aggregator obj shell
-        + 8 // ref to DoubleWritable
-        + 8 // DoubleWritable obj shell
-        + 8; // size of double
-        */
-    }
-
-    public static int guessLongMemBytes() {
-        // 29 to 44 returned by AggregationCacheMemSizeTest
-        return 44;
-        /*
-        return 8 // aggregator obj shell
-        + 8 // ref to LongWritable
-        + 8 // LongWritable obj shell
-        + 8; // size of long
-        */
-    }
-
-    // ============================================================================
-
-    @SuppressWarnings("rawtypes")
-    public void setDependentAggregator(MeasureAggregator agg) {
-    }
-
-    abstract public void reset();
-
-    abstract public void aggregate(V value);
-
-    abstract public V getState();
-
-    // get an estimate of memory consumption UPPER BOUND
-    abstract public int getMemBytesEstimate();
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregators.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregators.java b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregators.java
deleted file mode 100644
index 3aa575b..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregators.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.aggregation;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-/**
- */
-@SuppressWarnings({ "rawtypes", "unchecked", "serial" })
-public class MeasureAggregators implements Serializable {
-
-    private final MeasureAggregator[] aggs;
-    private final int descLength;
-
-    public MeasureAggregators(Collection<MeasureDesc> measureDescs) {
-        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
-    }
-
-    public MeasureAggregators(MeasureDesc... measureDescs) {
-        descLength = measureDescs.length;
-        aggs = new MeasureAggregator[descLength];
-
-        Map<String, Integer> measureIndexMap = new HashMap<String, Integer>();
-        for (int i = 0; i < descLength; i++) {
-            FunctionDesc func = measureDescs[i].getFunction();
-            aggs[i] = MeasureAggregator.create(func.getExpression(), func.getReturnType());
-            measureIndexMap.put(measureDescs[i].getName(), i);
-        }
-        // fill back dependent aggregator
-        for (int i = 0; i < descLength; i++) {
-            String depMsrRef = measureDescs[i].getDependentMeasureRef();
-            if (depMsrRef != null) {
-                int index = measureIndexMap.get(depMsrRef);
-                aggs[i].setDependentAggregator(aggs[index]);
-            }
-        }
-    }
-
-    public void reset() {
-        for (int i = 0; i < aggs.length; i++) {
-            aggs[i].reset();
-        }
-    }
-    
-    public void aggregate(Object[] values) {
-        assert values.length == descLength;
-
-        for (int i = 0; i < descLength; i++) {
-            aggs[i].aggregate(values[i]);
-        }
-    }
-
-    public void collectStates(Object[] states) {
-        for (int i = 0; i < descLength; i++) {
-            states[i] = aggs[i].getState();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java
deleted file mode 100644
index 8721954..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.aggregation;
-
-import java.nio.ByteBuffer;
-import java.util.Collection;
-
-import org.apache.kylin.common.datatype.DataTypeSerializer;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-/**
- * @author yangli9
- * 
- */
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class MeasureCodec {
-
-    int nMeasures;
-    DataTypeSerializer[] serializers;
-
-    public MeasureCodec(Collection<MeasureDesc> measureDescs) {
-        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
-    }
-
-    public MeasureCodec(MeasureDesc... measureDescs) {
-        String[] dataTypes = new String[measureDescs.length];
-        for (int i = 0; i < dataTypes.length; i++) {
-            dataTypes[i] = measureDescs[i].getFunction().getReturnType();
-        }
-        init(dataTypes);
-    }
-
-    public MeasureCodec(String... dataTypes) {
-        init(dataTypes);
-    }
-
-    private void init(String[] dataTypes) {
-        nMeasures = dataTypes.length;
-        serializers = new DataTypeSerializer[nMeasures];
-
-        for (int i = 0; i < nMeasures; i++) {
-            serializers[i] = DataTypeSerializer.create(dataTypes[i]);
-        }
-    }
-
-    public DataTypeSerializer getSerializer(int idx) {
-        return serializers[idx];
-    }
-
-    public void decode(ByteBuffer buf, Object[] result) {
-        assert result.length == nMeasures;
-        for (int i = 0; i < nMeasures; i++) {
-            result[i] = serializers[i].deserialize(buf);
-        }
-    }
-
-    public void encode(Object[] values, ByteBuffer out) {
-        assert values.length == nMeasures;
-        for (int i = 0; i < nMeasures; i++) {
-            serializers[i].serialize(values[i], out);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregation.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregation.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregation.java
deleted file mode 100644
index 7ea88bd..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregation.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.aggregation.basic;
-
-import java.util.List;
-
-import org.apache.kylin.aggregation.AggregationType;
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.BigDecimalSerializer;
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.DataTypeSerializer;
-import org.apache.kylin.common.datatype.DateTimeSerializer;
-import org.apache.kylin.common.datatype.DoubleSerializer;
-import org.apache.kylin.common.datatype.LongSerializer;
-import org.apache.kylin.common.datatype.StringSerializer;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-
-public class BasicAggregation extends AggregationType {
-    
-    private final String funcName;
-    private final DataType dataType;
-
-    public BasicAggregation(String funcName, String dataType) {
-        this.funcName = funcName;
-        this.dataType = DataType.getType(dataType);
-    }
-
-    @Override
-    public DataType getAggregationDataType() {
-        return dataType;
-    }
-
-    public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer() {
-        if (dataType.isStringFamily())
-            return StringSerializer.class;
-        else if (dataType.isIntegerFamily())
-            return LongSerializer.class;
-        else if (dataType.isDecimal())
-            return BigDecimalSerializer.class;
-        else if (dataType.isNumberFamily())
-            return DoubleSerializer.class;
-        else if (dataType.isDateTimeFamily())
-            return DateTimeSerializer.class;
-        else
-            throw new IllegalArgumentException("No default serializer for type " + dataType);
-    }
-    
-    @Override
-    public void validate(MeasureDesc measureDesc) throws IllegalArgumentException {
-        // TODO Auto-generated method stub
-        
-    }
-
-    @Override
-    public MeasureAggregator<?> newAggregator() {
-        if (isSum() || isCount()) {
-            if (dataType.isDecimal())
-                return new BigDecimalSumAggregator();
-            else if (dataType.isIntegerFamily())
-                return new LongSumAggregator();
-            else if (dataType.isNumberFamily())
-                return new DoubleSumAggregator();
-        } else if (isMax()) {
-            if (dataType.isDecimal())
-                return new BigDecimalMaxAggregator();
-            else if (dataType.isIntegerFamily())
-                return new LongMaxAggregator();
-            else if (dataType.isNumberFamily())
-                return new DoubleMaxAggregator();
-        } else if (isMin()) {
-            if (dataType.isDecimal())
-                return new BigDecimalMinAggregator();
-            else if (dataType.isIntegerFamily())
-                return new LongMinAggregator();
-            else if (dataType.isNumberFamily())
-                return new DoubleMinAggregator();
-        }
-        throw new IllegalArgumentException("No aggregator for func '" + funcName + "' and return type '" + dataType + "'");
-    }
-    
-    private boolean isSum() {
-        return FunctionDesc.FUNC_SUM.equalsIgnoreCase(funcName);
-    }
-
-    private boolean isCount() {
-        return FunctionDesc.FUNC_COUNT.equalsIgnoreCase(funcName);
-    }
-    
-    private boolean isMax() {
-        return FunctionDesc.FUNC_MAX.equalsIgnoreCase(funcName);
-    }
-    
-    private boolean isMin() {
-        return FunctionDesc.FUNC_MIN.equalsIgnoreCase(funcName);
-    }
-    
-    @Override
-    public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregationFactory.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregationFactory.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregationFactory.java
deleted file mode 100644
index 547d45f..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregationFactory.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.aggregation.basic;
-
-import org.apache.kylin.aggregation.AggregationType;
-import org.apache.kylin.aggregation.IAggregationFactory;
-
-public class BasicAggregationFactory implements IAggregationFactory {
-
-    @Override
-    public AggregationType createAggregationType(String funcName, String dataType) {
-        return new BasicAggregation(funcName, dataType);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMaxAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMaxAggregator.java
deleted file mode 100644
index ca044d0..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMaxAggregator.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.aggregation.basic;
-
-import java.math.BigDecimal;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-
-/**
- */
-@SuppressWarnings("serial")
-public class BigDecimalMaxAggregator extends MeasureAggregator<BigDecimal> {
-
-    BigDecimal max = null;
-
-    @Override
-    public void reset() {
-        max = null;
-    }
-
-    @Override
-    public void aggregate(BigDecimal value) {
-        if (max == null)
-            max = value;
-        else if (max.compareTo(value) < 0)
-            max = value;
-    }
-
-    @Override
-    public BigDecimal getState() {
-        return max;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessBigDecimalMemBytes();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMinAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMinAggregator.java
deleted file mode 100644
index 3c3c85e..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMinAggregator.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.aggregation.basic;
-
-import java.math.BigDecimal;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-
-/**
- */
-@SuppressWarnings("serial")
-public class BigDecimalMinAggregator extends MeasureAggregator<BigDecimal> {
-
-    BigDecimal max = null;
-
-    @Override
-    public void reset() {
-        max = null;
-    }
-
-    @Override
-    public void aggregate(BigDecimal value) {
-        if (max == null)
-            max = value;
-        else if (max.compareTo(value) > 0)
-            max = value;
-    }
-
-    @Override
-    public BigDecimal getState() {
-        return max;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessBigDecimalMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSumAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSumAggregator.java
deleted file mode 100644
index 19aef3c..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSumAggregator.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.aggregation.basic;
-
-import java.math.BigDecimal;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-
-/**
- */
-@SuppressWarnings("serial")
-public class BigDecimalSumAggregator extends MeasureAggregator<BigDecimal> {
-
-    BigDecimal sum = new BigDecimal(0);
-
-    @Override
-    public void reset() {
-        sum = new BigDecimal(0);
-    }
-
-    @Override
-    public void aggregate(BigDecimal value) {
-        sum = sum.add(value);
-    }
-
-    @Override
-    public BigDecimal getState() {
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessBigDecimalMemBytes();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java
deleted file mode 100644
index 99896a6..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.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.aggregation.basic;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.DoubleMutable;
-
-/**
- */
-@SuppressWarnings("serial")
-public class DoubleMaxAggregator extends MeasureAggregator<DoubleMutable> {
-
-    DoubleMutable max = null;
-
-    @Override
-    public void reset() {
-        max = null;
-    }
-
-    @Override
-    public void aggregate(DoubleMutable value) {
-        if (max == null)
-            max = new DoubleMutable(value.get());
-        else if (max.get() < value.get())
-            max.set(value.get());
-    }
-
-    @Override
-    public DoubleMutable getState() {
-        return max;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessDoubleMemBytes();
-    }
-
-}



[09/13] incubator-kylin git commit: KYLIN-976 Add ingester; Build part done, in-mem cube test pass

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregator.java
new file mode 100644
index 0000000..16563fa
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.measure.hllc;
+
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.measure.MeasureAggregator;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class HLLCAggregator extends MeasureAggregator<HyperLogLogPlusCounter> {
+
+    final int precision;
+    HyperLogLogPlusCounter sum = null;
+
+    public HLLCAggregator(int precision) {
+        this.precision = precision;
+    }
+
+    @Override
+    public void reset() {
+        sum = null;
+    }
+
+    @Override
+    public void aggregate(HyperLogLogPlusCounter value) {
+        if (sum == null)
+            sum = new HyperLogLogPlusCounter(value);
+        else
+            sum.merge(value);
+    }
+
+    @Override
+    public HyperLogLogPlusCounter getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        // 1024 + 60 returned by AggregationCacheMemSizeTest
+        return 8 // aggregator obj shell
+                + 4 // precision
+                + 8 // ref to HLLC
+                + 8 // HLLC obj shell
+                + 32 + (1 << precision); // HLLC internal
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
new file mode 100644
index 0000000..ee90818
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.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.measure.hllc;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class HLLCMeasureType extends MeasureType {
+
+    private final DataType dataType;
+
+    public HLLCMeasureType(DataType dataType) {
+        if ("hllc".equals(dataType.getName()) == false)
+            throw new IllegalArgumentException();
+        
+        this.dataType = dataType;
+
+        if (this.dataType.getPrecision() < 10 || this.dataType.getPrecision() > 16)
+            throw new IllegalArgumentException("HLLC precision must be between 10 and 16");
+    }
+
+    @Override
+    public DataType getAggregationDataType() {
+        return dataType;
+    }
+
+    @Override
+    public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer() {
+        return HLLCSerializer.class;
+    }
+
+    @Override
+    public void validate(MeasureDesc measureDesc) throws IllegalArgumentException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public MeasureIngester<?> newIngester() {
+        return new MeasureIngester<HyperLogLogPlusCounter>() {
+            HyperLogLogPlusCounter current = new HyperLogLogPlusCounter(dataType.getPrecision());
+
+            @Override
+            public HyperLogLogPlusCounter valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+                HyperLogLogPlusCounter hllc = current;
+                hllc.clear();
+                for (String v : values)
+                    hllc.add(v == null ? "__nUlL__" : v);
+                return hllc;
+            }
+        };
+    }
+
+    @Override
+    public MeasureAggregator<?> newAggregator() {
+        if (dataType.isHLLC())
+            return new HLLCAggregator(dataType.getPrecision());
+        else
+            return new LDCAggregator();
+    }
+
+    @Override
+    public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCSerializer.java
new file mode 100644
index 0000000..7131201
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCSerializer.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.measure.hllc;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class HLLCSerializer extends DataTypeSerializer<HyperLogLogPlusCounter> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<HyperLogLogPlusCounter> current = new ThreadLocal<HyperLogLogPlusCounter>();
+
+    private int precision;
+
+    public HLLCSerializer(DataType type) {
+        this.precision = type.getPrecision();
+    }
+
+    @Override
+    public void serialize(HyperLogLogPlusCounter value, ByteBuffer out) {
+        try {
+            value.writeRegisters(out);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private HyperLogLogPlusCounter current() {
+        HyperLogLogPlusCounter hllc = current.get();
+        if (hllc == null) {
+            hllc = new HyperLogLogPlusCounter(precision);
+            current.set(hllc);
+        }
+        return hllc;
+    }
+
+    @Override
+    public HyperLogLogPlusCounter deserialize(ByteBuffer in) {
+        HyperLogLogPlusCounter hllc = current();
+        try {
+            hllc.readRegisters(in);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return hllc;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return current().peekLength(in);
+    }
+
+    @Override
+    public int maxLength() {
+        return current().maxLength();
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return current().maxLength();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/LDCAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/LDCAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/LDCAggregator.java
new file mode 100644
index 0000000..5d96450
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/LDCAggregator.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.measure.hllc;
+
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.LongMutable;
+
+/**
+ * Long Distinct Count
+ */
+@SuppressWarnings("serial")
+public class LDCAggregator extends MeasureAggregator<LongMutable> {
+
+    private static LongMutable ZERO = new LongMutable(0);
+
+    private HLLCAggregator hllAgg = null;
+    private LongMutable state = new LongMutable(0);
+
+    @SuppressWarnings("rawtypes")
+    public void setDependentAggregator(MeasureAggregator agg) {
+        this.hllAgg = (HLLCAggregator) agg;
+    }
+
+    @Override
+    public void reset() {
+    }
+
+    @Override
+    public void aggregate(LongMutable value) {
+    }
+
+    @Override
+    public LongMutable getState() {
+        if (hllAgg == null) {
+            return ZERO;
+        } else {
+            state.set(hllAgg.getState().getCountEstimate());
+            return state;
+        }
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java
new file mode 100644
index 0000000..9b4c893
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.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.measure.topn;
+
+import java.util.Map;
+
+import org.apache.kylin.common.topn.TopNCounter;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.measure.MeasureAggregator;
+
+import com.google.common.collect.Maps;
+
+/**
+ * 
+ */
+@SuppressWarnings("serial")
+public class TopNAggregator extends MeasureAggregator<TopNCounter<ByteArray>> {
+
+    int capacity = 0;
+    TopNCounter<ByteArray> sum = null;
+    Map<ByteArray, Double> sanityCheckMap;
+
+    @Override
+    public void reset() {
+        sum = null;
+    }
+
+    @Override
+    public void aggregate(TopNCounter<ByteArray> value) {
+        if (sum == null) {
+            capacity = value.getCapacity();
+            sum = new TopNCounter<>(capacity);
+            sanityCheckMap = Maps.newHashMap();
+        }
+        sum.merge(value);
+    }
+
+    @Override
+    public TopNCounter<ByteArray> getState() {
+        
+        //sum.retain(capacity);
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return 8 * capacity / 4;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounterSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounterSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounterSerializer.java
new file mode 100644
index 0000000..b422316
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounterSerializer.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.measure.topn;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.kylin.common.topn.Counter;
+import org.apache.kylin.common.topn.DoubleDeltaSerializer;
+import org.apache.kylin.common.topn.TopNCounter;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+
+/**
+ * 
+ */
+public class TopNCounterSerializer extends DataTypeSerializer<TopNCounter<ByteArray>> {
+
+    private DoubleDeltaSerializer dds = new DoubleDeltaSerializer(3);
+
+    private int precision;
+
+    public TopNCounterSerializer(DataType dataType) {
+        this.precision = dataType.getPrecision();
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+        @SuppressWarnings("unused")
+        int capacity = in.getInt();
+        int size = in.getInt();
+        int keyLength = in.getInt();
+        dds.deserialize(in);
+        int len = in.position() - mark + keyLength * size;
+        in.position(mark);
+        return len;
+    }
+
+    @Override
+    public int maxLength() {
+        return precision * TopNCounter.EXTRA_SPACE_RATE * (4 + 8);
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return precision * TopNCounter.EXTRA_SPACE_RATE * 8;
+    }
+
+    @Override
+    public void serialize(TopNCounter<ByteArray> value, ByteBuffer out) {
+        double[] counters = value.getCounters();
+        List<ByteArray> peek = value.peek(1);
+        int keyLength = peek.size() > 0 ? peek.get(0).length() : 0;
+        out.putInt(value.getCapacity());
+        out.putInt(value.size());
+        out.putInt(keyLength);
+        dds.serialize(counters, out);
+        Iterator<Counter<ByteArray>> iterator = value.iterator();
+        while (iterator.hasNext()) {
+            out.put(iterator.next().getItem().array());
+        }
+    }
+
+    @Override
+    public TopNCounter<ByteArray> deserialize(ByteBuffer in) {
+        int capacity = in.getInt();
+        int size = in.getInt();
+        int keyLength = in.getInt();
+        double[] counters = dds.deserialize(in);
+
+        TopNCounter<ByteArray> counter = new TopNCounter<ByteArray>(capacity);
+        ByteArray byteArray;
+        for (int i = 0; i < size; i++) {
+            byteArray = new ByteArray(keyLength);
+            in.get(byteArray.array());
+            counter.offerToHead(byteArray, counters[i]);
+        }
+
+        return counter;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureFactory.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureFactory.java
new file mode 100644
index 0000000..1ceb607
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureFactory.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.measure.topn;
+
+import org.apache.kylin.measure.IMeasureFactory;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.model.FunctionDesc;
+
+public class TopNMeasureFactory implements IMeasureFactory {
+
+    @Override
+    public MeasureType createMeasureType(String funcName, String dataType) {
+        if (FunctionDesc.FUNC_TOP_N.equalsIgnoreCase(funcName) == false)
+            throw new IllegalArgumentException();
+        
+        return new TopNMeasureType(DataType.getType(dataType));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
new file mode 100644
index 0000000..1d2c87b
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.measure.topn;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.common.topn.TopNCounter;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.measure.hllc.HLLCSerializer;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class TopNMeasureType extends MeasureType {
+
+    private final DataType dataType;
+
+    public TopNMeasureType(DataType dataType) {
+        if ("topn".equals(dataType.getName()) == false)
+            throw new IllegalArgumentException();
+        
+        this.dataType = dataType;
+        
+        if (this.dataType.getPrecision() < 1 || this.dataType.getPrecision() > 1000)
+            throw new IllegalArgumentException("TopN precision must be between 1 and 1000");
+    }
+
+    @Override
+    public DataType getAggregationDataType() {
+        return dataType;
+    }
+
+    @Override
+    public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer() {
+        return HLLCSerializer.class;
+    }
+    
+    @Override
+    public void validate(MeasureDesc measureDesc) throws IllegalArgumentException {
+        // TODO Auto-generated method stub
+        
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    public MeasureIngester<?> newIngester() {
+        return new MeasureIngester<TopNCounter>() {
+            @Override
+            public TopNCounter valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+                if (values.length != 2)
+                    throw new IllegalArgumentException();
+                
+                double counter = values[0] == null ? 0 : Double.parseDouble(values[0]);
+                String literal = values[1];
+                
+                // encode literal using dictionary
+                TblColRef literalCol = measureDesc.getFunction().getTopNLiteralColumn();
+                Dictionary<String> dictionary = dictionaryMap.get(literalCol);
+                int keyEncodedValue = dictionary.getIdFromValue(literal);
+
+                ByteArray key = new ByteArray(dictionary.getSizeOfId());
+                BytesUtil.writeUnsigned(keyEncodedValue, key.array(), 0, dictionary.getSizeOfId());
+
+                TopNCounter<ByteArray> topNCounter = new TopNCounter<ByteArray>(dataType.getPrecision() * TopNCounter.EXTRA_SPACE_RATE);
+                topNCounter.offer(key, counter);
+                return topNCounter;
+            }
+        };
+    }
+
+    @Override
+    public MeasureAggregator<?> newAggregator() {
+        return new TopNAggregator();
+    }
+
+    @Override
+    public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
new file mode 100644
index 0000000..134f305
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.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.metadata.datatype;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.BytesUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class BigDecimalSerializer extends DataTypeSerializer<BigDecimal> {
+
+    private static final Logger logger = LoggerFactory.getLogger(BigDecimalSerializer.class);
+
+    final DataType type;
+    final int maxLength;
+
+    int avoidVerbose = 0;
+
+    public BigDecimalSerializer(DataType type) {
+        this.type = type;
+        // see serialize(): 1 byte scale, 1 byte length, assume every 2 digits takes 1 byte
+        this.maxLength = 1 + 1 + (type.getPrecision() + 1) / 2;
+    }
+
+    @Override
+    public void serialize(BigDecimal value, ByteBuffer out) {
+        if (value.scale() > type.getScale()) {
+            if (avoidVerbose % 10000 == 0) {
+                logger.warn("value's scale has exceeded the " + type.getScale() + ", cut it off, to ensure encoded value do not exceed maxLength " + maxLength + " times:" + (avoidVerbose++));
+            }
+            value = value.setScale(type.getScale(), BigDecimal.ROUND_HALF_EVEN);
+        }
+        byte[] bytes = value.unscaledValue().toByteArray();
+        if (bytes.length + 2 > maxLength) {
+            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
+        }
+
+        BytesUtil.writeVInt(value.scale(), out);
+        BytesUtil.writeVInt(bytes.length, out);
+        out.put(bytes);
+    }
+
+    @Override
+    public BigDecimal deserialize(ByteBuffer in) {
+        int scale = BytesUtil.readVInt(in);
+        int n = BytesUtil.readVInt(in);
+
+        byte[] bytes = new byte[n];
+        in.get(bytes);
+
+        return new BigDecimal(new BigInteger(bytes), scale);
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+
+        @SuppressWarnings("unused")
+        int scale = BytesUtil.readVInt(in);
+        int n = BytesUtil.readVInt(in);
+        int len = in.position() - mark + n;
+
+        in.position(mark);
+        return len;
+    }
+
+    @Override
+    public int maxLength() {
+        return maxLength;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataType.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataType.java
new file mode 100644
index 0000000..235c99f
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataType.java
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metadata.datatype;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class DataType implements Serializable {
+
+    // standard sql types, ref: http://www.w3schools.com/sql/sql_datatypes_general.asp
+    public static final String VALID_TYPES_STRING = "any|char|varchar|boolean|binary" //
+            + "|integer|tinyint|smallint|bigint|decimal|numeric|float|real|double" //
+            + "|date|time|datetime|timestamp|byte|int|short|long|string|hllc|topn";
+
+    private static final String TYPE_PATTEN_TAIL = "\\s*" //
+            + "(?:" + "[(]" + "([\\d\\s,]+)" + "[)]" + ")?";
+
+    private static final Pattern TYPE_PATTERN = Pattern.compile( //
+            "(" + VALID_TYPES_STRING + ")" + TYPE_PATTEN_TAIL, Pattern.CASE_INSENSITIVE);
+
+    public static final Set<String> INTEGER_FAMILY = new HashSet<String>();
+    public static final Set<String> NUMBER_FAMILY = new HashSet<String>();
+    public static final Set<String> DATETIME_FAMILY = new HashSet<String>();
+    public static final Set<String> STRING_FAMILY = new HashSet<String>();
+    private static final Map<String, String> LEGACY_TYPE_MAP = new HashMap<String, String>();
+    static {
+        INTEGER_FAMILY.add("tinyint");
+        INTEGER_FAMILY.add("smallint");
+        INTEGER_FAMILY.add("integer");
+        INTEGER_FAMILY.add("bigint");
+
+        NUMBER_FAMILY.addAll(INTEGER_FAMILY);
+        NUMBER_FAMILY.add("float");
+        NUMBER_FAMILY.add("double");
+        NUMBER_FAMILY.add("decimal");
+        NUMBER_FAMILY.add("real");
+        NUMBER_FAMILY.add("numeric");
+
+        DATETIME_FAMILY.add("date");
+        DATETIME_FAMILY.add("time");
+        DATETIME_FAMILY.add("datetime");
+        DATETIME_FAMILY.add("timestamp");
+
+        STRING_FAMILY.add("varchar");
+        STRING_FAMILY.add("char");
+
+        LEGACY_TYPE_MAP.put("byte", "tinyint");
+        LEGACY_TYPE_MAP.put("int", "integer");
+        LEGACY_TYPE_MAP.put("short", "smallint");
+        LEGACY_TYPE_MAP.put("long", "bigint");
+        LEGACY_TYPE_MAP.put("string", "varchar");
+        LEGACY_TYPE_MAP.put("hllc10", "hllc(10)");
+        LEGACY_TYPE_MAP.put("hllc12", "hllc(12)");
+        LEGACY_TYPE_MAP.put("hllc14", "hllc(14)");
+        LEGACY_TYPE_MAP.put("hllc15", "hllc(15)");
+        LEGACY_TYPE_MAP.put("hllc16", "hllc(16)");
+    }
+
+    private static final ConcurrentMap<DataType, DataType> CACHE = new ConcurrentHashMap<DataType, DataType>();
+
+    public static final DataType ANY = DataType.getType("any");
+
+    public static DataType getType(String type) {
+        if (type == null)
+            return null;
+
+        DataType dataType = new DataType(type);
+        DataType cached = CACHE.get(dataType);
+        if (cached == null) {
+            CACHE.put(dataType, dataType);
+            cached = dataType;
+        }
+        return cached;
+    }
+
+    // ============================================================================
+
+    private String name;
+    private int precision;
+    private int scale;
+
+    private DataType(String datatype) {
+        datatype = datatype.trim().toLowerCase();
+        datatype = replaceLegacy(datatype);
+
+        Pattern pattern = TYPE_PATTERN;
+        Matcher m = pattern.matcher(datatype);
+        if (m.matches() == false)
+            throw new IllegalArgumentException("bad data type -- " + datatype + ", does not match " + pattern);
+
+        name = replaceLegacy(m.group(1));
+        precision = -1;
+        scale = -1;
+
+        String leftover = m.group(2);
+        if (leftover != null) {
+            String[] parts = leftover.split("\\s*,\\s*");
+            for (int i = 0; i < parts.length; i++) {
+                int n;
+                try {
+                    n = Integer.parseInt(parts[i]);
+                } catch (NumberFormatException e) {
+                    throw new IllegalArgumentException("bad data type -- " + datatype + ", precision/scale not numeric");
+                }
+                if (i == 0)
+                    precision = n;
+                else if (i == 1)
+                    scale = n;
+                else
+                    throw new IllegalArgumentException("bad data type -- " + datatype + ", too many precision/scale parts");
+            }
+        }
+
+        // FIXME 256 for unknown string precision
+        if ((name.equals("char") || name.equals("varchar")) && precision == -1) {
+            precision = 256; // to save memory at frontend, e.g. tableau will
+                             // allocate memory according to this
+        }
+
+        // FIXME (19,4) for unknown decimal precision
+        if ((name.equals("decimal") || name.equals("numeric")) && precision == -1) {
+            precision = 19;
+            scale = 4;
+        }
+    }
+
+    private String replaceLegacy(String str) {
+        String replace = LEGACY_TYPE_MAP.get(str);
+        return replace == null ? str : replace;
+    }
+
+    public int getStorageBytesEstimate() {
+        return DataTypeSerializer.create(this).getStorageBytesEstimate();
+    }
+
+    public boolean isStringFamily() {
+        return STRING_FAMILY.contains(name);
+    }
+
+    public boolean isIntegerFamily() {
+        return INTEGER_FAMILY.contains(name);
+    }
+
+    public boolean isNumberFamily() {
+        return NUMBER_FAMILY.contains(name);
+    }
+
+    public boolean isDateTimeFamily() {
+        return DATETIME_FAMILY.contains(name);
+    }
+
+    public boolean isDate() {
+        return name.equals("date");
+    }
+
+    public boolean isTime() {
+        return name.equals("time");
+    }
+
+    public boolean isTimestamp() {
+        return name.equals("timestamp");
+    }
+
+    public boolean isDatetime() {
+        return name.equals("datetime");
+    }
+
+    public boolean isTinyInt() {
+        return name.equals("tinyint");
+    }
+
+    public boolean isSmallInt() {
+        return name.equals("smallint");
+    }
+
+    public boolean isInt() {
+        return name.equals("integer");
+    }
+
+    public boolean isBigInt() {
+        return name.equals("bigint");
+    }
+
+    public boolean isFloat() {
+        return name.equals("float");
+    }
+
+    public boolean isDouble() {
+        return name.equals("double");
+    }
+
+    public boolean isDecimal() {
+        return name.equals("decimal");
+    }
+
+    public boolean isHLLC() {
+        return name.equals("hllc");
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public int getPrecision() {
+        return precision;
+    }
+
+    public int getScale() {
+        return scale;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        result = prime * result + precision;
+        result = prime * result + scale;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        DataType other = (DataType) obj;
+        if (name == null) {
+            if (other.name != null)
+                return false;
+        } else if (!name.equals(other.name))
+            return false;
+        if (precision != other.precision)
+            return false;
+        if (scale != other.scale)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        if (precision < 0 && scale < 0)
+            return name;
+        else if (scale < 0)
+            return name + "(" + precision + ")";
+        else
+            return name + "(" + precision + "," + scale + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java
new file mode 100644
index 0000000..fd3121f
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.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.metadata.datatype;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kylin.common.util.BytesSerializer;
+import org.apache.kylin.measure.hllc.HLLCSerializer;
+import org.apache.kylin.measure.topn.TopNCounterSerializer;
+
+import com.google.common.collect.Maps;
+
+/**
+ * @author yangli9
+ * 
+ * Note: the implementations MUST be thread-safe.
+ */
+abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
+
+    final static Map<String, Class<?>> implementations;
+    static {
+        HashMap<String, Class<?>> impl = Maps.newHashMap();
+        impl.put("varchar", StringSerializer.class);
+        impl.put("decimal", BigDecimalSerializer.class);
+        impl.put("double", DoubleSerializer.class);
+        impl.put("float", DoubleSerializer.class);
+        impl.put("bigint", LongSerializer.class);
+        impl.put("long", LongSerializer.class);
+        impl.put("integer", LongSerializer.class);
+        impl.put("int", LongSerializer.class);
+        impl.put("smallint", LongSerializer.class);
+        impl.put("date", DateTimeSerializer.class);
+        impl.put("datetime", DateTimeSerializer.class);
+        impl.put("timestamp", DateTimeSerializer.class);
+        impl.put("topn", TopNCounterSerializer.class);
+        impl.put("hllc", HLLCSerializer.class);
+        implementations = Collections.unmodifiableMap(impl);
+    }
+    
+    public static boolean hasRegistered(String dataTypeName) {
+        return implementations.containsKey(dataTypeName);
+    }
+    
+    public static void register(String dataTypeName, Class<? extends DataTypeSerializer<?>> impl) {
+        implementations.put(dataTypeName, impl);
+    }
+
+    public static DataTypeSerializer<?> create(String dataType) {
+        return create(DataType.getType(dataType));
+    }
+
+    public static DataTypeSerializer<?> create(DataType type) {
+        Class<?> clz = implementations.get(type.getName());
+        if (clz == null)
+            throw new RuntimeException("No DataTypeSerializer for type " + type);
+
+        try {
+            return (DataTypeSerializer<?>) clz.getConstructor(DataType.class).newInstance(type);
+        } catch (Exception e) {
+            throw new RuntimeException(e); // never happen
+        }
+    }
+    
+    /** peek into buffer and return the length of serialization */
+    abstract public int peekLength(ByteBuffer in);
+
+    /** return the max number of bytes to the longest serialization */
+    abstract public int maxLength();
+
+    /** get an estimate of size in bytes of the serialized data */
+    abstract public int getStorageBytesEstimate();
+
+    /** convert from obj to string */
+    public String toString(T value) {
+        if (value == null)
+            return "NULL";
+        else
+            return value.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DateTimeSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DateTimeSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DateTimeSerializer.java
new file mode 100644
index 0000000..4f7935c
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DateTimeSerializer.java
@@ -0,0 +1,49 @@
+package org.apache.kylin.metadata.datatype;
+
+import java.nio.ByteBuffer;
+
+public class DateTimeSerializer extends DataTypeSerializer<LongMutable> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+
+    public DateTimeSerializer(DataType type) {
+    }
+
+    @Override
+    public void serialize(LongMutable value, ByteBuffer out) {
+        out.putLong(value.get());
+    }
+
+    private LongMutable current() {
+        LongMutable l = current.get();
+        if (l == null) {
+            l = new LongMutable();
+            current.set(l);
+        }
+        return l;
+    }
+
+    @Override
+    public LongMutable deserialize(ByteBuffer in) {
+        LongMutable l = current();
+        l.set(in.getLong());
+        return l;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return 8;
+    }
+
+    @Override
+    public int maxLength() {
+        return 8;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleMutable.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleMutable.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleMutable.java
new file mode 100644
index 0000000..5bd75e8
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleMutable.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metadata.datatype;
+
+import java.io.Serializable;
+
+@SuppressWarnings("serial")
+public class DoubleMutable implements Comparable<DoubleMutable>, Serializable {
+
+    private double v;
+
+    public DoubleMutable() {
+        this(0);
+    }
+
+    public DoubleMutable(double v) {
+        set(v);
+    }
+
+    public double get() {
+        return v;
+    }
+
+    public void set(double v) {
+        this.v = v;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof DoubleMutable)) {
+            return false;
+        }
+        DoubleMutable other = (DoubleMutable) o;
+        return this.v == other.v;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) Double.doubleToLongBits(v);
+    }
+
+    @Override
+    public int compareTo(DoubleMutable o) {
+        return (v < o.v ? -1 : (v == o.v ? 0 : 1));
+    }
+
+    @Override
+    public String toString() {
+        return Double.toString(v);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleSerializer.java
new file mode 100644
index 0000000..5accf1d
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleSerializer.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.metadata.datatype;
+
+import java.nio.ByteBuffer;
+
+/**
+ */
+public class DoubleSerializer extends DataTypeSerializer<DoubleMutable> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<DoubleMutable> current = new ThreadLocal<DoubleMutable>();
+
+    public DoubleSerializer(DataType type) {
+    }
+
+    @Override
+    public void serialize(DoubleMutable value, ByteBuffer out) {
+        out.putDouble(value.get());
+    }
+
+    private DoubleMutable current() {
+        DoubleMutable d = current.get();
+        if (d == null) {
+            d = new DoubleMutable();
+            current.set(d);
+        }
+        return d;
+    }
+
+    @Override
+    public DoubleMutable deserialize(ByteBuffer in) {
+        DoubleMutable d = current();
+        d.set(in.getDouble());
+        return d;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return 8;
+    }
+
+    @Override
+    public int maxLength() {
+        return 8;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongMutable.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongMutable.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongMutable.java
new file mode 100644
index 0000000..b978049
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongMutable.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metadata.datatype;
+
+import java.io.Serializable;
+
+@SuppressWarnings("serial")
+public class LongMutable implements Comparable<LongMutable>, Serializable {
+
+    private long v;
+
+    public LongMutable() {
+        this(0);
+    }
+
+    public LongMutable(long v) {
+        set(v);
+    }
+
+    public long get() {
+        return v;
+    }
+
+    public void set(long v) {
+        this.v = v;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof LongMutable)) {
+            return false;
+        }
+        LongMutable other = (LongMutable) o;
+        return this.v == other.v;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) v;
+    }
+
+    @Override
+    public int compareTo(LongMutable o) {
+        long thisValue = this.v;
+        long thatValue = o.v;
+        return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
+    }
+
+    @Override
+    public String toString() {
+        return Long.toString(v);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongSerializer.java
new file mode 100644
index 0000000..777f494
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongSerializer.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metadata.datatype;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.BytesUtil;
+
+/**
+ */
+public class LongSerializer extends DataTypeSerializer<LongMutable> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+
+    public LongSerializer(DataType type) {
+    }
+
+    @Override
+    public void serialize(LongMutable value, ByteBuffer out) {
+        BytesUtil.writeVLong(value.get(), out);
+    }
+
+    private LongMutable current() {
+        LongMutable l = current.get();
+        if (l == null) {
+            l = new LongMutable();
+            current.set(l);
+        }
+        return l;
+    }
+
+    @Override
+    public LongMutable deserialize(ByteBuffer in) {
+        LongMutable l = current();
+        l.set(BytesUtil.readVLong(in));
+        return l;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+
+        BytesUtil.readVLong(in);
+        int len = in.position() - mark;
+
+        in.position(mark);
+        return len;
+    }
+
+    @Override
+    public int maxLength() {
+        return 9; // vlong: 1 + 8
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 5;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/StringSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/StringSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/StringSerializer.java
new file mode 100644
index 0000000..14c8909
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/StringSerializer.java
@@ -0,0 +1,48 @@
+package org.apache.kylin.metadata.datatype;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.BytesUtil;
+
+public class StringSerializer extends DataTypeSerializer<String> {
+
+    final DataType type;
+    final int maxLength;
+
+    public StringSerializer(DataType type) {
+        this.type = type;
+        // see serialize(): 2 byte length, rest is String.toBytes()
+        this.maxLength = 2 + type.getPrecision();
+    }
+
+    @Override
+    public void serialize(String value, ByteBuffer out) {
+        int start = out.position();
+
+        BytesUtil.writeUTFString(value, out);
+
+        if (out.position() - start > maxLength)
+            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
+    }
+
+    @Override
+    public String deserialize(ByteBuffer in) {
+        return BytesUtil.readUTFString(in);
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return BytesUtil.peekByteArrayLength(in);
+    }
+
+    @Override
+    public int maxLength() {
+        return maxLength;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return maxLength;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java
index 0e687f7..b894fd3 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java
@@ -5,8 +5,8 @@ import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.google.common.collect.Maps;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
index f03e736..2ddc75a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
@@ -21,7 +21,7 @@ package org.apache.kylin.metadata.model;
 import java.io.Serializable;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataType;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
index d22d0a3..0c36873 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
@@ -21,7 +21,7 @@ package org.apache.kylin.metadata.model;
 import java.util.ArrayList;
 import java.util.Collection;
 
-import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataType;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
@@ -171,6 +171,14 @@ public class FunctionDesc {
         this.parameter = parameter;
     }
 
+    public int getParameterCount() {
+        int count = 0;
+        for (ParameterDesc p = parameter; p != null; p = p.getNextParameter()) {
+            count++;
+        }
+        return count;
+    }
+    
     public DataType getSQLType() {
         if (isCountDistinct() || isTopN())
             return DataType.ANY;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java
index 618d25a..1561b1f 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java
@@ -18,9 +18,6 @@
 
 package org.apache.kylin.metadata.model;
 
-import java.util.Collections;
-import java.util.List;
-
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -40,15 +37,6 @@ public class MeasureDesc {
     @JsonProperty("dependent_measure_ref")
     private String dependentMeasureRef;
 
-    public List<TblColRef> getColumnsNeedDictionary() {
-        // measure could store literal values using dictionary encoding to save space, like TopN
-        if (function.isTopN()) {
-            return Collections.singletonList(function.getTopNLiteralColumn());
-        } else {
-            return Collections.emptyList();
-        }
-    }
-
     public int getId() {
         return id;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
index de27145..61ba73b 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
@@ -21,7 +21,7 @@ package org.apache.kylin.metadata.model;
 import java.io.Serializable;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataType;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java
index 4a8c5d1..e52bf3b 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java
@@ -1,7 +1,7 @@
 package org.apache.kylin.metadata.realization;
 
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java b/core-metadata/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
new file mode 100644
index 0000000..6d1a7b8
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
@@ -0,0 +1,46 @@
+package org.apache.kylin.aggregation.topn;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.topn.TopNCounter;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.measure.topn.TopNCounterSerializer;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class TopNCounterSerializerTest {
+
+    private static TopNCounterSerializer serializer = new TopNCounterSerializer(DataType.getType("topn(10)"));
+
+    @Test
+    public void testSerialization() {
+        TopNCounter<ByteArray> vs = new TopNCounter<ByteArray>(50);
+        Integer[] stream = { 1, 1, 2, 9, 1, 2, 3, 7, 7, 1, 3, 1, 1 };
+        for (Integer i : stream) {
+            vs.offer(new ByteArray(Bytes.toBytes(i)));
+        }
+
+        ByteBuffer out = ByteBuffer.allocate(1024);
+        serializer.serialize(vs, out);
+        
+        byte[] copyBytes = new byte[out.position()];
+        System.arraycopy(out.array(), 0, copyBytes, 0, out.position());
+
+        ByteBuffer in = ByteBuffer.wrap(copyBytes);
+        TopNCounter<ByteArray> vsNew = serializer.deserialize(in);
+
+        Assert.assertEquals(vs.toString(), vsNew.toString());
+
+    }
+    
+    @Test
+    public void testValueOf() {
+        // FIXME need a good unit test for valueOf()
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/test/java/org/apache/kylin/metadata/datatype/BigDecimalSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/datatype/BigDecimalSerializerTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/datatype/BigDecimalSerializerTest.java
new file mode 100644
index 0000000..f920ee7
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/metadata/datatype/BigDecimalSerializerTest.java
@@ -0,0 +1,53 @@
+package org.apache.kylin.metadata.datatype;
+
+import static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.metadata.datatype.BigDecimalSerializer;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ */
+public class BigDecimalSerializerTest {
+
+    private static BigDecimalSerializer bigDecimalSerializer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        bigDecimalSerializer = new BigDecimalSerializer(DataType.getType("decimal"));
+    }
+
+    @Test
+    public void testNormal() {
+        BigDecimal input = new BigDecimal("1234.1234");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        buffer.mark();
+        bigDecimalSerializer.serialize(input, buffer);
+        buffer.reset();
+        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
+        assertEquals(input, output);
+    }
+
+    @Test
+    public void testScaleOutOfRange() {
+        BigDecimal input = new BigDecimal("1234.1234567890");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        buffer.mark();
+        bigDecimalSerializer.serialize(input, buffer);
+        buffer.reset();
+        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
+        assertEquals(input.setScale(bigDecimalSerializer.type.getScale(), BigDecimal.ROUND_HALF_EVEN), output);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testOutOfPrecision() {
+        BigDecimal input = new BigDecimal("66855344214907231736.4924");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        bigDecimalSerializer.serialize(input, buffer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-storage/src/main/java/org/apache/kylin/storage/translate/ColumnValueRange.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/translate/ColumnValueRange.java b/core-storage/src/main/java/org/apache/kylin/storage/translate/ColumnValueRange.java
index fbd6f97..0dc1afa 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/translate/ColumnValueRange.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/translate/ColumnValueRange.java
@@ -23,8 +23,8 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.kv.RowKeyColumnOrder;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 4f011cf..3bca687 100644
--- 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
@@ -24,11 +24,11 @@ import java.util.List;
 
 import net.sf.ehcache.pool.sizeof.annotations.IgnoreSizeOf;
 
-import org.apache.kylin.common.datatype.DoubleMutable;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.metadata.datatype.DoubleMutable;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-storage/src/test/java/org/apache/kylin/storage/translate/ColumnValueRangeTest.java
----------------------------------------------------------------------
diff --git a/core-storage/src/test/java/org/apache/kylin/storage/translate/ColumnValueRangeTest.java b/core-storage/src/test/java/org/apache/kylin/storage/translate/ColumnValueRangeTest.java
index aaf07fe..d32f171 100644
--- a/core-storage/src/test/java/org/apache/kylin/storage/translate/ColumnValueRangeTest.java
+++ b/core-storage/src/test/java/org/apache/kylin/storage/translate/ColumnValueRangeTest.java
@@ -6,7 +6,7 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.dict.StringBytesConverter;
 import org.apache.kylin.dict.TrieDictionaryBuilder;
 import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 3fa7d5c..45cc88e 100644
--- 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
@@ -6,13 +6,11 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
-import com.google.common.collect.Maps;
-
 import org.apache.hadoop.io.Text;
-import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesSplitter;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.SplittedBytes;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -22,11 +20,16 @@ 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.dict.Dictionary;
 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.*;
+import org.apache.kylin.measure.MeasureCodec;
+import org.apache.kylin.measure.MeasureIngester;
+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.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,6 +52,8 @@ public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VAL
     protected String intermediateTableRowDelimiter;
     protected byte byteRowDelimiter;
     protected int counter;
+    protected MeasureIngester<?>[] aggrIngesters;
+    protected Map<TblColRef, Dictionary<String>> dictionaryMap;
     protected Object[] measures;
     protected byte[][] keyBytesBuf;
     protected BytesSplitter bytesSplitter;
@@ -58,7 +63,6 @@ public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VAL
     private Text outputKey = new Text();
     private Text outputValue = new Text();
     private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-    private Map<Integer, Dictionary<String>> topNLiteralColDictMap;
 
     @Override
     protected void setup(Context context) throws IOException {
@@ -93,25 +97,12 @@ public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VAL
         int colCount = cubeDesc.getRowkey().getRowKeyColumns().length;
         keyBytesBuf = new byte[colCount][];
 
-        initTopNLiteralColDictionaryMap();
+        aggrIngesters = MeasureIngester.create(cubeDesc.getMeasures());
+        dictionaryMap = cubeSegment.buildDictionaryMap();
+        
         initNullBytes();
     }
-
-    private void initTopNLiteralColDictionaryMap() {
-        topNLiteralColDictMap = Maps.newHashMap();
-        for (int measureIdx = 0; measureIdx < measures.length; measureIdx++) {
-            MeasureDesc measureDesc = cubeDesc.getMeasures().get(measureIdx);
-            FunctionDesc func = measureDesc.getFunction();
-            if (func.isTopN()) {
-                int[] flatTableIdx = intermediateTableDesc.getMeasureColumnIndexes()[measureIdx];
-                int literalColIdx = flatTableIdx[flatTableIdx.length - 1];
-                TblColRef literalCol = func.getTopNLiteralColumn();
-                Dictionary<String> dictionary = (Dictionary<String>) cubeSegment.getDictionary(literalCol);
-                topNLiteralColDictMap.put(literalColIdx, dictionary);
-            }
-        }
-    }
-
+    
     private void initNullBytes() {
         nullBytes = Lists.newArrayList();
         nullBytes.add(HIVE_NULL);
@@ -146,72 +137,46 @@ public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VAL
     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);
+            measures[i] = buildValueOf(i, splitBuffers);
         }
 
         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());
-        }
-        // count and count distinct
-        else if (func.isCount() || func.isHolisticCountDistinct()) {
-            // note for holistic count distinct, this value will be ignored
-            result = ONE;
-        }
-        // topN, need encode the key column
-        else if (func.isTopN()) {
-            // encode the key column with dict, and get the counter column;
-            int keyColIndex = flatTableIdx[flatTableIdx.length - 1];
-            Dictionary<String> literalColDict = topNLiteralColDictMap.get(keyColIndex);
-            int keyColEncoded = literalColDict.getIdFromValue(Bytes.toString(splitBuffers[keyColIndex].value));
-            valueBuf.clear();
-            valueBuf.putInt(literalColDict.getSizeOfId());
-            valueBuf.putInt(keyColEncoded);
-            if (flatTableIdx.length == 1) {
-                // only literalCol, use 1.0 as counter
-                valueBuf.putDouble(1.0);
+    private Object buildValueOf(int idxOfMeasure, SplittedBytes[] splitBuffers) {
+        MeasureDesc measure = cubeDesc.getMeasures().get(idxOfMeasure);
+        FunctionDesc function = measure.getFunction();
+        int[] colIdxOnFlatTable = intermediateTableDesc.getMeasureColumnIndexes()[idxOfMeasure];
+        
+        int paramCount = function.getParameterCount();
+        String[] inputToMeasure = new String[paramCount];
+
+        // pick up parameter values
+        ParameterDesc param = function.getParameter();
+        int colParamIdx = 0; // index among parameters of column type
+        for (int i = 0; i < paramCount; i++, param = param.getNextParameter()) {
+            String value;
+            if (function.isCount() || function.isHolisticCountDistinct()) {
+                // note for holistic count distinct, this value will be ignored
+                value = "1";
+            } else if (param.isColumnType()) {
+                value = getCell(colIdxOnFlatTable[colParamIdx++], splitBuffers);
             } else {
-                // get the counter column value
-                valueBuf.putDouble(Double.valueOf(Bytes.toString(splitBuffers[flatTableIdx[0]].value)));
-            }
-
-            result = valueBuf.array();
-
-        }
-        // normal case, concat 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;
-                }
+                value = param.getValue();
             }
+            inputToMeasure[i] = value;
         }
+        
+        return aggrIngesters[idxOfMeasure].valueOf(inputToMeasure, measure, dictionaryMap);
+    }
 
-        if (isNull(result)) {
-            result = null;
-        }
-
-        return result;
+    private String getCell(int i, SplittedBytes[] splitBuffers) {
+        byte[] bytes = Arrays.copyOf(splitBuffers[i].value, splitBuffers[i].length);
+        if (isNull(bytes))
+            return null;
+        else
+            return Bytes.toString(bytes);
     }
 
     protected void outputKV(Context context) throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 0f94dca..381b07c 100644
--- 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
@@ -23,8 +23,6 @@ import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.hadoop.io.Text;
-import org.apache.kylin.aggregation.MeasureAggregators;
-import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.kv.RowConstants;
@@ -32,6 +30,8 @@ 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.measure.MeasureAggregators;
+import org.apache.kylin.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index d724c76..752c01d 100644
--- 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
@@ -16,20 +16,18 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Dictionary;
 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.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -63,7 +61,7 @@ public class InMemCuboidMapper<KEYIN> extends KylinMapper<KEYIN, Object, ByteArr
         cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
         flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSegment).getFlatTableInputFormat();
 
-        Map<TblColRef, Dictionary<?>> dictionaryMap = Maps.newHashMap();
+        Map<TblColRef, Dictionary<String>> dictionaryMap = Maps.newHashMap();
 
         // dictionary
         for (TblColRef col : cubeDesc.getAllColumnsNeedDictionary()) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index ab87b21..c35e77f 100644
--- 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
@@ -3,8 +3,6 @@ package org.apache.kylin.engine.mr.steps;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.kylin.aggregation.MeasureAggregators;
-import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -16,6 +14,8 @@ 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.measure.MeasureAggregators;
+import org.apache.kylin.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 8d00084..bc1c883 100644
--- 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
@@ -23,12 +23,12 @@ import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.List;
 
-import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.SplittedBytes;
 import org.apache.kylin.cube.CubeInstance;
@@ -40,7 +40,6 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.kv.RowKeyEncoder;
 import org.apache.kylin.cube.kv.RowKeyEncoderProvider;
 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.ByteArrayWritable;
 import org.apache.kylin.engine.mr.IMROutput2.IMRStorageInputFormat;
@@ -48,6 +47,7 @@ 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.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index d08d2a4..c0277b5 100644
--- 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
@@ -27,12 +27,12 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.SplittedBytes;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -43,11 +43,11 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.kv.RowKeyEncoder;
 import org.apache.kylin.cube.kv.RowKeyEncoderProvider;
 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.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index b73fda4..84fd46b 100644
--- 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
@@ -32,7 +32,6 @@ import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.CubeUpdate;
 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;



[13/13] incubator-kylin git commit: KYLIN-976 extract MeasureIngester.reEncodeDictionary()

Posted by li...@apache.org.
KYLIN-976 extract MeasureIngester.reEncodeDictionary()


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

Branch: refs/heads/KYLIN-976
Commit: dac248804b331f491175e06fd9f598235b2c9e6f
Parents: ce61309
Author: Li, Yang <ya...@ebay.com>
Authored: Fri Nov 27 16:59:51 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Fri Nov 27 16:59:51 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/measure/MeasureIngester.java   |  2 +
 .../org/apache/kylin/measure/MeasureType.java   |  3 -
 .../kylin/measure/basic/BasicMeasureType.java   |  7 --
 .../kylin/measure/basic/BigDecimalIngester.java |  6 ++
 .../kylin/measure/basic/DoubleIngester.java     |  6 ++
 .../kylin/measure/basic/LongIngester.java       |  6 ++
 .../kylin/measure/hllc/HLLCMeasureType.java     | 11 ++-
 .../kylin/measure/topn/TopNMeasureType.java     | 43 +++++++++---
 .../mr/steps/MergeCuboidFromStorageMapper.java  | 68 +++++++------------
 .../engine/mr/steps/MergeCuboidMapper.java      | 71 +++++++-------------
 10 files changed, 107 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
index 8d6e601..9c7b406 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
@@ -41,4 +41,6 @@ abstract public class MeasureIngester<V> {
     }
 
     abstract public V valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap);
+    
+    abstract public V reEncodeDictionary(V value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDicts, Map<TblColRef, Dictionary<String>> newDicts);
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
index 604a7b6..4fe59c0 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
@@ -21,7 +21,6 @@ package org.apache.kylin.measure;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.measure.basic.BasicMeasureFactory;
 import org.apache.kylin.measure.hllc.HLLCAggregationFactory;
 import org.apache.kylin.measure.topn.TopNMeasureFactory;
@@ -86,8 +85,6 @@ abstract public class MeasureType {
  
     abstract public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc);
     
-    abstract public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts);
-
     /* ============================================================================
      * Cube Selection
      * ---------------------------------------------------------------------------- */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureType.java
index f6bf090..fe53bab 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureType.java
@@ -20,7 +20,6 @@ package org.apache.kylin.measure.basic;
 
 import java.util.List;
 
-import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.measure.MeasureIngester;
 import org.apache.kylin.measure.MeasureType;
@@ -126,10 +125,4 @@ public class BasicMeasureType extends MeasureType {
         return null;
     }
 
-    @Override
-    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalIngester.java
index bb743d6..ea1495c 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalIngester.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalIngester.java
@@ -28,6 +28,7 @@ import org.apache.kylin.metadata.model.TblColRef;
 
 public class BigDecimalIngester extends MeasureIngester<BigDecimal> {
 
+    @Override
     public BigDecimal valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
         if (values.length > 1)
             throw new IllegalArgumentException();
@@ -37,4 +38,9 @@ public class BigDecimalIngester extends MeasureIngester<BigDecimal> {
         else
             return new BigDecimal(values[0]);
     }
+
+    @Override
+    public BigDecimal reEncodeDictionary(BigDecimal value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDict, Map<TblColRef, Dictionary<String>> newDict) {
+        throw new UnsupportedOperationException();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
index 506ed19..aaa754a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
@@ -31,6 +31,7 @@ public class DoubleIngester extends MeasureIngester<DoubleMutable> {
     // avoid repeated object creation
     private DoubleMutable current = new DoubleMutable();
 
+    @Override
     public DoubleMutable valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
         if (values.length > 1)
             throw new IllegalArgumentException();
@@ -42,4 +43,9 @@ public class DoubleIngester extends MeasureIngester<DoubleMutable> {
             l.set(Double.parseDouble(values[0]));
         return l;
     }
+
+    @Override
+    public DoubleMutable reEncodeDictionary(DoubleMutable value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDict, Map<TblColRef, Dictionary<String>> newDict) {
+        throw new UnsupportedOperationException();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
index 5bf1257..bdc1704 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
@@ -31,6 +31,7 @@ public class LongIngester extends MeasureIngester<LongMutable> {
     // avoid repeated object creation
     private LongMutable current = new LongMutable();
 
+    @Override
     public LongMutable valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
         if (values.length > 1)
             throw new IllegalArgumentException();
@@ -42,4 +43,9 @@ public class LongIngester extends MeasureIngester<LongMutable> {
             l.set(Long.parseLong(values[0]));
         return l;
     }
+
+    @Override
+    public LongMutable reEncodeDictionary(LongMutable value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDict, Map<TblColRef, Dictionary<String>> newDict) {
+        throw new UnsupportedOperationException();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
index ee90818..2ad7630 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
@@ -74,6 +74,11 @@ public class HLLCMeasureType extends MeasureType {
                     hllc.add(v == null ? "__nUlL__" : v);
                 return hllc;
             }
+
+            @Override
+            public HyperLogLogPlusCounter reEncodeDictionary(HyperLogLogPlusCounter value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDict, Map<TblColRef, Dictionary<String>> newDict) {
+                throw new UnsupportedOperationException();
+            }
         };
     }
 
@@ -91,10 +96,4 @@ public class HLLCMeasureType extends MeasureType {
         return null;
     }
 
-    @Override
-    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
index 1d2c87b..561f9f1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
@@ -18,9 +18,11 @@
 
 package org.apache.kylin.measure.topn;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
@@ -88,6 +90,37 @@ public class TopNMeasureType extends MeasureType {
                 topNCounter.offer(key, counter);
                 return topNCounter;
             }
+            
+            @SuppressWarnings("unchecked")
+            @Override
+            public TopNCounter reEncodeDictionary(TopNCounter value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDicts, Map<TblColRef, Dictionary<String>> newDicts) {
+                TopNCounter<ByteArray> topNCounter = (TopNCounter<ByteArray>) value;
+
+                TblColRef colRef = measureDesc.getFunction().getTopNLiteralColumn();
+                Dictionary<String> sourceDict = oldDicts.get(colRef);
+                Dictionary<String> mergedDict = newDicts.get(colRef);
+
+                int topNSize = topNCounter.size();
+                byte[] newIdBuf = new byte[topNSize * mergedDict.getSizeOfId()];
+                byte[] literal = new byte[sourceDict.getSizeOfValue()];
+
+                int bufOffset = 0;
+                for (Counter<ByteArray> c : topNCounter) {
+                    int oldId = BytesUtil.readUnsigned(c.getItem().array(), c.getItem().offset(), c.getItem().length());
+                    int newId;
+                    int size = sourceDict.getValueBytesFromId(oldId, literal, 0);
+                    if (size < 0) {
+                        newId = mergedDict.nullId();
+                    } else {
+                        newId = mergedDict.getIdFromValueBytes(literal, 0, size);
+                    }
+
+                    BytesUtil.writeUnsigned(newId, newIdBuf, bufOffset, mergedDict.getSizeOfId());
+                    c.getItem().set(newIdBuf, bufOffset, mergedDict.getSizeOfId());
+                    bufOffset += mergedDict.getSizeOfId();
+                }
+                return value;
+            }
         };
     }
 
@@ -98,14 +131,8 @@ public class TopNMeasureType extends MeasureType {
 
     @Override
     public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
-        // TODO Auto-generated method stub
-        return null;
+        TblColRef literalCol = measureDesc.getFunction().getParameter().getColRefs().get(1);
+        return Collections.singletonList(literalCol);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/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
index bc1c883..b4682dd 100644
--- 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
@@ -22,10 +22,9 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.topn.Counter;
-import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.Dictionary;
@@ -48,6 +47,8 @@ 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.measure.MeasureCodec;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -60,6 +61,7 @@ import com.google.common.collect.Lists;
 /**
  * @author shaoshi
  */
+@SuppressWarnings({ "rawtypes", "unchecked" })
 public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, ByteArrayWritable, ByteArrayWritable> {
 
     private static final Logger logger = LoggerFactory.getLogger(MergeCuboidFromStorageMapper.class);
@@ -87,7 +89,9 @@ public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, By
     private MeasureCodec codec;
     private ByteArrayWritable outputValue = new ByteArrayWritable();
 
-    private List<Integer> topNMeasureIdx;
+    private List<Pair<Integer, MeasureIngester>> dictMeasures;
+    private Map<TblColRef, Dictionary<String>> oldDicts;
+    private Map<TblColRef, Dictionary<String>> newDicts;
 
     @Override
     protected void setup(Context context) throws IOException, InterruptedException {
@@ -115,12 +119,18 @@ public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, By
         measureDescs = cubeDesc.getMeasures();
         codec = new MeasureCodec(measureDescs);
 
-        topNMeasureIdx = Lists.newArrayList();
+        dictMeasures = Lists.newArrayList();
         for (int i = 0; i < measureDescs.size(); i++) {
-            if (measureDescs.get(i).getFunction().isTopN()) {
-                topNMeasureIdx.add(i);
+            MeasureDesc measureDesc = measureDescs.get(i);
+            MeasureType measureType = MeasureType.create(measureDesc.getFunction());
+            if (measureType.getColumnsNeedDictionary(measureDesc).isEmpty() == false) {
+                dictMeasures.add(new Pair<Integer, MeasureIngester>(i, measureType.newIngester()));
             }
         }
+        if (dictMeasures.size() > 0) {
+            oldDicts = sourceCubeSegment.buildDictionaryMap();
+            newDicts = mergedCubeSegment.buildDictionaryMap();
+        }
     }
 
     @Override
@@ -193,7 +203,9 @@ public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, By
         outputKey.set(newKeyBuf.array(), 0, fullKeySize);
 
         // re-encode measures if dictionary is used
-        reEncodeMeasure(value);
+        if (dictMeasures.size() > 0) {
+            reEncodeMeasure(value);
+        }
 
         valueBuf.clear();
         codec.encode(value, valueBuf);
@@ -217,45 +229,11 @@ public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, By
         }
     }
 
-    @SuppressWarnings("unchecked")
     private void reEncodeMeasure(Object[] measureObjs) throws IOException, InterruptedException {
-        // currently only topN uses dictionary in measure obj
-        if (topNMeasureIdx.isEmpty())
-            return;
-
-        int bufOffset = 0;
-        for (int idx : topNMeasureIdx) {
-            TopNCounter<ByteArray> topNCounters = (TopNCounter<ByteArray>) measureObjs[idx];
-
-            MeasureDesc measureDesc = measureDescs.get(idx);
-            TblColRef colRef = measureDesc.getFunction().getTopNLiteralColumn();
-            DictionaryManager dictMgr = DictionaryManager.getInstance(config);
-            Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(colRef));
-            Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(colRef));
-
-            int topNSize = topNCounters.size();
-            while (sourceDict.getSizeOfValue() * topNSize > newKeyBodyBuf.length - bufOffset || //
-                    mergedDict.getSizeOfValue() * topNSize > newKeyBodyBuf.length - bufOffset || //
-                    mergedDict.getSizeOfId() * topNSize > newKeyBodyBuf.length - bufOffset) {
-                byte[] oldBuf = newKeyBodyBuf;
-                newKeyBodyBuf = new byte[2 * newKeyBodyBuf.length];
-                System.arraycopy(oldBuf, 0, newKeyBodyBuf, 0, oldBuf.length);
-            }
-
-            for (Counter<ByteArray> c : topNCounters) {
-                int idInSourceDict = BytesUtil.readUnsigned(c.getItem().array(), c.getItem().offset(), c.getItem().length());
-                int idInMergedDict;
-                int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBodyBuf, bufOffset);
-                if (size < 0) {
-                    idInMergedDict = mergedDict.nullId();
-                } else {
-                    idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBodyBuf, bufOffset, size);
-                }
-
-                BytesUtil.writeUnsigned(idInMergedDict, newKeyBodyBuf, bufOffset, mergedDict.getSizeOfId());
-                c.getItem().set(newKeyBodyBuf, bufOffset, mergedDict.getSizeOfId());
-                bufOffset += mergedDict.getSizeOfId();
-            }
+        for (Pair<Integer, MeasureIngester> pair : dictMeasures) {
+            int i = pair.getFirst();
+            MeasureIngester ingester = pair.getSecond();
+            measureObjs[i] = ingester.reEncodeDictionary(measureObjs[i], measureDescs.get(i), oldDicts, newDicts);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dac24880/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
index c0277b5..4fc7236 100644
--- 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
@@ -22,17 +22,17 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.topn.Counter;
-import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.SplittedBytes;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -48,6 +48,8 @@ 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.measure.MeasureCodec;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -57,6 +59,7 @@ import com.google.common.collect.Lists;
 /**
  * @author ysong1, honma
  */
+@SuppressWarnings({"rawtypes", "unchecked"})
 public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
 
     private KylinConfig config;
@@ -78,7 +81,9 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
     private HashMap<TblColRef, Boolean> dimensionsNeedDict = new HashMap<TblColRef, Boolean>();
 
     // for re-encode measures that use dictionary
-    private List<Integer> topNMeasureIdx;
+    private List<Pair<Integer, MeasureIngester>> dictMeasures;
+    private Map<TblColRef, Dictionary<String>> oldDicts;
+    private Map<TblColRef, Dictionary<String>> newDicts;
     private List<MeasureDesc> measureDescs;
     private MeasureCodec codec;
     private Object[] measureObjs;
@@ -115,12 +120,19 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
         measureObjs = new Object[measureDescs.size()];
         valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
         outputValue = new Text();
-        topNMeasureIdx = Lists.newArrayList();
+        
+        dictMeasures = Lists.newArrayList();
         for (int i = 0; i < measureDescs.size(); i++) {
-            if (measureDescs.get(i).getFunction().isTopN()) {
-                topNMeasureIdx.add(i);
+            MeasureDesc measureDesc = measureDescs.get(i);
+            MeasureType measureType = MeasureType.create(measureDesc.getFunction());
+            if (measureType.getColumnsNeedDictionary(measureDesc).isEmpty() == false) {
+                dictMeasures.add(new Pair<Integer, MeasureIngester>(i, measureType.newIngester()));
             }
         }
+        if (dictMeasures.size() > 0) {
+            oldDicts = sourceCubeSegment.buildDictionaryMap();
+            newDicts = mergedCubeSegment.buildDictionaryMap();
+        }
     }
 
     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})");
@@ -214,9 +226,13 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
         outputKey.set(newKeyBuf.array(), 0, fullKeySize);
 
         // re-encode measures if dictionary is used
-        if (topNMeasureIdx.size() > 0) {
+        if (dictMeasures.size() > 0) {
             codec.decode(ByteBuffer.wrap(value.getBytes(), 0, value.getLength()), measureObjs);
-            reEncodeMeasure(measureObjs);
+            for (Pair<Integer, MeasureIngester> pair : dictMeasures) {
+                int i = pair.getFirst();
+                MeasureIngester ingester = pair.getSecond();
+                measureObjs[i] = ingester.reEncodeDictionary(measureObjs[i], measureDescs.get(i), oldDicts, newDicts);
+            }
             valueBuf.clear();
             codec.encode(measureObjs, valueBuf);
             outputValue.set(valueBuf.array(), 0, valueBuf.position());
@@ -240,43 +256,4 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
             return ret;
         }
     }
-
-    @SuppressWarnings("unchecked")
-    private void reEncodeMeasure(Object[] measureObjs) throws IOException, InterruptedException {
-        int bufOffset = 0;
-        for (int idx : topNMeasureIdx) {
-            // only TopN measure uses dic
-            TopNCounter<ByteArray> topNCounters = (TopNCounter<ByteArray>) measureObjs[idx];
-
-            MeasureDesc measureDesc = measureDescs.get(idx);
-            TblColRef colRef = measureDesc.getFunction().getTopNLiteralColumn();
-            DictionaryManager dictMgr = DictionaryManager.getInstance(config);
-            Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(colRef));
-            Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(colRef));
-
-            int topNSize = topNCounters.size();
-            while (sourceDict.getSizeOfValue() * topNSize > newKeyBodyBuf.length - bufOffset || //
-                    mergedDict.getSizeOfValue() * topNSize > newKeyBodyBuf.length - bufOffset || //
-                    mergedDict.getSizeOfId() * topNSize > newKeyBodyBuf.length - bufOffset) {
-                byte[] oldBuf = newKeyBodyBuf;
-                newKeyBodyBuf = new byte[2 * newKeyBodyBuf.length];
-                System.arraycopy(oldBuf, 0, newKeyBodyBuf, 0, oldBuf.length);
-            }
-
-            for (Counter<ByteArray> c : topNCounters) {
-                int idInSourceDict = BytesUtil.readUnsigned(c.getItem().array(), c.getItem().offset(), c.getItem().length());
-                int idInMergedDict;
-                int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBodyBuf, bufOffset);
-                if (size < 0) {
-                    idInMergedDict = mergedDict.nullId();
-                } else {
-                    idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBodyBuf, bufOffset, size);
-                }
-
-                BytesUtil.writeUnsigned(idInMergedDict, newKeyBodyBuf, bufOffset, mergedDict.getSizeOfId());
-                c.getItem().set(newKeyBodyBuf, bufOffset, mergedDict.getSizeOfId());
-                bufOffset += mergedDict.getSizeOfId();
-            }
-        }
-    }
 }


[10/13] incubator-kylin git commit: KYLIN-976 Add ingester; Build part done, in-mem cube test pass

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index ed0a166..832584c 100644
--- 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
@@ -33,10 +33,10 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Dictionary;
 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.gridtable.GTRecord;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.junit.AfterClass;
@@ -58,7 +58,7 @@ public class DoggedCubeBuilderTest extends LocalFileMetadataTestCase {
 
     private static CubeInstance cube;
     private static String flatTable;
-    private static Map<TblColRef, Dictionary<?>> dictionaryMap;
+    private static Map<TblColRef, Dictionary<String>> dictionaryMap;
 
     @BeforeClass
     public static void before() throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index f853b08..ab87c2b 100644
--- 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
@@ -33,13 +33,13 @@ 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.Dictionary;
 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.dict.IterableDictionaryValueEnumerator;
 import org.apache.kylin.gridtable.GTRecord;
@@ -66,7 +66,7 @@ public class InMemCubeBuilderTest extends LocalFileMetadataTestCase {
 
     private static CubeInstance cube;
     private static String flatTable;
-    private static Map<TblColRef, Dictionary<?>> dictionaryMap;
+    private static Map<TblColRef, Dictionary<String>> dictionaryMap;
 
     @BeforeClass
     public static void before() throws IOException {
@@ -166,8 +166,8 @@ public class InMemCubeBuilderTest extends LocalFileMetadataTestCase {
         queue.put(new ArrayList<String>(0));
     }
 
-    static Map<TblColRef, Dictionary<?>> getDictionaryMap(CubeInstance cube, String flatTable) throws IOException {
-        Map<TblColRef, Dictionary<?>> result = Maps.newHashMap();
+    static Map<TblColRef, Dictionary<String>> getDictionaryMap(CubeInstance cube, String flatTable) throws IOException {
+        Map<TblColRef, Dictionary<String>> result = Maps.newHashMap();
         CubeDesc desc = cube.getDescriptor();
         CubeJoinedFlatTableDesc flatTableDesc = new CubeJoinedFlatTableDesc(desc, null);
         int nColumns = flatTableDesc.getColumnList().size();
@@ -178,7 +178,7 @@ public class InMemCubeBuilderTest extends LocalFileMetadataTestCase {
             if (desc.getRowkey().isUseDictionary(col)) {
                 logger.info("Building dictionary for " + col);
                 List<byte[]> valueList = readValueList(flatTable, nColumns, flatTableDesc.getRowKeyColumnIndexes()[c]);
-                Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(col.getType(), new IterableDictionaryValueEnumerator(valueList));
+                Dictionary<String> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(col.getType(), new IterableDictionaryValueEnumerator(valueList));
                 result.put(col, dict);
             }
         }
@@ -192,7 +192,7 @@ public class InMemCubeBuilderTest extends LocalFileMetadataTestCase {
                 TblColRef literalCol = func.getTopNLiteralColumn();
                 logger.info("Building dictionary for " + literalCol);
                 List<byte[]> valueList = readValueList(flatTable, nColumns, literalColIdx);
-                Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(literalCol.getType(), new IterableDictionaryValueEnumerator(valueList));
+                Dictionary<String> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(literalCol.getType(), new IterableDictionaryValueEnumerator(valueList));
 
                 result.put(literalCol, dict);
             }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 93f0419..6ae0d0e 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
@@ -23,15 +23,15 @@ import java.util.Random;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.aggregation.basic.BigDecimalSumAggregator;
-import org.apache.kylin.aggregation.basic.DoubleSumAggregator;
-import org.apache.kylin.aggregation.basic.LongSumAggregator;
-import org.apache.kylin.aggregation.hllc.HLLCAggregator;
-import org.apache.kylin.common.datatype.DoubleMutable;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.measure.basic.BigDecimalSumAggregator;
+import org.apache.kylin.measure.basic.DoubleSumAggregator;
+import org.apache.kylin.measure.basic.LongSumAggregator;
+import org.apache.kylin.measure.hllc.HLLCAggregator;
+import org.apache.kylin.metadata.datatype.DoubleMutable;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.junit.Test;
 
 public class AggregationCacheMemSizeTest {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index b3981e8..836bafd 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java
@@ -27,18 +27,18 @@ import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.Pair;
 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.datatype.DataType;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 4313f4b..5454490 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
@@ -26,9 +26,9 @@ import java.math.BigDecimal;
 import java.util.BitSet;
 import java.util.List;
 
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.gridtable.memstore.GTSimpleMemStore;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.junit.Test;
 
 public class SimpleGridTableTest {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 7f4da61..02fbecc 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
@@ -24,10 +24,10 @@ import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.LongMutable;
-import org.apache.kylin.common.datatype.StringSerializer;
 import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.LongMutable;
+import org.apache.kylin.metadata.datatype.StringSerializer;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
index 4da5471..e9f74a4 100644
--- a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
@@ -23,11 +23,11 @@ import static org.junit.Assert.assertEquals;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
-import org.apache.kylin.aggregation.MeasureCodec;
-import org.apache.kylin.common.datatype.DoubleMutable;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.measure.MeasureCodec;
+import org.apache.kylin.metadata.datatype.DoubleMutable;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
index b882e58..62b06aa 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
@@ -30,6 +30,7 @@ import java.io.UnsupportedEncodingException;
 import java.util.Date;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.util.Dictionary;
 
 /**
  * A dictionary for date string (date only, no time).

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/DictCodeSystem.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictCodeSystem.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictCodeSystem.java
index f19a2a8..b0326c1 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictCodeSystem.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictCodeSystem.java
@@ -3,6 +3,7 @@ package org.apache.kylin.dict;
 import java.nio.ByteBuffer;
 
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.metadata.filter.IFilterCodeSystem;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/Dictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/Dictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/Dictionary.java
deleted file mode 100644
index d1fc6f9..0000000
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/Dictionary.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.dict;
-
-import java.io.PrintStream;
-import java.io.Serializable;
-import java.io.UnsupportedEncodingException;
-
-import org.apache.kylin.common.persistence.Writable;
-import org.apache.kylin.common.util.BytesUtil;
-
-/**
- * A bi-way dictionary that maps from dimension/column values to IDs and vice
- * versa. By storing IDs instead of real values, the size of cube is
- * significantly reduced.
- * 
- * - IDs are smallest integers possible for the cardinality of a column, for the
- * purpose of minimal storage space - IDs preserve ordering of values, such that
- * range query can be applied to IDs directly
- * 
- * A dictionary once built, is immutable. This allows optimal memory footprint
- * by e.g. flatten the Trie structure into a byte array, replacing node pointers
- * with array offsets.
- * 
- * @author yangli9
- */
-@SuppressWarnings("serial")
-abstract public class Dictionary<T> implements Writable, Serializable {
-
-    public static final byte NULL = (byte) 0xff;
-
-    // ID with all bit-1 (0xff e.g.) reserved for NULL value
-    public static final int NULL_ID[] = new int[] { 0, 0xff, 0xffff, 0xffffff, 0xffffffff };
-
-    abstract public int getMinId();
-
-    abstract public int getMaxId();
-
-    public int getSize() {
-        return getMaxId() - getMinId() + 1;
-    }
-
-    /**
-     * @return the size of an ID in bytes, determined by the cardinality of column
-     */
-    abstract public int getSizeOfId();
-
-    /**
-     * @return the (maximum) size of value in bytes, determined by the longest value
-     */
-    abstract public int getSizeOfValue();
-
-    /**
-     * @return true if each entry of this dict is contained by the dict in param
-     */
-    abstract public boolean contains(Dictionary<?> another);
-
-    /**
-     * Convenient form of <code>getIdFromValue(value, 0)</code>
-     */
-    final public int getIdFromValue(T value) throws IllegalArgumentException {
-        return getIdFromValue(value, 0);
-    }
-
-    /**
-     * Returns the ID integer of given value. In case of not found
-     * <p>
-     * - if roundingFlag=0, throw IllegalArgumentException; <br>
-     * - if roundingFlag<0, the closest smaller ID integer if exist; <br>
-     * - if roundingFlag>0, the closest bigger ID integer if exist. <br>
-     * <p>
-     * The implementation often has cache, thus faster than the byte[] version getIdFromValueBytes()
-     * 
-     * @throws IllegalArgumentException
-     *             if value is not found in dictionary and rounding is off;
-     *             or if rounding cannot find a smaller or bigger ID
-     */
-    final public int getIdFromValue(T value, int roundingFlag) throws IllegalArgumentException {
-        if (isNullObjectForm(value))
-            return nullId();
-        else
-            return getIdFromValueImpl(value, roundingFlag);
-    }
-
-    final public boolean containsValue(T value) throws IllegalArgumentException {
-        if (isNullObjectForm(value)) {
-            return true;
-        } else {
-            try {
-                //if no key found, it will throw exception
-                getIdFromValueImpl(value, 0);
-            } catch (IllegalArgumentException e) {
-                return false;
-            }
-            return true;
-        }
-    }
-
-    protected boolean isNullObjectForm(T value) {
-        return value == null;
-    }
-
-    abstract protected int getIdFromValueImpl(T value, int roundingFlag);
-
-    /**
-     * @return the value corresponds to the given ID
-     * @throws IllegalArgumentException
-     *             if ID is not found in dictionary
-     */
-    final public T getValueFromId(int id) throws IllegalArgumentException {
-        if (isNullId(id))
-            return null;
-        else
-            return getValueFromIdImpl(id);
-    }
-
-    abstract protected T getValueFromIdImpl(int id);
-
-    /**
-     * Convenient form of
-     * <code>getIdFromValueBytes(value, offset, len, 0)</code>
-     */
-    final public int getIdFromValueBytes(byte[] value, int offset, int len) throws IllegalArgumentException {
-        return getIdFromValueBytes(value, offset, len, 0);
-    }
-
-    /**
-     * A lower level API, return ID integer from raw value bytes. In case of not found 
-     * <p>
-     * - if roundingFlag=0, throw IllegalArgumentException; <br>
-     * - if roundingFlag<0, the closest smaller ID integer if exist; <br>
-     * - if roundingFlag>0, the closest bigger ID integer if exist. <br>
-     * <p>
-     * Bypassing the cache layer, this could be significantly slower than getIdFromValue(T value).
-     * 
-     * @throws IllegalArgumentException
-     *             if value is not found in dictionary and rounding is off;
-     *             or if rounding cannot find a smaller or bigger ID
-     */
-    final public int getIdFromValueBytes(byte[] value, int offset, int len, int roundingFlag) throws IllegalArgumentException {
-        if (isNullByteForm(value, offset, len))
-            return nullId();
-        else {
-            int id = getIdFromValueBytesImpl(value, offset, len, roundingFlag);
-            if (id < 0)
-                throw new IllegalArgumentException("Value not exists!");
-            return id;
-        }
-    }
-
-    protected boolean isNullByteForm(byte[] value, int offset, int len) {
-        return value == null;
-    }
-
-    abstract protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag);
-
-    final public byte[] getValueBytesFromId(int id) {
-        if (isNullId(id))
-            return BytesUtil.EMPTY_BYTE_ARRAY;
-        else
-            return getValueBytesFromIdImpl(id);
-    }
-
-    abstract protected byte[] getValueBytesFromIdImpl(int id);
-
-    /**
-     * A lower level API, get byte values from ID, return the number of bytes
-     * written. Bypassing the cache layer, this could be significantly slower
-     * than getIdFromValue(T value).
-     *
-     * @return size of value bytes, 0 if empty string, -1 if null
-     *
-     * @throws IllegalArgumentException
-     *             if ID is not found in dictionary
-     */
-    final public int getValueBytesFromId(int id, byte[] returnValue, int offset) throws IllegalArgumentException {
-        if (isNullId(id))
-            return -1;
-        else
-            return getValueBytesFromIdImpl(id, returnValue, offset);
-    }
-
-    abstract protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset);
-
-    abstract public void dump(PrintStream out);
-
-    public int nullId() {
-        return NULL_ID[getSizeOfId()];
-    }
-
-    public boolean isNullId(int id) {
-        int nullId = NULL_ID[getSizeOfId()];
-        return (nullId & id) == nullId;
-    }
-
-    /** utility that converts a dictionary ID to string, preserving order */
-    public static String dictIdToString(byte[] idBytes, int offset, int length) {
-        try {
-            return new String(idBytes, offset, length, "ISO-8859-1");
-        } catch (UnsupportedEncodingException e) {
-            // never happen
-            return null;
-        }
-    }
-
-    /** the reverse of dictIdToString(), returns integer ID */
-    public static int stringToDictId(String str) {
-        try {
-            byte[] bytes = str.getBytes("ISO-8859-1");
-            return BytesUtil.readUnsigned(bytes, 0, bytes.length);
-        } catch (UnsupportedEncodingException e) {
-            // never happen
-            return 0;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
index 300c240..4b01e60 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
@@ -26,9 +26,10 @@ import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.source.ReadableTable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -55,7 +56,7 @@ public class DictionaryGenerator {
         }
     }
 
-    public static Dictionary<?> buildDictionaryFromValueEnumerator(DataType dataType, IDictionaryValueEnumerator valueEnumerator) throws IOException {
+    public static Dictionary<String> buildDictionaryFromValueEnumerator(DataType dataType, IDictionaryValueEnumerator valueEnumerator) throws IOException {
         Preconditions.checkNotNull(dataType, "dataType cannot be null");
         Dictionary dict;
         int baseId = 0; // always 0 for now
@@ -94,7 +95,7 @@ public class DictionaryGenerator {
         return buildDictionaryFromValueEnumerator(dataType, new MultipleDictionaryValueEnumerator(sourceDicts));
     }
 
-    public static Dictionary<?> buildDictionary(DictionaryInfo info, ReadableTable inpTable) throws IOException {
+    public static Dictionary<String> buildDictionary(DictionaryInfo info, ReadableTable inpTable) throws IOException {
 
         // currently all data types are casted to string to build dictionary
         // String dataType = info.getDataType();
@@ -111,7 +112,7 @@ public class DictionaryGenerator {
         }
     }
 
-    private static Dictionary buildDateDict(IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList samples) throws IOException {
+    private static Dictionary<String> buildDateDict(IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList samples) throws IOException {
         final int BAD_THRESHOLD = 0;
         String matchPattern = null;
         byte[] value;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfo.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfo.java
index f482002..4fba59a 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfo.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfo.java
@@ -20,6 +20,7 @@ package org.apache.kylin.dict;
 
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.source.ReadableTable.TableSignature;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfoSerializer.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfoSerializer.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfoSerializer.java
index 5b7a318..69b29fe 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfoSerializer.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryInfoSerializer.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 
 import org.apache.kylin.common.persistence.Serializer;
 import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.JsonUtil;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
index 2f4b761..f538142 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
@@ -30,9 +30,10 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionarySerializer.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionarySerializer.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionarySerializer.java
index b80f838..6b47868 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionarySerializer.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionarySerializer.java
@@ -10,6 +10,7 @@ import java.io.OutputStream;
 
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.common.util.Dictionary;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryAware.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryAware.java b/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryAware.java
index 1c7a009..4586163 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryAware.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryAware.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.dict;
 
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.metadata.model.TblColRef;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java b/core-dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java
index 13f7394..df7b1c6 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java
@@ -19,7 +19,9 @@
 package org.apache.kylin.dict;
 
 import com.google.common.collect.Lists;
+
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.Dictionary;
 
 import java.io.IOException;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
index 6bae129..65c6c05 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
@@ -7,6 +7,7 @@ import java.io.PrintStream;
 import java.io.UnsupportedEncodingException;
 
 import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.common.util.Dictionary;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
index 3a05d0a..552aa92 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
@@ -19,9 +19,11 @@
 package org.apache.kylin.dict;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java
index e9a99b9..6297906 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotTable.java
@@ -30,7 +30,7 @@ import java.util.List;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.dict.StringBytesConverter;
 import org.apache.kylin.dict.TrieDictionary;
 import org.apache.kylin.dict.TrieDictionaryBuilder;
@@ -44,6 +44,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 /**
  * @author yangli9
  */
+@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class SnapshotTable extends RootPersistentEntity implements ReadableTable {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
index 8c8dcfc..8020729 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
@@ -28,8 +28,9 @@ import java.util.List;
 import java.util.Random;
 import java.util.Set;
 
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregator.java
new file mode 100644
index 0000000..32e5128
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregator.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.measure;
+
+import java.io.Serializable;
+
+/**
+ */
+@SuppressWarnings("serial")
+abstract public class MeasureAggregator<V> implements Serializable {
+
+    public static MeasureAggregator<?> create(String funcName, String dataType) {
+        return MeasureType.create(funcName, dataType).newAggregator();
+    }
+
+    public static int guessBigDecimalMemBytes() {
+        // 116 returned by AggregationCacheMemSizeTest
+        return 8 // aggregator obj shell
+        + 8 // ref to BigDecimal
+        + 8 // BigDecimal obj shell
+        + 100; // guess of BigDecimal internal
+    }
+
+    public static int guessDoubleMemBytes() {
+        // 29 to 44 returned by AggregationCacheMemSizeTest
+        return 44;
+        /*
+        return 8 // aggregator obj shell
+        + 8 // ref to DoubleWritable
+        + 8 // DoubleWritable obj shell
+        + 8; // size of double
+        */
+    }
+
+    public static int guessLongMemBytes() {
+        // 29 to 44 returned by AggregationCacheMemSizeTest
+        return 44;
+        /*
+        return 8 // aggregator obj shell
+        + 8 // ref to LongWritable
+        + 8 // LongWritable obj shell
+        + 8; // size of long
+        */
+    }
+
+    // ============================================================================
+
+    @SuppressWarnings("rawtypes")
+    public void setDependentAggregator(MeasureAggregator agg) {
+    }
+
+    abstract public void reset();
+
+    abstract public void aggregate(V value);
+
+    abstract public V getState();
+
+    // get an estimate of memory consumption UPPER BOUND
+    abstract public int getMemBytesEstimate();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
new file mode 100644
index 0000000..c6b456e
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.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.measure;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+/**
+ */
+@SuppressWarnings({ "rawtypes", "unchecked", "serial" })
+public class MeasureAggregators implements Serializable {
+
+    private final MeasureAggregator[] aggs;
+    private final int descLength;
+
+    public MeasureAggregators(Collection<MeasureDesc> measureDescs) {
+        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
+    }
+
+    public MeasureAggregators(MeasureDesc... measureDescs) {
+        descLength = measureDescs.length;
+        aggs = new MeasureAggregator[descLength];
+
+        Map<String, Integer> measureIndexMap = new HashMap<String, Integer>();
+        for (int i = 0; i < descLength; i++) {
+            FunctionDesc func = measureDescs[i].getFunction();
+            aggs[i] = MeasureAggregator.create(func.getExpression(), func.getReturnType());
+            measureIndexMap.put(measureDescs[i].getName(), i);
+        }
+        // fill back dependent aggregator
+        for (int i = 0; i < descLength; i++) {
+            String depMsrRef = measureDescs[i].getDependentMeasureRef();
+            if (depMsrRef != null) {
+                int index = measureIndexMap.get(depMsrRef);
+                aggs[i].setDependentAggregator(aggs[index]);
+            }
+        }
+    }
+
+    public void reset() {
+        for (int i = 0; i < aggs.length; i++) {
+            aggs[i].reset();
+        }
+    }
+    
+    public void aggregate(Object[] values) {
+        assert values.length == descLength;
+
+        for (int i = 0; i < descLength; i++) {
+            aggs[i].aggregate(values[i]);
+        }
+    }
+
+    public void collectStates(Object[] states) {
+        for (int i = 0; i < descLength; i++) {
+            states[i] = aggs[i].getState();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
new file mode 100644
index 0000000..b02addd
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.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.measure;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class MeasureCodec {
+
+    int nMeasures;
+    DataTypeSerializer[] serializers;
+
+    public MeasureCodec(Collection<MeasureDesc> measureDescs) {
+        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
+    }
+
+    public MeasureCodec(MeasureDesc... measureDescs) {
+        String[] dataTypes = new String[measureDescs.length];
+        for (int i = 0; i < dataTypes.length; i++) {
+            dataTypes[i] = measureDescs[i].getFunction().getReturnType();
+        }
+        init(dataTypes);
+    }
+
+    public MeasureCodec(String... dataTypes) {
+        init(dataTypes);
+    }
+
+    private void init(String[] dataTypes) {
+        nMeasures = dataTypes.length;
+        serializers = new DataTypeSerializer[nMeasures];
+
+        for (int i = 0; i < nMeasures; i++) {
+            serializers[i] = DataTypeSerializer.create(dataTypes[i]);
+        }
+    }
+
+    public DataTypeSerializer getSerializer(int idx) {
+        return serializers[idx];
+    }
+
+    public void decode(ByteBuffer buf, Object[] result) {
+        assert result.length == nMeasures;
+        for (int i = 0; i < nMeasures; i++) {
+            result[i] = serializers[i].deserialize(buf);
+        }
+    }
+
+    public void encode(Object[] values, ByteBuffer out) {
+        assert values.length == nMeasures;
+        for (int i = 0; i < nMeasures; i++) {
+            serializers[i].serialize(values[i], out);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
new file mode 100644
index 0000000..8d6e601
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.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.measure;
+
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+abstract public class MeasureIngester<V> {
+    
+    public static MeasureIngester<?> create(MeasureDesc measure) {
+        return MeasureType.create(measure.getFunction()).newIngester();
+    }
+    
+    public static MeasureIngester<?>[] create(Collection<MeasureDesc> measures) {
+        MeasureIngester<?>[] result = new MeasureIngester<?>[measures.size()];
+        int i = 0;
+        for (MeasureDesc measure : measures) {
+            result[i++] = create(measure);
+        }
+        return result;
+    }
+
+    abstract public V valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
new file mode 100644
index 0000000..604a7b6
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.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.measure;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.measure.basic.BasicMeasureFactory;
+import org.apache.kylin.measure.hllc.HLLCAggregationFactory;
+import org.apache.kylin.measure.topn.TopNMeasureFactory;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+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.Maps;
+
+abstract public class MeasureType {
+    
+    private static final Map<String, IMeasureFactory> factoryRegistry = Maps.newConcurrentMap();
+    private static final IMeasureFactory defaultFactory = new BasicMeasureFactory();
+    
+    static {
+        factoryRegistry.put(FunctionDesc.FUNC_COUNT_DISTINCT, new HLLCAggregationFactory());
+        factoryRegistry.put(FunctionDesc.FUNC_TOP_N, new TopNMeasureFactory());
+    }
+    
+    public static MeasureType create(FunctionDesc function) {
+        return create(function.getExpression(), function.getReturnType());
+    }
+
+    public static MeasureType create(String funcName, String dataType) {
+        funcName = funcName.toUpperCase();
+        dataType = dataType.toLowerCase();
+        
+        IMeasureFactory factory = factoryRegistry.get(funcName);
+        if (factory == null)
+            factory = defaultFactory;
+        
+        MeasureType result = factory.createMeasureType(funcName, dataType);
+        
+        // register serializer for aggr data type
+        DataType aggregationDataType = result.getAggregationDataType();
+        if (DataTypeSerializer.hasRegistered(aggregationDataType.getName()) == false) {
+            DataTypeSerializer.register(aggregationDataType.getName(), result.getAggregationDataSeralizer());
+        }
+        
+        return result;
+    }
+    
+    /* ============================================================================
+     * Define
+     * ---------------------------------------------------------------------------- */
+    
+    abstract public DataType getAggregationDataType();
+    
+    abstract public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer();
+    
+    abstract public void validate(MeasureDesc measureDesc) throws IllegalArgumentException;
+    
+    /* ============================================================================
+     * Build
+     * ---------------------------------------------------------------------------- */
+    
+    abstract public MeasureIngester<?> newIngester();
+    
+    abstract public MeasureAggregator<?> newAggregator();
+ 
+    abstract public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc);
+    
+    abstract public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts);
+
+    /* ============================================================================
+     * Cube Selection
+     * ---------------------------------------------------------------------------- */
+    
+    /* ============================================================================
+     * Query
+     * ---------------------------------------------------------------------------- */
+    
+    /* ============================================================================
+     * Storage
+     * ---------------------------------------------------------------------------- */
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureFactory.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureFactory.java
new file mode 100644
index 0000000..7bfee49
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureFactory.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.measure.basic;
+
+import org.apache.kylin.measure.IMeasureFactory;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.metadata.datatype.DataType;
+
+public class BasicMeasureFactory implements IMeasureFactory {
+
+    @Override
+    public MeasureType createMeasureType(String funcName, String dataType) {
+        return new BasicMeasureType(funcName, DataType.getType(dataType));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureType.java
new file mode 100644
index 0000000..f6bf090
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BasicMeasureType.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.measure.basic;
+
+import java.util.List;
+
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.metadata.datatype.BigDecimalSerializer;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.apache.kylin.metadata.datatype.DoubleSerializer;
+import org.apache.kylin.metadata.datatype.LongSerializer;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class BasicMeasureType extends MeasureType {
+    
+    private final String funcName;
+    private final DataType dataType;
+
+    public BasicMeasureType(String funcName, DataType dataType) {
+        this.funcName = funcName;
+        this.dataType = dataType;
+    }
+
+    @Override
+    public DataType getAggregationDataType() {
+        return dataType;
+    }
+
+    public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer() {
+        if (dataType.isIntegerFamily())
+            return LongSerializer.class;
+        else if (dataType.isDecimal())
+            return BigDecimalSerializer.class;
+        else if (dataType.isNumberFamily())
+            return DoubleSerializer.class;
+        else
+            throw new IllegalArgumentException("No serializer for aggregation type " + dataType);
+    }
+    
+    @Override
+    public void validate(MeasureDesc measureDesc) throws IllegalArgumentException {
+        // TODO Auto-generated method stub
+        
+    }
+
+    @Override
+    public MeasureIngester<?> newIngester() {
+        if (dataType.isIntegerFamily())
+            return new LongIngester();
+        else if (dataType.isDecimal())
+            return new BigDecimalIngester();
+        else if (dataType.isNumberFamily())
+            return new DoubleIngester();
+        else
+            throw new IllegalArgumentException("No ingester for aggregation type " + dataType);
+    }
+    
+    @Override
+    public MeasureAggregator<?> newAggregator() {
+        if (isSum() || isCount()) {
+            if (dataType.isDecimal())
+                return new BigDecimalSumAggregator();
+            else if (dataType.isIntegerFamily())
+                return new LongSumAggregator();
+            else if (dataType.isNumberFamily())
+                return new DoubleSumAggregator();
+        } else if (isMax()) {
+            if (dataType.isDecimal())
+                return new BigDecimalMaxAggregator();
+            else if (dataType.isIntegerFamily())
+                return new LongMaxAggregator();
+            else if (dataType.isNumberFamily())
+                return new DoubleMaxAggregator();
+        } else if (isMin()) {
+            if (dataType.isDecimal())
+                return new BigDecimalMinAggregator();
+            else if (dataType.isIntegerFamily())
+                return new LongMinAggregator();
+            else if (dataType.isNumberFamily())
+                return new DoubleMinAggregator();
+        }
+        throw new IllegalArgumentException("No aggregator for func '" + funcName + "' and return type '" + dataType + "'");
+    }
+    
+    private boolean isSum() {
+        return FunctionDesc.FUNC_SUM.equalsIgnoreCase(funcName);
+    }
+
+    private boolean isCount() {
+        return FunctionDesc.FUNC_COUNT.equalsIgnoreCase(funcName);
+    }
+    
+    private boolean isMax() {
+        return FunctionDesc.FUNC_MAX.equalsIgnoreCase(funcName);
+    }
+    
+    private boolean isMin() {
+        return FunctionDesc.FUNC_MIN.equalsIgnoreCase(funcName);
+    }
+    
+    @Override
+    public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalIngester.java
new file mode 100644
index 0000000..bb743d6
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalIngester.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.measure.basic;
+
+import java.math.BigDecimal;
+import java.util.Map;
+
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class BigDecimalIngester extends MeasureIngester<BigDecimal> {
+
+    public BigDecimal valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+        if (values.length > 1)
+            throw new IllegalArgumentException();
+        
+        if (values[0] == null)
+            return new BigDecimal(0);
+        else
+            return new BigDecimal(values[0]);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMaxAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMaxAggregator.java
new file mode 100644
index 0000000..aa42476
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMaxAggregator.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.measure.basic;
+
+import java.math.BigDecimal;
+
+import org.apache.kylin.measure.MeasureAggregator;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class BigDecimalMaxAggregator extends MeasureAggregator<BigDecimal> {
+
+    BigDecimal max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(BigDecimal value) {
+        if (max == null)
+            max = value;
+        else if (max.compareTo(value) < 0)
+            max = value;
+    }
+
+    @Override
+    public BigDecimal getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessBigDecimalMemBytes();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMinAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMinAggregator.java
new file mode 100644
index 0000000..81193ad
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMinAggregator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.measure.basic;
+
+import java.math.BigDecimal;
+
+import org.apache.kylin.measure.MeasureAggregator;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class BigDecimalMinAggregator extends MeasureAggregator<BigDecimal> {
+
+    BigDecimal max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(BigDecimal value) {
+        if (max == null)
+            max = value;
+        else if (max.compareTo(value) > 0)
+            max = value;
+    }
+
+    @Override
+    public BigDecimal getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessBigDecimalMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalSumAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalSumAggregator.java
new file mode 100644
index 0000000..5e00c63
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalSumAggregator.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.measure.basic;
+
+import java.math.BigDecimal;
+
+import org.apache.kylin.measure.MeasureAggregator;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class BigDecimalSumAggregator extends MeasureAggregator<BigDecimal> {
+
+    BigDecimal sum = new BigDecimal(0);
+
+    @Override
+    public void reset() {
+        sum = new BigDecimal(0);
+    }
+
+    @Override
+    public void aggregate(BigDecimal value) {
+        sum = sum.add(value);
+    }
+
+    @Override
+    public BigDecimal getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessBigDecimalMemBytes();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
new file mode 100644
index 0000000..506ed19
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.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.measure.basic;
+
+import java.util.Map;
+
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.metadata.datatype.DoubleMutable;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class DoubleIngester extends MeasureIngester<DoubleMutable> {
+    
+    // avoid repeated object creation
+    private DoubleMutable current = new DoubleMutable();
+
+    public DoubleMutable valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+        if (values.length > 1)
+            throw new IllegalArgumentException();
+        
+        DoubleMutable l = current;
+        if (values[0] == null)
+            l.set(0L);
+        else
+            l.set(Double.parseDouble(values[0]));
+        return l;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMaxAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMaxAggregator.java
new file mode 100644
index 0000000..25911e8
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMaxAggregator.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.measure.basic;
+
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.DoubleMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class DoubleMaxAggregator extends MeasureAggregator<DoubleMutable> {
+
+    DoubleMutable max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(DoubleMutable value) {
+        if (max == null)
+            max = new DoubleMutable(value.get());
+        else if (max.get() < value.get())
+            max.set(value.get());
+    }
+
+    @Override
+    public DoubleMutable getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessDoubleMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMinAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMinAggregator.java
new file mode 100644
index 0000000..be97deb
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMinAggregator.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.measure.basic;
+
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.DoubleMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class DoubleMinAggregator extends MeasureAggregator<DoubleMutable> {
+
+    DoubleMutable min = null;
+
+    @Override
+    public void reset() {
+        min = null;
+    }
+
+    @Override
+    public void aggregate(DoubleMutable value) {
+        if (min == null)
+            min = new DoubleMutable(value.get());
+        else if (min.get() > value.get())
+            min.set(value.get());
+    }
+
+    @Override
+    public DoubleMutable getState() {
+        return min;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessDoubleMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleSumAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleSumAggregator.java
new file mode 100644
index 0000000..f276817
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleSumAggregator.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.measure.basic;
+
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.DoubleMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class DoubleSumAggregator extends MeasureAggregator<DoubleMutable> {
+
+    DoubleMutable sum = new DoubleMutable();
+
+    @Override
+    public void reset() {
+        sum.set(0.0);
+    }
+
+    @Override
+    public void aggregate(DoubleMutable value) {
+        sum.set(sum.get() + value.get());
+    }
+
+    @Override
+    public DoubleMutable getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessDoubleMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
new file mode 100644
index 0000000..5bf1257
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.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.measure.basic;
+
+import java.util.Map;
+
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.metadata.datatype.LongMutable;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class LongIngester extends MeasureIngester<LongMutable> {
+    
+    // avoid repeated object creation
+    private LongMutable current = new LongMutable();
+
+    public LongMutable valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+        if (values.length > 1)
+            throw new IllegalArgumentException();
+        
+        LongMutable l = current;
+        if (values[0] == null)
+            l.set(0L);
+        else
+            l.set(Long.parseLong(values[0]));
+        return l;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMaxAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMaxAggregator.java
new file mode 100644
index 0000000..ca44f15
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMaxAggregator.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.measure.basic;
+
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.LongMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class LongMaxAggregator extends MeasureAggregator<LongMutable> {
+
+    LongMutable max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(LongMutable value) {
+        if (max == null)
+            max = new LongMutable(value.get());
+        else if (max.get() < value.get())
+            max.set(value.get());
+    }
+
+    @Override
+    public LongMutable getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMinAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMinAggregator.java
new file mode 100644
index 0000000..dadc64e
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMinAggregator.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.measure.basic;
+
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.LongMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class LongMinAggregator extends MeasureAggregator<LongMutable> {
+
+    LongMutable min = null;
+
+    @Override
+    public void reset() {
+        min = null;
+    }
+
+    @Override
+    public void aggregate(LongMutable value) {
+        if (min == null)
+            min = new LongMutable(value.get());
+        else if (min.get() > value.get())
+            min.set(value.get());
+    }
+
+    @Override
+    public LongMutable getState() {
+        return min;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongSumAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongSumAggregator.java
new file mode 100644
index 0000000..e7fdc9d
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongSumAggregator.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.measure.basic;
+
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.LongMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class LongSumAggregator extends MeasureAggregator<LongMutable> {
+
+    LongMutable sum = new LongMutable();
+
+    @Override
+    public void reset() {
+        sum.set(0);
+    }
+
+    @Override
+    public void aggregate(LongMutable value) {
+        sum.set(sum.get() + value.get());
+    }
+
+    @Override
+    public LongMutable getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregationFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregationFactory.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregationFactory.java
new file mode 100644
index 0000000..13e5520
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregationFactory.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.measure.hllc;
+
+import org.apache.kylin.measure.IMeasureFactory;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.model.FunctionDesc;
+
+public class HLLCAggregationFactory implements IMeasureFactory {
+
+    @Override
+    public MeasureType createMeasureType(String funcName, String dataType) {
+        if (FunctionDesc.FUNC_COUNT_DISTINCT.equalsIgnoreCase(funcName) == false)
+            throw new IllegalArgumentException();
+        
+        return new HLLCMeasureType(DataType.getType(dataType));
+    }
+
+}



[03/13] incubator-kylin git commit: KYLIN-976 very initial

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 18884f7..be92f73 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java
@@ -2,9 +2,9 @@ package org.apache.kylin.gridtable;
 
 import java.nio.ByteBuffer;
 
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.aggregation.MeasureAggregator;
 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" })
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 48bda9f..0e61cf2 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/IGTCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTCodeSystem.java
@@ -2,8 +2,8 @@ package org.apache.kylin.gridtable;
 
 import java.nio.ByteBuffer;
 
+import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
 
 public interface IGTCodeSystem {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 1a9f637..9852acc 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
@@ -24,8 +24,8 @@ import java.util.List;
 
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.gridtable.GTInfo.Builder;
-import org.apache.kylin.metadata.measure.LongMutable;
 import org.apache.kylin.metadata.model.DataType;
 
 public class UnitTestSupport {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/test/java/org/apache/kylin/aggregation/basic/BigDecimalSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/aggregation/basic/BigDecimalSerializerTest.java b/core-cube/src/test/java/org/apache/kylin/aggregation/basic/BigDecimalSerializerTest.java
new file mode 100644
index 0000000..27aa07c
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/aggregation/basic/BigDecimalSerializerTest.java
@@ -0,0 +1,53 @@
+package org.apache.kylin.aggregation.basic;
+
+import static org.junit.Assert.assertEquals;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.aggregation.basic.BigDecimalSerializer;
+import org.apache.kylin.metadata.model.DataType;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ */
+public class BigDecimalSerializerTest {
+
+    private static BigDecimalSerializer bigDecimalSerializer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        bigDecimalSerializer = new BigDecimalSerializer(DataType.getInstance("decimal"));
+    }
+
+    @Test
+    public void testNormal() {
+        BigDecimal input = new BigDecimal("1234.1234");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        buffer.mark();
+        bigDecimalSerializer.serialize(input, buffer);
+        buffer.reset();
+        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
+        assertEquals(input, output);
+    }
+
+    @Test
+    public void testScaleOutOfRange() {
+        BigDecimal input = new BigDecimal("1234.1234567890");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        buffer.mark();
+        bigDecimalSerializer.serialize(input, buffer);
+        buffer.reset();
+        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
+        assertEquals(input.setScale(bigDecimalSerializer.type.getScale(), BigDecimal.ROUND_HALF_EVEN), output);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testOutOfPrecision() {
+        BigDecimal input = new BigDecimal("66855344214907231736.4924");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        bigDecimalSerializer.serialize(input, buffer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java b/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
new file mode 100644
index 0000000..8ee7b8d
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
@@ -0,0 +1,61 @@
+package org.apache.kylin.aggregation.topn;
+
+import org.apache.kylin.aggregation.topn.TopNCounterSerializer;
+import org.apache.kylin.common.topn.TopNCounter;
+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.metadata.model.DataType;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+/**
+ * 
+ */
+public class TopNCounterSerializerTest {
+
+    private static TopNCounterSerializer serializer = new TopNCounterSerializer(DataType.getInstance("topn(10)"));
+
+    @Test
+    public void testSerialization() {
+        TopNCounter<ByteArray> vs = new TopNCounter<ByteArray>(50);
+        Integer[] stream = { 1, 1, 2, 9, 1, 2, 3, 7, 7, 1, 3, 1, 1 };
+        for (Integer i : stream) {
+            vs.offer(new ByteArray(Bytes.toBytes(i)));
+        }
+
+        ByteBuffer out = ByteBuffer.allocate(1024);
+        serializer.serialize(vs, out);
+        
+        byte[] copyBytes = new byte[out.position()];
+        System.arraycopy(out.array(), 0, copyBytes, 0, out.position());
+
+        ByteBuffer in = ByteBuffer.wrap(copyBytes);
+        TopNCounter<ByteArray> vsNew = serializer.deserialize(in);
+
+        Assert.assertEquals(vs.toString(), vsNew.toString());
+
+    }
+    
+    @Test
+    public void testValueOf() {
+
+        TopNCounter<ByteArray> origin = new TopNCounter<ByteArray>(10);
+        ByteArray key = new ByteArray(1);
+        ByteBuffer byteBuffer = key.asBuffer();
+        BytesUtil.writeVLong(20l, byteBuffer);
+        origin.offer(key, 1.0);
+
+        byteBuffer = ByteBuffer.allocate(1024);
+        byteBuffer.putInt(1);
+        byteBuffer.putInt(20);
+        byteBuffer.putDouble(1.0);
+        TopNCounter<ByteArray> counter = serializer.valueOf(byteBuffer.array());
+
+
+        Assert.assertEquals(origin.toString(), counter.toString());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index fa7d611..f407977 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
@@ -23,15 +23,15 @@ import java.util.Random;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.aggregation.basic.BigDecimalSumAggregator;
+import org.apache.kylin.aggregation.basic.DoubleSumAggregator;
+import org.apache.kylin.aggregation.basic.LongSumAggregator;
+import org.apache.kylin.aggregation.hllc.HLLCAggregator;
 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.DoubleMutable;
-import org.apache.kylin.metadata.measure.DoubleSumAggregator;
-import org.apache.kylin.metadata.measure.HLLCAggregator;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.LongSumAggregator;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
+import org.apache.kylin.common.util.DoubleMutable;
+import org.apache.kylin.common.util.LongMutable;
 import org.junit.Test;
 
 public class AggregationCacheMemSizeTest {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 91e7e18..f5247e2 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java
@@ -29,6 +29,7 @@ import java.util.Map;
 
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.gridtable.CubeCodeSystem;
 import org.apache.kylin.dict.Dictionary;
@@ -44,7 +45,6 @@ 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;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 6395a3b..361617a 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
@@ -27,8 +27,8 @@ import java.util.BitSet;
 import java.util.List;
 
 import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.gridtable.memstore.GTSimpleMemStore;
-import org.apache.kylin.metadata.measure.LongMutable;
 import org.junit.Test;
 
 public class SimpleGridTableTest {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 41d75c9..a964e67 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
@@ -23,20 +23,21 @@ import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 
+import org.apache.kylin.aggregation.basic.StringSerializer;
 import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.LongMutable;
 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;
+
 import it.uniroma3.mat.extendedset.intset.ConciseSet;
 
 public class SimpleInvertedIndexTest {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
index ad5048b..0e13d7e 100644
--- a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
@@ -23,7 +23,10 @@ import static org.junit.Assert.assertEquals;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.util.DoubleMutable;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalMaxAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalMaxAggregator.java
deleted file mode 100644
index 91713a4..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalMaxAggregator.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.metadata.measure;
-
-import java.math.BigDecimal;
-
-/**
- */
-@SuppressWarnings("serial")
-public class BigDecimalMaxAggregator extends MeasureAggregator<BigDecimal> {
-
-    BigDecimal max = null;
-
-    @Override
-    public void reset() {
-        max = null;
-    }
-
-    @Override
-    public void aggregate(BigDecimal value) {
-        if (max == null)
-            max = value;
-        else if (max.compareTo(value) < 0)
-            max = value;
-    }
-
-    @Override
-    public BigDecimal getState() {
-        return max;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessBigDecimalMemBytes();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalMinAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalMinAggregator.java
deleted file mode 100644
index 6bbb6a9..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalMinAggregator.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.metadata.measure;
-
-import java.math.BigDecimal;
-
-/**
- */
-@SuppressWarnings("serial")
-public class BigDecimalMinAggregator extends MeasureAggregator<BigDecimal> {
-
-    BigDecimal max = null;
-
-    @Override
-    public void reset() {
-        max = null;
-    }
-
-    @Override
-    public void aggregate(BigDecimal value) {
-        if (max == null)
-            max = value;
-        else if (max.compareTo(value) > 0)
-            max = value;
-    }
-
-    @Override
-    public BigDecimal getState() {
-        return max;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessBigDecimalMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalSumAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalSumAggregator.java
deleted file mode 100644
index f1c60a1..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/BigDecimalSumAggregator.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.metadata.measure;
-
-import java.math.BigDecimal;
-
-/**
- */
-@SuppressWarnings("serial")
-public class BigDecimalSumAggregator extends MeasureAggregator<BigDecimal> {
-
-    BigDecimal sum = new BigDecimal(0);
-
-    @Override
-    public void reset() {
-        sum = new BigDecimal(0);
-    }
-
-    @Override
-    public void aggregate(BigDecimal value) {
-        sum = sum.add(value);
-    }
-
-    @Override
-    public BigDecimal getState() {
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessBigDecimalMemBytes();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMaxAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMaxAggregator.java
deleted file mode 100644
index 962cd75..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMaxAggregator.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.metadata.measure;
-
-/**
- */
-@SuppressWarnings("serial")
-public class DoubleMaxAggregator extends MeasureAggregator<DoubleMutable> {
-
-    DoubleMutable max = null;
-
-    @Override
-    public void reset() {
-        max = null;
-    }
-
-    @Override
-    public void aggregate(DoubleMutable value) {
-        if (max == null)
-            max = new DoubleMutable(value.get());
-        else if (max.get() < value.get())
-            max.set(value.get());
-    }
-
-    @Override
-    public DoubleMutable getState() {
-        return max;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessDoubleMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMinAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMinAggregator.java
deleted file mode 100644
index f8c3680..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMinAggregator.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.metadata.measure;
-
-/**
- */
-@SuppressWarnings("serial")
-public class DoubleMinAggregator extends MeasureAggregator<DoubleMutable> {
-
-    DoubleMutable min = null;
-
-    @Override
-    public void reset() {
-        min = null;
-    }
-
-    @Override
-    public void aggregate(DoubleMutable value) {
-        if (min == null)
-            min = new DoubleMutable(value.get());
-        else if (min.get() > value.get())
-            min.set(value.get());
-    }
-
-    @Override
-    public DoubleMutable getState() {
-        return min;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessDoubleMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMutable.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMutable.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMutable.java
deleted file mode 100644
index 10072a6..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleMutable.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.metadata.measure;
-
-import java.io.Serializable;
-
-@SuppressWarnings("serial")
-public class DoubleMutable implements Comparable<DoubleMutable>, Serializable {
-
-    private double v;
-
-    public DoubleMutable() {
-        this(0);
-    }
-
-    public DoubleMutable(double v) {
-        set(v);
-    }
-
-    public double get() {
-        return v;
-    }
-
-    public void set(double v) {
-        this.v = v;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof DoubleMutable)) {
-            return false;
-        }
-        DoubleMutable other = (DoubleMutable) o;
-        return this.v == other.v;
-    }
-
-    @Override
-    public int hashCode() {
-        return (int) Double.doubleToLongBits(v);
-    }
-
-    @Override
-    public int compareTo(DoubleMutable o) {
-        return (v < o.v ? -1 : (v == o.v ? 0 : 1));
-    }
-
-    @Override
-    public String toString() {
-        return Double.toString(v);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleSumAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleSumAggregator.java
deleted file mode 100644
index 7593cac..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/DoubleSumAggregator.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.metadata.measure;
-
-/**
- */
-@SuppressWarnings("serial")
-public class DoubleSumAggregator extends MeasureAggregator<DoubleMutable> {
-
-    DoubleMutable sum = new DoubleMutable();
-
-    @Override
-    public void reset() {
-        sum.set(0.0);
-    }
-
-    @Override
-    public void aggregate(DoubleMutable value) {
-        sum.set(sum.get() + value.get());
-    }
-
-    @Override
-    public DoubleMutable getState() {
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessDoubleMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/HLLCAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/HLLCAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/HLLCAggregator.java
deleted file mode 100644
index f76cf64..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/HLLCAggregator.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.metadata.measure;
-
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-
-/**
- */
-@SuppressWarnings("serial")
-public class HLLCAggregator extends MeasureAggregator<HyperLogLogPlusCounter> {
-
-    final int precision;
-    HyperLogLogPlusCounter sum = null;
-
-    public HLLCAggregator(int precision) {
-        this.precision = precision;
-    }
-
-    @Override
-    public void reset() {
-        sum = null;
-    }
-
-    @Override
-    public void aggregate(HyperLogLogPlusCounter value) {
-        if (sum == null)
-            sum = new HyperLogLogPlusCounter(value);
-        else
-            sum.merge(value);
-    }
-
-    @Override
-    public HyperLogLogPlusCounter getState() {
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        // 1024 + 60 returned by AggregationCacheMemSizeTest
-        return 8 // aggregator obj shell
-                + 4 // precision
-                + 8 // ref to HLLC
-                + 8 // HLLC obj shell
-                + 32 + (1 << precision); // HLLC internal
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LDCAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LDCAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LDCAggregator.java
deleted file mode 100644
index e5cf2a9..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LDCAggregator.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.metadata.measure;
-
-/**
- * Long Distinct Count
- */
-@SuppressWarnings("serial")
-public class LDCAggregator extends MeasureAggregator<LongMutable> {
-
-    private static LongMutable ZERO = new LongMutable(0);
-
-    private HLLCAggregator hllAgg = null;
-    private LongMutable state = new LongMutable(0);
-
-    @SuppressWarnings("rawtypes")
-    public void setDependentAggregator(MeasureAggregator agg) {
-        this.hllAgg = (HLLCAggregator) agg;
-    }
-
-    @Override
-    public void reset() {
-    }
-
-    @Override
-    public void aggregate(LongMutable value) {
-    }
-
-    @Override
-    public LongMutable getState() {
-        if (hllAgg == null) {
-            return ZERO;
-        } else {
-            state.set(hllAgg.getState().getCountEstimate());
-            return state;
-        }
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessLongMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMaxAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMaxAggregator.java
deleted file mode 100644
index 5b080bb..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMaxAggregator.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.metadata.measure;
-
-/**
- */
-@SuppressWarnings("serial")
-public class LongMaxAggregator extends MeasureAggregator<LongMutable> {
-
-    LongMutable max = null;
-
-    @Override
-    public void reset() {
-        max = null;
-    }
-
-    @Override
-    public void aggregate(LongMutable value) {
-        if (max == null)
-            max = new LongMutable(value.get());
-        else if (max.get() < value.get())
-            max.set(value.get());
-    }
-
-    @Override
-    public LongMutable getState() {
-        return max;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessLongMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMinAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMinAggregator.java
deleted file mode 100644
index a2509ac..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMinAggregator.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.metadata.measure;
-
-/**
- */
-@SuppressWarnings("serial")
-public class LongMinAggregator extends MeasureAggregator<LongMutable> {
-
-    LongMutable min = null;
-
-    @Override
-    public void reset() {
-        min = null;
-    }
-
-    @Override
-    public void aggregate(LongMutable value) {
-        if (min == null)
-            min = new LongMutable(value.get());
-        else if (min.get() > value.get())
-            min.set(value.get());
-    }
-
-    @Override
-    public LongMutable getState() {
-        return min;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessLongMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMutable.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMutable.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMutable.java
deleted file mode 100644
index 220b6a6..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongMutable.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.metadata.measure;
-
-import java.io.Serializable;
-
-@SuppressWarnings("serial")
-public class LongMutable implements Comparable<LongMutable>, Serializable {
-
-    private long v;
-
-    public LongMutable() {
-        this(0);
-    }
-
-    public LongMutable(long v) {
-        set(v);
-    }
-
-    public long get() {
-        return v;
-    }
-
-    public void set(long v) {
-        this.v = v;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof LongMutable)) {
-            return false;
-        }
-        LongMutable other = (LongMutable) o;
-        return this.v == other.v;
-    }
-
-    @Override
-    public int hashCode() {
-        return (int) v;
-    }
-
-    @Override
-    public int compareTo(LongMutable o) {
-        long thisValue = this.v;
-        long thatValue = o.v;
-        return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
-    }
-
-    @Override
-    public String toString() {
-        return Long.toString(v);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongSumAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongSumAggregator.java
deleted file mode 100644
index 63422f5..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/LongSumAggregator.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.metadata.measure;
-
-/**
- */
-@SuppressWarnings("serial")
-public class LongSumAggregator extends MeasureAggregator<LongMutable> {
-
-    LongMutable sum = new LongMutable();
-
-    @Override
-    public void reset() {
-        sum.set(0);
-    }
-
-    @Override
-    public void aggregate(LongMutable value) {
-        sum.set(sum.get() + value.get());
-    }
-
-    @Override
-    public LongMutable getState() {
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessLongMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureAggregator.java
deleted file mode 100644
index 1b38aa5..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureAggregator.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.metadata.measure;
-
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.metadata.model.FunctionDesc;
-
-import java.io.Serializable;
-
-/**
- */
-@SuppressWarnings("serial")
-abstract public class MeasureAggregator<V> implements Serializable {
-
-    public static MeasureAggregator<?> create(String funcName, String returnType) {
-        if (FunctionDesc.FUNC_SUM.equalsIgnoreCase(funcName) || FunctionDesc.FUNC_COUNT.equalsIgnoreCase(funcName)) {
-            if (isInteger(returnType))
-                return new LongSumAggregator();
-            else if (isBigDecimal(returnType))
-                return new BigDecimalSumAggregator();
-            else if (isDouble(returnType))
-                return new DoubleSumAggregator();
-        } else if (FunctionDesc.FUNC_COUNT_DISTINCT.equalsIgnoreCase(funcName)) {
-            DataType hllcType = DataType.getInstance(returnType);
-            if (hllcType.isHLLC())
-                return new HLLCAggregator(hllcType.getPrecision());
-            else
-                return new LDCAggregator();
-        } else if (FunctionDesc.FUNC_MAX.equalsIgnoreCase(funcName)) {
-            if (isInteger(returnType))
-                return new LongMaxAggregator();
-            else if (isBigDecimal(returnType))
-                return new BigDecimalMaxAggregator();
-            else if (isDouble(returnType))
-                return new DoubleMaxAggregator();
-        } else if (FunctionDesc.FUNC_MIN.equalsIgnoreCase(funcName)) {
-            if (isInteger(returnType))
-                return new LongMinAggregator();
-            else if (isBigDecimal(returnType))
-                return new BigDecimalMinAggregator();
-            else if (isDouble(returnType))
-                return new DoubleMinAggregator();
-        } else if (FunctionDesc.FUNC_TOP_N.equalsIgnoreCase(funcName)) {
-            return new TopNAggregator();
-        }
-        throw new IllegalArgumentException("No aggregator for func '" + funcName + "' and return type '" + returnType + "'");
-    }
-
-    public static boolean isBigDecimal(String type) {
-        return type.startsWith("decimal");
-    }
-
-    public static boolean isDouble(String type) {
-        return "double".equalsIgnoreCase(type) || "float".equalsIgnoreCase(type) || "real".equalsIgnoreCase(type);
-    }
-
-    public static boolean isInteger(String type) {
-        return "long".equalsIgnoreCase(type) || "bigint".equalsIgnoreCase(type) || "int".equalsIgnoreCase(type) || "integer".equalsIgnoreCase(type);
-    }
-
-    public static int guessBigDecimalMemBytes() {
-        // 116 returned by AggregationCacheMemSizeTest
-        return 8 // aggregator obj shell
-        + 8 // ref to BigDecimal
-        + 8 // BigDecimal obj shell
-        + 100; // guess of BigDecimal internal
-    }
-
-    public static int guessDoubleMemBytes() {
-        // 29 to 44 returned by AggregationCacheMemSizeTest
-        return 44;
-        /*
-        return 8 // aggregator obj shell
-        + 8 // ref to DoubleWritable
-        + 8 // DoubleWritable obj shell
-        + 8; // size of double
-        */
-    }
-
-    public static int guessLongMemBytes() {
-        // 29 to 44 returned by AggregationCacheMemSizeTest
-        return 44;
-        /*
-        return 8 // aggregator obj shell
-        + 8 // ref to LongWritable
-        + 8 // LongWritable obj shell
-        + 8; // size of long
-        */
-    }
-
-    // ============================================================================
-
-    @SuppressWarnings("rawtypes")
-    public void setDependentAggregator(MeasureAggregator agg) {
-    }
-
-    abstract public void reset();
-
-    abstract public void aggregate(V value);
-
-    abstract public V getState();
-
-    // get an estimate of memory consumption UPPER BOUND
-    abstract public int getMemBytesEstimate();
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureAggregators.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureAggregators.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureAggregators.java
deleted file mode 100644
index 09e553e..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureAggregators.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.metadata.measure;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-/**
- */
-@SuppressWarnings({ "rawtypes", "unchecked", "serial" })
-public class MeasureAggregators implements Serializable {
-
-    private final MeasureAggregator[] aggs;
-    private final int descLength;
-
-    public MeasureAggregators(Collection<MeasureDesc> measureDescs) {
-        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
-    }
-
-    public MeasureAggregators(MeasureDesc... measureDescs) {
-        descLength = measureDescs.length;
-        aggs = new MeasureAggregator[descLength];
-
-        Map<String, Integer> measureIndexMap = new HashMap<String, Integer>();
-        for (int i = 0; i < descLength; i++) {
-            FunctionDesc func = measureDescs[i].getFunction();
-            aggs[i] = MeasureAggregator.create(func.getExpression(), func.getReturnType());
-            measureIndexMap.put(measureDescs[i].getName(), i);
-        }
-        // fill back dependent aggregator
-        for (int i = 0; i < descLength; i++) {
-            String depMsrRef = measureDescs[i].getDependentMeasureRef();
-            if (depMsrRef != null) {
-                int index = measureIndexMap.get(depMsrRef);
-                aggs[i].setDependentAggregator(aggs[index]);
-            }
-        }
-    }
-
-    public void reset() {
-        for (int i = 0; i < aggs.length; i++) {
-            aggs[i].reset();
-        }
-    }
-    
-    public void aggregate(Object[] values) {
-        assert values.length == descLength;
-
-        for (int i = 0; i < descLength; i++) {
-            aggs[i].aggregate(values[i]);
-        }
-    }
-
-    public void collectStates(Object[] states) {
-        for (int i = 0; i < descLength; i++) {
-            states[i] = aggs[i].getState();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureCodec.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureCodec.java
deleted file mode 100644
index 296290a..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/MeasureCodec.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.metadata.measure;
-
-import java.nio.ByteBuffer;
-import java.util.Collection;
-
-import org.apache.kylin.metadata.measure.serializer.DataTypeSerializer;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-/**
- * @author yangli9
- * 
- */
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class MeasureCodec {
-
-    int nMeasures;
-    DataTypeSerializer[] serializers;
-
-    public MeasureCodec(Collection<MeasureDesc> measureDescs) {
-        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
-    }
-
-    public MeasureCodec(MeasureDesc... measureDescs) {
-        String[] dataTypes = new String[measureDescs.length];
-        for (int i = 0; i < dataTypes.length; i++) {
-            dataTypes[i] = measureDescs[i].getFunction().getReturnType();
-        }
-        init(dataTypes);
-    }
-
-    public MeasureCodec(String... dataTypes) {
-        init(dataTypes);
-    }
-
-    private void init(String[] dataTypes) {
-        nMeasures = dataTypes.length;
-        serializers = new DataTypeSerializer[nMeasures];
-
-        for (int i = 0; i < nMeasures; i++) {
-            serializers[i] = DataTypeSerializer.create(dataTypes[i]);
-        }
-    }
-
-    public DataTypeSerializer getSerializer(int idx) {
-        return serializers[idx];
-    }
-
-    public void decode(ByteBuffer buf, Object[] result) {
-        assert result.length == nMeasures;
-        for (int i = 0; i < nMeasures; i++) {
-            result[i] = serializers[i].deserialize(buf);
-        }
-    }
-
-    public void encode(Object[] values, ByteBuffer out) {
-        assert values.length == nMeasures;
-        for (int i = 0; i < nMeasures; i++) {
-            serializers[i].serialize(values[i], out);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/TopNAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/TopNAggregator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/TopNAggregator.java
deleted file mode 100644
index 9c21f7b..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/TopNAggregator.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.metadata.measure;
-
-import com.google.common.collect.Maps;
-import org.apache.kylin.common.topn.Counter;
-import org.apache.kylin.common.topn.TopNCounter;
-import org.apache.kylin.common.util.ByteArray;
-
-import java.util.Iterator;
-import java.util.Map;
-
-/**
- * 
- */
-@SuppressWarnings("serial")
-public class TopNAggregator extends MeasureAggregator<TopNCounter<ByteArray>> {
-
-    int capacity = 0;
-    TopNCounter<ByteArray> sum = null;
-    Map<ByteArray, Double> sanityCheckMap;
-
-    @Override
-    public void reset() {
-        sum = null;
-    }
-
-    @Override
-    public void aggregate(TopNCounter<ByteArray> value) {
-        if (sum == null) {
-            capacity = value.getCapacity();
-            sum = new TopNCounter<>(capacity);
-            sanityCheckMap = Maps.newHashMap();
-        }
-        sum.merge(value);
-    }
-
-    @Override
-    public TopNCounter<ByteArray> getState() {
-        
-        //sum.retain(capacity);
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return 8 * capacity / 4;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedHLLCodec.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedHLLCodec.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedHLLCodec.java
deleted file mode 100644
index a0af6a2..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedHLLCodec.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.metadata.measure.fixedlen;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.metadata.model.DataType;
-
-/**
- */
-public class FixedHLLCodec extends FixedLenMeasureCodec<HyperLogLogPlusCounter> {
-
-    private DataType type;
-    private int presision;
-    private HyperLogLogPlusCounter current;
-
-    public FixedHLLCodec(DataType type) {
-        this.type = type;
-        this.presision = type.getPrecision();
-        this.current = new HyperLogLogPlusCounter(this.presision);
-    }
-
-    @Override
-    public int getLength() {
-        return 1 << presision;
-    }
-
-    @Override
-    public DataType getDataType() {
-        return type;
-    }
-
-    @Override
-    public HyperLogLogPlusCounter valueOf(String value) {
-        current.clear();
-        if (value == null)
-            current.add("__nUlL__");
-        else
-            current.add(value.getBytes());
-        return current;
-    }
-
-    @Override
-    public Object getValue() {
-        return current;
-    }
-
-    @Override
-    public HyperLogLogPlusCounter read(byte[] buf, int offset) {
-        current.readRegistersArray(ByteBuffer.wrap(buf, offset, buf.length - offset));
-        return current;
-    }
-
-    @Override
-    public void write(HyperLogLogPlusCounter v, byte[] buf, int offset) {
-        v.writeRegistersArray(ByteBuffer.wrap(buf, offset, buf.length - offset));
-    }
-
-    @Override
-    public HyperLogLogPlusCounter read(ByteBuffer buffer) {
-        current.readRegistersArray(buffer);
-        return current;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedLenMeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedLenMeasureCodec.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedLenMeasureCodec.java
deleted file mode 100644
index 54c4eb8..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedLenMeasureCodec.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.metadata.measure.fixedlen;
-
-import org.apache.kylin.metadata.model.DataType;
-
-import java.nio.ByteBuffer;
-
-abstract public class FixedLenMeasureCodec<T> {
-
-    public static FixedLenMeasureCodec<?> get(DataType type) {
-        if (type.isHLLC()) {
-            return new FixedHLLCodec(type);
-        } else {
-            return new FixedPointLongCodec(type);
-        }
-    }
-
-    abstract public int getLength();
-
-    abstract public DataType getDataType();
-
-    abstract public T valueOf(String value);
-
-    abstract public Object getValue();
-
-    abstract public T read(byte[] buf, int offset);
-
-    abstract public void write(T v, byte[] buf, int offset);
-
-    abstract public T read(ByteBuffer buffer);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodec.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodec.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodec.java
deleted file mode 100644
index 37eb0be..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodec.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.metadata.measure.fixedlen;
-
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.model.DataType;
-
-import java.nio.ByteBuffer;
-
-public class FixedPointLongCodec extends FixedLenMeasureCodec<LongMutable> {
-
-    private static final int SIZE = 8;
-    // number of digits after decimal point
-    int scale;
-    DataType type;
-    // avoid massive object creation
-    LongMutable current = new LongMutable();
-
-    public FixedPointLongCodec(DataType type) {
-        this.type = type;
-        this.scale = Math.max(0, type.getScale());
-    }
-
-    @Override
-    public int getLength() {
-        return SIZE;
-    }
-
-    @Override
-    public DataType getDataType() {
-        return type;
-    }
-
-    long getValueIgnoringDecimalPoint(String value) {
-        int index = value.indexOf('.');
-
-        if (index == 0 || index == value.length() - 1) {
-            throw new RuntimeException("Bad decimal format: " + value);
-        } else if (index < 0) {
-            return Long.valueOf(value) * (int) Math.pow(10, scale);
-        } else {
-            StringBuilder sb = new StringBuilder();
-            sb.append(value.substring(0, index));
-
-            //if there are more than scale digits after the decimal point, the tail will be discarded
-            int end = Math.min(value.length(), index + scale + 1);
-            sb.append(value.substring(index + 1, end));
-            int diff = index + scale + 1 - value.length();
-            //if there are less than scale digits after the decimal point, the tail will be compensated
-            for (int i = 0; i < diff; i++) {
-                sb.append('0');
-            }
-            return Long.valueOf(sb.toString());
-        }
-    }
-
-    String restoreDecimalPoint(long value) {
-        if (scale < 0) {
-            throw new RuntimeException("Bad scale: " + scale + " with value: " + value);
-        } else if (scale == 0) {
-            return Long.toString(value);
-        } else {
-            return String.format("%." + scale + "f", value / (Math.pow(10, scale)));
-        }
-    }
-
-    @Override
-    public LongMutable valueOf(String value) {
-        if (value == null)
-            current.set(0L);
-        else
-            current.set(getValueIgnoringDecimalPoint(value));
-        return current;
-    }
-
-    @Override
-    public String getValue() {
-        if (scale == 0)
-            return current.toString();
-        else
-            return restoreDecimalPoint(current.get());
-    }
-
-    @Override
-    public LongMutable read(byte[] buf, int offset) {
-        current.set(BytesUtil.readLong(buf, offset, SIZE));
-        return current;
-    }
-
-    @Override
-    public void write(LongMutable v, byte[] buf, int offset) {
-        BytesUtil.writeLong(v == null ? 0 : v.get(), buf, offset, SIZE);
-    }
-
-    @Override
-    public LongMutable read(ByteBuffer buffer) {
-        current.set(BytesUtil.readLong(buffer, SIZE));
-        return current;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/BigDecimalSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/BigDecimalSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/BigDecimalSerializer.java
deleted file mode 100644
index d10c565..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/BigDecimalSerializer.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.metadata.measure.serializer;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.model.DataType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author yangli9
- * 
- */
-public class BigDecimalSerializer extends DataTypeSerializer<BigDecimal> {
-
-    private static final Logger logger = LoggerFactory.getLogger(BigDecimalSerializer.class);
-
-    final DataType type;
-    final int maxLength;
-
-    int avoidVerbose = 0;
-
-    public BigDecimalSerializer(DataType type) {
-        this.type = type;
-        // see serialize(): 1 byte scale, 1 byte length, assume every 2 digits takes 1 byte
-        this.maxLength = 1 + 1 + (type.getPrecision() + 1) / 2;
-    }
-
-    @Override
-    public void serialize(BigDecimal value, ByteBuffer out) {
-        if (value.scale() > type.getScale()) {
-            if (avoidVerbose % 10000 == 0) {
-                logger.warn("value's scale has exceeded the " + type.getScale() + ", cut it off, to ensure encoded value do not exceed maxLength " + maxLength + " times:" + (avoidVerbose++));
-            }
-            value = value.setScale(type.getScale(), BigDecimal.ROUND_HALF_EVEN);
-        }
-        byte[] bytes = value.unscaledValue().toByteArray();
-        if (bytes.length + 2 > maxLength) {
-            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
-        }
-
-        BytesUtil.writeVInt(value.scale(), out);
-        BytesUtil.writeVInt(bytes.length, out);
-        out.put(bytes);
-    }
-
-    @Override
-    public BigDecimal deserialize(ByteBuffer in) {
-        int scale = BytesUtil.readVInt(in);
-        int n = BytesUtil.readVInt(in);
-
-        byte[] bytes = new byte[n];
-        in.get(bytes);
-
-        return new BigDecimal(new BigInteger(bytes), scale);
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-
-        @SuppressWarnings("unused")
-        int scale = BytesUtil.readVInt(in);
-        int n = BytesUtil.readVInt(in);
-        int len = in.position() - mark + n;
-
-        in.position(mark);
-        return len;
-    }
-
-    @Override
-    public int maxLength() {
-        return maxLength;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 8;
-    }
-
-    @Override
-    public BigDecimal valueOf(byte[] value) {
-        if (value == null)
-            return new BigDecimal(0);
-        else
-            return new BigDecimal(Bytes.toString(value));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DataTypeSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DataTypeSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DataTypeSerializer.java
deleted file mode 100644
index d542098..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DataTypeSerializer.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.metadata.measure.serializer;
-
-import java.io.UnsupportedEncodingException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.kylin.common.util.BytesSerializer;
-import org.apache.kylin.metadata.model.DataType;
-
-import com.google.common.collect.Maps;
-
-/**
- * @author yangli9
- * 
- * Note: the implementations MUST be thread-safe.
- * 
- */
-abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
-
-    final static Map<String, Class<?>> implementations;
-    static {
-        HashMap<String, Class<?>> impl = Maps.newHashMap();
-        impl.put("varchar", StringSerializer.class);
-        impl.put("decimal", BigDecimalSerializer.class);
-        impl.put("double", DoubleSerializer.class);
-        impl.put("float", DoubleSerializer.class);
-        impl.put("bigint", LongSerializer.class);
-        impl.put("long", LongSerializer.class);
-        impl.put("integer", LongSerializer.class);
-        impl.put("int", LongSerializer.class);
-        impl.put("smallint", LongSerializer.class);
-        impl.put("date", DateTimeSerializer.class);
-        impl.put("datetime", DateTimeSerializer.class);
-        impl.put("timestamp", DateTimeSerializer.class);
-        implementations = Collections.unmodifiableMap(impl);
-
-    }
-
-    public static DataTypeSerializer<?> create(String dataType) {
-        return create(DataType.getInstance(dataType));
-    }
-
-    public static DataTypeSerializer<?> create(DataType type) {
-        if (type.isHLLC()) {
-            return new HLLCSerializer(type);
-        }
-
-        if (type.isTopN()) {
-            return new TopNCounterSerializer(type);
-        }
-
-        Class<?> clz = implementations.get(type.getName());
-        if (clz == null)
-            throw new RuntimeException("No MeasureSerializer for type " + type);
-
-        try {
-            return (DataTypeSerializer<?>) clz.getConstructor(DataType.class).newInstance(type);
-        } catch (Exception e) {
-            throw new RuntimeException(e); // never happen
-        }
-    }
-    
-    /** peek into buffer and return the length of serialization */
-    abstract public int peekLength(ByteBuffer in);
-
-    /** return the max number of bytes to the longest serialization */
-    abstract public int maxLength();
-
-    /** get an estimate of size in bytes of the serialized data */
-    abstract public int getStorageBytesEstimate();
-
-    /** convert from String to obj (string often come as byte[] in mapred) */
-    abstract public T valueOf(byte[] value);
-
-    /** convert from String to obj */
-    public T valueOf(String value) {
-        try {
-            return valueOf(value.getBytes("UTF-8"));
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException(e); // never happen
-        }
-    }
-
-    /** convert from obj to string */
-    public String toString(T value) {
-        if (value == null)
-            return "NULL";
-        else
-            return value.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DateTimeSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DateTimeSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DateTimeSerializer.java
deleted file mode 100644
index d79c382..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DateTimeSerializer.java
+++ /dev/null
@@ -1,64 +0,0 @@
-package org.apache.kylin.metadata.measure.serializer;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.model.DataType;
-
-public class DateTimeSerializer extends DataTypeSerializer<LongMutable> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
-
-    public DateTimeSerializer(DataType type) {
-    }
-
-    @Override
-    public void serialize(LongMutable value, ByteBuffer out) {
-        out.putLong(value.get());
-    }
-
-    private LongMutable current() {
-        LongMutable l = current.get();
-        if (l == null) {
-            l = new LongMutable();
-            current.set(l);
-        }
-        return l;
-    }
-
-    @Override
-    public LongMutable deserialize(ByteBuffer in) {
-        LongMutable l = current();
-        l.set(in.getLong());
-        return l;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return 8;
-    }
-
-    @Override
-    public int maxLength() {
-        return 8;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 8;
-    }
-
-    @Override
-    public LongMutable valueOf(byte[] value) {
-        LongMutable l = current();
-        if (value == null)
-            l.set(0L);
-        else
-            l.set(DateFormat.stringToMillis(Bytes.toString(value)));
-        return l;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DoubleSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DoubleSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DoubleSerializer.java
deleted file mode 100644
index 2dc8a67..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/DoubleSerializer.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.metadata.measure.serializer;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.metadata.measure.DoubleMutable;
-import org.apache.kylin.metadata.model.DataType;
-
-/**
- */
-public class DoubleSerializer extends DataTypeSerializer<DoubleMutable> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<DoubleMutable> current = new ThreadLocal<DoubleMutable>();
-
-    public DoubleSerializer(DataType type) {
-    }
-
-    @Override
-    public void serialize(DoubleMutable value, ByteBuffer out) {
-        out.putDouble(value.get());
-    }
-
-    private DoubleMutable current() {
-        DoubleMutable d = current.get();
-        if (d == null) {
-            d = new DoubleMutable();
-            current.set(d);
-        }
-        return d;
-    }
-
-    @Override
-    public DoubleMutable deserialize(ByteBuffer in) {
-        DoubleMutable d = current();
-        d.set(in.getDouble());
-        return d;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return 8;
-    }
-
-    @Override
-    public int maxLength() {
-        return 8;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 8;
-    }
-
-    @Override
-    public DoubleMutable valueOf(byte[] value) {
-        DoubleMutable d = current();
-        if (value == null)
-            d.set(0d);
-        else
-            d.set(Double.parseDouble(Bytes.toString(value)));
-        return d;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/HLLCSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/HLLCSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/HLLCSerializer.java
deleted file mode 100644
index 7aed458..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/HLLCSerializer.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.metadata.measure.serializer;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.metadata.model.DataType;
-
-/**
- * @author yangli9
- * 
- */
-public class HLLCSerializer extends DataTypeSerializer<HyperLogLogPlusCounter> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<HyperLogLogPlusCounter> current = new ThreadLocal<HyperLogLogPlusCounter>();
-
-    private int precision;
-
-    public HLLCSerializer(DataType type) {
-        this.precision = type.getPrecision();
-    }
-
-    @Override
-    public void serialize(HyperLogLogPlusCounter value, ByteBuffer out) {
-        try {
-            value.writeRegisters(out);
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private HyperLogLogPlusCounter current() {
-        HyperLogLogPlusCounter hllc = current.get();
-        if (hllc == null) {
-            hllc = new HyperLogLogPlusCounter(precision);
-            current.set(hllc);
-        }
-        return hllc;
-    }
-
-    @Override
-    public HyperLogLogPlusCounter deserialize(ByteBuffer in) {
-        HyperLogLogPlusCounter hllc = current();
-        try {
-            hllc.readRegisters(in);
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-        return hllc;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return current().peekLength(in);
-    }
-
-    @Override
-    public int maxLength() {
-        return current().maxLength();
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return current().maxLength();
-    }
-
-    @Override
-    public HyperLogLogPlusCounter valueOf(byte[] value) {
-        HyperLogLogPlusCounter hllc = current();
-        hllc.clear();
-        if (value == null)
-            hllc.add("__nUlL__");
-        else
-            hllc.add(value);
-        return hllc;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/LongSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/LongSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/LongSerializer.java
deleted file mode 100644
index f51a52c..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/LongSerializer.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.metadata.measure.serializer;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.model.DataType;
-
-/**
- */
-public class LongSerializer extends DataTypeSerializer<LongMutable> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
-
-    public LongSerializer(DataType type) {
-    }
-
-    @Override
-    public void serialize(LongMutable value, ByteBuffer out) {
-        BytesUtil.writeVLong(value.get(), out);
-    }
-
-    private LongMutable current() {
-        LongMutable l = current.get();
-        if (l == null) {
-            l = new LongMutable();
-            current.set(l);
-        }
-        return l;
-    }
-
-    @Override
-    public LongMutable deserialize(ByteBuffer in) {
-        LongMutable l = current();
-        l.set(BytesUtil.readVLong(in));
-        return l;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-
-        BytesUtil.readVLong(in);
-        int len = in.position() - mark;
-
-        in.position(mark);
-        return len;
-    }
-
-    @Override
-    public int maxLength() {
-        return 9; // vlong: 1 + 8
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 5;
-    }
-
-    @Override
-    public LongMutable valueOf(byte[] value) {
-        LongMutable l = current();
-        if (value == null)
-            l.set(0L);
-        else
-            l.set(Long.parseLong(Bytes.toString(value)));
-        return l;
-    }
-
-}


[02/13] incubator-kylin git commit: KYLIN-976 very initial

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/StringSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/StringSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/StringSerializer.java
deleted file mode 100644
index db27ca0..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/StringSerializer.java
+++ /dev/null
@@ -1,55 +0,0 @@
-package org.apache.kylin.metadata.measure.serializer;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.model.DataType;
-
-public class StringSerializer extends DataTypeSerializer<String> {
-
-    final DataType type;
-    final int maxLength;
-
-    public StringSerializer(DataType type) {
-        this.type = type;
-        // see serialize(): 2 byte length, rest is String.toBytes()
-        this.maxLength = 2 + type.getPrecision();
-    }
-
-    @Override
-    public void serialize(String value, ByteBuffer out) {
-        int start = out.position();
-
-        BytesUtil.writeUTFString(value, out);
-
-        if (out.position() - start > maxLength)
-            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
-    }
-
-    @Override
-    public String deserialize(ByteBuffer in) {
-        return BytesUtil.readUTFString(in);
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return BytesUtil.peekByteArrayLength(in);
-    }
-
-    @Override
-    public int maxLength() {
-        return maxLength;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return maxLength;
-    }
-
-    @Override
-    public String valueOf(byte[] value) {
-        return Bytes.toString(value);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/TopNCounterSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/TopNCounterSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/TopNCounterSerializer.java
deleted file mode 100644
index 468d077..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/measure/serializer/TopNCounterSerializer.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.metadata.measure.serializer;
-
-import org.apache.kylin.common.topn.Counter;
-import org.apache.kylin.common.topn.DoubleDeltaSerializer;
-import org.apache.kylin.common.topn.TopNCounter;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.model.DataType;
-
-import java.nio.ByteBuffer;
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * 
- */
-public class TopNCounterSerializer extends DataTypeSerializer<TopNCounter<ByteArray>> {
-
-    private DoubleDeltaSerializer dds = new DoubleDeltaSerializer(3);
-
-    private int precision;
-
-    public TopNCounterSerializer(DataType dataType) {
-        this.precision = dataType.getPrecision();
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-        @SuppressWarnings("unused")
-        int capacity = in.getInt();
-        int size = in.getInt();
-        int keyLength = in.getInt();
-        dds.deserialize(in);
-        int len = in.position() - mark + keyLength * size;
-        in.position(mark);
-        return len;
-    }
-
-    @Override
-    public int maxLength() {
-        return precision * TopNCounter.EXTRA_SPACE_RATE * (4 + 8);
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return precision * TopNCounter.EXTRA_SPACE_RATE * 8;
-    }
-
-    @Override
-    public TopNCounter<ByteArray> valueOf(byte[] value) {
-        ByteBuffer buffer = ByteBuffer.wrap(value);
-        int sizeOfId = buffer.getInt();
-        int keyEncodedValue = buffer.getInt();
-        double counter = buffer.getDouble();
-
-        ByteArray key = new ByteArray(sizeOfId);
-        BytesUtil.writeUnsigned(keyEncodedValue, key.array(), 0, sizeOfId);
-
-        TopNCounter<ByteArray> topNCounter = new TopNCounter<ByteArray>(precision * TopNCounter.EXTRA_SPACE_RATE);
-        topNCounter.offer(key, counter);
-        return topNCounter;
-    }
-
-    @Override
-    public void serialize(TopNCounter<ByteArray> value, ByteBuffer out) {
-        double[] counters = value.getCounters();
-        List<ByteArray> peek = value.peek(1);
-        int keyLength = peek.size() > 0 ? peek.get(0).length() : 0;
-        out.putInt(value.getCapacity());
-        out.putInt(value.size());
-        out.putInt(keyLength);
-        dds.serialize(counters, out);
-        Iterator<Counter<ByteArray>> iterator = value.iterator();
-        while (iterator.hasNext()) {
-            out.put(iterator.next().getItem().array());
-        }
-    }
-
-    @Override
-    public TopNCounter<ByteArray> deserialize(ByteBuffer in) {
-        int capacity = in.getInt();
-        int size = in.getInt();
-        int keyLength = in.getInt();
-        double[] counters = dds.deserialize(in);
-
-        TopNCounter<ByteArray> counter = new TopNCounter<ByteArray>(capacity);
-        ByteArray byteArray;
-        for (int i = 0; i < size; i++) {
-            byteArray = new ByteArray(keyLength);
-            in.get(byteArray.array());
-            counter.offerToHead(byteArray, counters[i]);
-        }
-
-        return counter;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java
index 590ef3d..1333426 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java
@@ -28,7 +28,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.kylin.metadata.measure.serializer.DataTypeSerializer;
+import org.apache.kylin.aggregation.DataTypeSerializer;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/test/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodecTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodecTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodecTest.java
deleted file mode 100644
index 6ba7115..0000000
--- a/core-metadata/src/test/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodecTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-package org.apache.kylin.metadata.measure.fixedlen;
-
-import org.apache.kylin.metadata.model.DataType;
-import org.junit.Test;
-
-/**
- */
-public class FixedPointLongCodecTest {
-
-    @Test
-    public void testEncode1() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
-        long x = codec.getValueIgnoringDecimalPoint("12.12345");
-        org.junit.Assert.assertEquals(1212345, x);
-    }
-
-    @Test
-    public void testEncode2() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
-        long x = codec.getValueIgnoringDecimalPoint("12.1234");
-        org.junit.Assert.assertEquals(1212340, x);
-    }
-
-    @Test
-    public void testEncode3() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
-        long x = codec.getValueIgnoringDecimalPoint("12.123456");
-        org.junit.Assert.assertEquals(1212345, x);
-    }
-
-    @Test
-    public void testEncode4() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
-        long x = codec.getValueIgnoringDecimalPoint("12");
-        org.junit.Assert.assertEquals(1200000, x);
-    }
-
-    @Test
-    public void testDecode1() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
-        String x = codec.restoreDecimalPoint(1212345);
-        org.junit.Assert.assertEquals("12.12345", x);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/test/java/org/apache/kylin/metadata/measure/serializer/BigDecimalSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/measure/serializer/BigDecimalSerializerTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/measure/serializer/BigDecimalSerializerTest.java
deleted file mode 100644
index 682bc24..0000000
--- a/core-metadata/src/test/java/org/apache/kylin/metadata/measure/serializer/BigDecimalSerializerTest.java
+++ /dev/null
@@ -1,52 +0,0 @@
-package org.apache.kylin.metadata.measure.serializer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.metadata.model.DataType;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- */
-public class BigDecimalSerializerTest {
-
-    private static BigDecimalSerializer bigDecimalSerializer;
-
-    @BeforeClass
-    public static void beforeClass() {
-        bigDecimalSerializer = new BigDecimalSerializer(DataType.getInstance("decimal"));
-    }
-
-    @Test
-    public void testNormal() {
-        BigDecimal input = new BigDecimal("1234.1234");
-        ByteBuffer buffer = ByteBuffer.allocate(256);
-        buffer.mark();
-        bigDecimalSerializer.serialize(input, buffer);
-        buffer.reset();
-        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
-        assertEquals(input, output);
-    }
-
-    @Test
-    public void testScaleOutOfRange() {
-        BigDecimal input = new BigDecimal("1234.1234567890");
-        ByteBuffer buffer = ByteBuffer.allocate(256);
-        buffer.mark();
-        bigDecimalSerializer.serialize(input, buffer);
-        buffer.reset();
-        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
-        assertEquals(input.setScale(bigDecimalSerializer.type.getScale(), BigDecimal.ROUND_HALF_EVEN), output);
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testOutOfPrecision() {
-        BigDecimal input = new BigDecimal("66855344214907231736.4924");
-        ByteBuffer buffer = ByteBuffer.allocate(256);
-        bigDecimalSerializer.serialize(input, buffer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-metadata/src/test/java/org/apache/kylin/metadata/measure/serializer/TopNCounterSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/measure/serializer/TopNCounterSerializerTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/measure/serializer/TopNCounterSerializerTest.java
deleted file mode 100644
index 3c88446..0000000
--- a/core-metadata/src/test/java/org/apache/kylin/metadata/measure/serializer/TopNCounterSerializerTest.java
+++ /dev/null
@@ -1,61 +0,0 @@
-package org.apache.kylin.metadata.measure.serializer;
-
-import org.apache.kylin.common.topn.TopNCounter;
-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.metadata.model.DataType;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-
-/**
- * 
- */
-public class TopNCounterSerializerTest {
-
-    private static TopNCounterSerializer serializer = new TopNCounterSerializer(DataType.getInstance("topn(10)"));
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testSerialization() {
-        TopNCounter<ByteArray> vs = new TopNCounter<ByteArray>(50);
-        Integer[] stream = { 1, 1, 2, 9, 1, 2, 3, 7, 7, 1, 3, 1, 1 };
-        for (Integer i : stream) {
-            vs.offer(new ByteArray(Bytes.toBytes(i)));
-        }
-
-        ByteBuffer out = ByteBuffer.allocate(1024);
-        serializer.serialize(vs, out);
-        
-        byte[] copyBytes = new byte[out.position()];
-        System.arraycopy(out.array(), 0, copyBytes, 0, out.position());
-
-        ByteBuffer in = ByteBuffer.wrap(copyBytes);
-        TopNCounter<ByteArray> vsNew = serializer.deserialize(in);
-
-        Assert.assertEquals(vs.toString(), vsNew.toString());
-
-    }
-    
-    @Test
-    public void testValueOf() {
-
-        TopNCounter<ByteArray> origin = new TopNCounter<ByteArray>(10);
-        ByteArray key = new ByteArray(1);
-        ByteBuffer byteBuffer = key.asBuffer();
-        BytesUtil.writeVLong(20l, byteBuffer);
-        origin.offer(key, 1.0);
-
-        byteBuffer = ByteBuffer.allocate(1024);
-        byteBuffer.putInt(1);
-        byteBuffer.putInt(20);
-        byteBuffer.putDouble(1.0);
-        TopNCounter<ByteArray> counter = serializer.valueOf(byteBuffer.array());
-
-
-        Assert.assertEquals(origin.toString(), counter.toString());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 11b03bd..da009df 100644
--- 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
@@ -27,8 +27,8 @@ import net.sf.ehcache.pool.sizeof.annotations.IgnoreSizeOf;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.topn.TopNCounter;
 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.common.util.DoubleMutable;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 557e325..3fa7d5c 100644
--- 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
@@ -7,7 +7,9 @@ import java.util.List;
 import java.util.Map;
 
 import com.google.common.collect.Maps;
+
 import org.apache.hadoop.io.Text;
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesSplitter;
@@ -24,7 +26,6 @@ import org.apache.kylin.dict.Dictionary;
 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.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index f7b9e9a..0f94dca 100644
--- 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
@@ -23,6 +23,8 @@ import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.hadoop.io.Text;
+import org.apache.kylin.aggregation.MeasureAggregators;
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.kv.RowConstants;
@@ -30,8 +32,6 @@ 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.measure.MeasureAggregators;
-import org.apache.kylin.metadata.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index db254f6..ab87b21 100644
--- 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
@@ -3,6 +3,8 @@ package org.apache.kylin.engine.mr.steps;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.kylin.aggregation.MeasureAggregators;
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -14,8 +16,6 @@ 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;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index fc616fa..8d00084 100644
--- 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
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.List;
 
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.TopNCounter;
@@ -47,7 +48,6 @@ 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;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 6c2679e..d08d2a4 100644
--- 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
@@ -27,6 +27,7 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.TopNCounter;
@@ -47,7 +48,6 @@ 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.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index fac3ff6..5b2d20e 100644
--- 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
@@ -31,13 +31,13 @@ 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.aggregation.MeasureCodec;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.common.util.LongMutable;
 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.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.MeasureCodec;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
index 01d97fd..3d2badd 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
@@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.aggregation.MeasureAggregators;
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.util.ByteArray;
@@ -56,15 +58,11 @@ import org.apache.kylin.cube.model.*;
 import org.apache.kylin.cube.util.CubingUtils;
 import org.apache.kylin.dict.*;
 import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.dict.DictionaryGenerator;
-import org.apache.kylin.dict.IterableDictionaryValueEnumerator;
 import org.apache.kylin.engine.spark.cube.BufferedCuboidWriter;
 import org.apache.kylin.engine.spark.cube.DefaultTupleConverter;
 import org.apache.kylin.engine.spark.util.IteratorUtils;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
-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.apache.kylin.metadata.model.TblColRef;
@@ -84,9 +82,11 @@ import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.hive.HiveContext;
 import org.reflections.Reflections;
+
 import scala.Tuple2;
 
 import javax.annotation.Nullable;
+
 import java.io.File;
 import java.io.FileFilter;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
index c126fdd..11b1897 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
@@ -23,9 +23,9 @@ import java.util.Arrays;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.fixedlen.FixedLenMeasureCodec;
+import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
index 2521fbf..71d7bae 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
@@ -23,9 +23,9 @@ import java.util.Arrays;
 import org.apache.commons.lang.ObjectUtils;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.common.util.ShardingHash;
 import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.measure.LongMutable;
 
 /**
  * TableRecord extends RawTableRecord by decorating it with a TableRecordInfo.

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
index 9627187..27519bc 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
@@ -23,8 +23,8 @@ import java.util.List;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
 import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.measure.fixedlen.FixedLenMeasureCodec;
 import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
index 2b2c514..bd27e38 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
@@ -23,8 +23,8 @@ import java.util.Arrays;
 
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.measure.LongMutable;
-import org.apache.kylin.metadata.measure.fixedlen.FixedLenMeasureCodec;
+import org.apache.kylin.common.util.LongMutable;
+import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
 import org.apache.kylin.metadata.model.DataType;
 
 import com.google.common.base.Objects;

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
new file mode 100644
index 0000000..35872be
--- /dev/null
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.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.invertedindex.measure;
+
+import org.apache.kylin.metadata.model.DataType;
+
+import java.nio.ByteBuffer;
+
+abstract public class FixedLenMeasureCodec<T> {
+
+    public static FixedLenMeasureCodec<?> get(DataType type) {
+        if (type.isHLLC()) {
+            return new FixedHLLCodec(type);
+        } else {
+            return new FixedPointLongCodec(type);
+        }
+    }
+
+    abstract public int getLength();
+
+    abstract public DataType getDataType();
+
+    abstract public T valueOf(String value);
+
+    abstract public Object getValue();
+
+    abstract public T read(byte[] buf, int offset);
+
+    abstract public void write(T v, byte[] buf, int offset);
+
+    abstract public T read(ByteBuffer buffer);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
new file mode 100644
index 0000000..a5d74f7
--- /dev/null
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.invertedindex.measure;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.LongMutable;
+import org.apache.kylin.metadata.model.DataType;
+
+public class FixedPointLongCodec extends FixedLenMeasureCodec<LongMutable> {
+
+    private static final int SIZE = 8;
+    // number of digits after decimal point
+    int scale;
+    DataType type;
+    // avoid massive object creation
+    LongMutable current = new LongMutable();
+
+    public FixedPointLongCodec(DataType type) {
+        this.type = type;
+        this.scale = Math.max(0, type.getScale());
+    }
+
+    @Override
+    public int getLength() {
+        return SIZE;
+    }
+
+    @Override
+    public DataType getDataType() {
+        return type;
+    }
+
+    long getValueIgnoringDecimalPoint(String value) {
+        int index = value.indexOf('.');
+
+        if (index == 0 || index == value.length() - 1) {
+            throw new RuntimeException("Bad decimal format: " + value);
+        } else if (index < 0) {
+            return Long.valueOf(value) * (int) Math.pow(10, scale);
+        } else {
+            StringBuilder sb = new StringBuilder();
+            sb.append(value.substring(0, index));
+
+            //if there are more than scale digits after the decimal point, the tail will be discarded
+            int end = Math.min(value.length(), index + scale + 1);
+            sb.append(value.substring(index + 1, end));
+            int diff = index + scale + 1 - value.length();
+            //if there are less than scale digits after the decimal point, the tail will be compensated
+            for (int i = 0; i < diff; i++) {
+                sb.append('0');
+            }
+            return Long.valueOf(sb.toString());
+        }
+    }
+
+    String restoreDecimalPoint(long value) {
+        if (scale < 0) {
+            throw new RuntimeException("Bad scale: " + scale + " with value: " + value);
+        } else if (scale == 0) {
+            return Long.toString(value);
+        } else {
+            return String.format("%." + scale + "f", value / (Math.pow(10, scale)));
+        }
+    }
+
+    @Override
+    public LongMutable valueOf(String value) {
+        if (value == null)
+            current.set(0L);
+        else
+            current.set(getValueIgnoringDecimalPoint(value));
+        return current;
+    }
+
+    @Override
+    public String getValue() {
+        if (scale == 0)
+            return current.toString();
+        else
+            return restoreDecimalPoint(current.get());
+    }
+
+    @Override
+    public LongMutable read(byte[] buf, int offset) {
+        current.set(BytesUtil.readLong(buf, offset, SIZE));
+        return current;
+    }
+
+    @Override
+    public void write(LongMutable v, byte[] buf, int offset) {
+        BytesUtil.writeLong(v == null ? 0 : v.get(), buf, offset, SIZE);
+    }
+
+    @Override
+    public LongMutable read(ByteBuffer buffer) {
+        current.set(BytesUtil.readLong(buffer, SIZE));
+        return current;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
index e17133f..4dd1723 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
@@ -32,7 +32,7 @@ import org.apache.kylin.invertedindex.index.ColumnValueContainer;
 import org.apache.kylin.invertedindex.index.CompressedValueContainer;
 import org.apache.kylin.invertedindex.index.Slice;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
-import org.apache.kylin.metadata.measure.fixedlen.FixedLenMeasureCodec;
+import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
 import org.apache.kylin.metadata.model.DataType;
 
 import com.google.common.base.Preconditions;

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescTest.java
new file mode 100644
index 0000000..aac915d
--- /dev/null
+++ b/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIDescTest.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.invertedindex;
+
+import java.io.IOException;
+
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.invertedindex.IIDescManager;
+import org.apache.kylin.invertedindex.model.IIDesc;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ */
+public class IIDescTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setup() {
+        this.createTestMetadata();
+
+    }
+
+    @After
+    public void clear() {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testGetIIDesc() {
+
+        IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_left_join_desc");
+        DataModelDesc model = iiDesc.getModel();
+        Assert.assertNotNull(iiDesc);
+        Assert.assertNotNull(model);
+
+    }
+
+    @Test
+    public void testSerialization() throws IOException {
+        IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_left_join_desc");
+        String str = JsonUtil.writeValueAsIndentString(iiDesc);
+        System.out.println(str);
+        IIDesc desc2 = JsonUtil.readValue(str, IIDesc.class);
+
+        Assert.assertEquals(iiDesc, desc2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
new file mode 100644
index 0000000..f30a876
--- /dev/null
+++ b/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.invertedindex;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.invertedindex.IIDescManager;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IIManager;
+import org.apache.kylin.invertedindex.model.IIDesc;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ */
+public class IIInstanceTest extends LocalFileMetadataTestCase {
+    @Before
+    public void setup() {
+        createTestMetadata();
+    }
+
+    @After
+    public void clean() {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void testGetIIsByDesc() throws IOException {
+        IIManager mgr = IIManager.getInstance(getTestConfig());
+
+        List<IIInstance> iiInstances = mgr.getIIsByDesc("test_kylin_ii_left_join_desc");
+
+        Assert.assertTrue(iiInstances.size() > 0);
+
+    }
+
+    @Test
+    public void testCreateIIInstance() throws IOException {
+
+        IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_left_join_desc");
+
+        IIInstance ii = IIInstance.create("new ii", "default", iiDesc);
+
+        IIManager iiMgr = IIManager.getInstance(getTestConfig());
+
+        List<IIInstance> allIIList = iiMgr.listAllIIs();
+
+        iiMgr.createII(ii);
+
+        Assert.assertNotNull(iiMgr.getII("new ii"));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
new file mode 100644
index 0000000..d72899b
--- /dev/null
+++ b/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.invertedindex;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import javax.annotation.Nullable;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.dict.DictionaryGenerator;
+import org.apache.kylin.dict.IterableDictionaryValueEnumerator;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IIManager;
+import org.apache.kylin.invertedindex.index.CompressedValueContainer;
+import org.apache.kylin.invertedindex.index.RawTableRecord;
+import org.apache.kylin.invertedindex.index.ShardingSliceBuilder;
+import org.apache.kylin.invertedindex.index.Slice;
+import org.apache.kylin.invertedindex.index.TableRecord;
+import org.apache.kylin.invertedindex.index.TableRecordInfo;
+import org.apache.kylin.invertedindex.model.IIDesc;
+import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
+import org.apache.kylin.invertedindex.model.IIRow;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.SetMultimap;
+import com.google.common.collect.Sets;
+
+public class InvertedIndexLocalTest extends LocalFileMetadataTestCase {
+
+    IIInstance ii;
+    TableRecordInfo info;
+    List<String> lines;
+    private Dictionary<?>[] dictionaryMap;
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii_left_join");
+
+        File file = new File(LOCALMETA_TEST_DATA, "data/flatten_data_for_ii.csv");
+        FileInputStream in = new FileInputStream(file);
+        this.lines = IOUtils.readLines(in, "UTF-8");
+        in.close();
+
+        dictionaryMap = buildDictionary(Lists.transform(lines, new Function<String, List<String>>() {
+            @Nullable
+            @Override
+            public List<String> apply(@Nullable String input) {
+                return Lists.newArrayList(input.split(","));
+            }
+        }), ii.getDescriptor());
+        this.info = new TableRecordInfo(ii.getDescriptor(), dictionaryMap);
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testCompressedValueContainer() {
+        // create container
+        CompressedValueContainer container = new CompressedValueContainer(info.getDigest(), 0, 500);
+        Dictionary<String> dict = info.dict(0);
+
+        byte[] buf = new byte[dict.getSizeOfId()];
+        ImmutableBytesWritable bytes = new ImmutableBytesWritable(buf);
+
+        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
+            BytesUtil.writeUnsigned(v, buf, 0, dict.getSizeOfId());
+            container.append(bytes);
+        }
+        BytesUtil.writeUnsigned(Dictionary.NULL_ID[dict.getSizeOfId()], buf, 0, dict.getSizeOfId());
+        container.append(bytes);
+        container.closeForChange();
+
+        // copy by serialization
+        ImmutableBytesWritable copy = container.toBytes();
+        CompressedValueContainer container2 = new CompressedValueContainer(info.getDigest(), 0, 500);
+        container2.fromBytes(copy);
+
+        // check the copy
+        int i = 0;
+        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
+            container2.getValueAt(i++, bytes);
+            int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength());
+            assertEquals(v, value);
+        }
+        container2.getValueAt(i++, bytes);
+        int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength());
+        assertEquals(Dictionary.NULL_ID[dict.getSizeOfId()], value);
+        assertEquals(container, container2);
+    }
+
+    @Test
+    public void testCodec() throws IOException {
+        List<TableRecord> records = loadRecordsSorted();
+        System.out.println(records.size() + " records");
+        List<Slice> slices = buildTimeSlices(records);
+        System.out.println(slices.size() + " slices");
+
+        IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest());
+        List<IIRow> kvs = encodeKVs(codec, slices);
+        System.out.println(kvs.size() + " KV pairs");
+
+        List<Slice> slicesCopy = decodeKVs(codec, kvs);
+        assertEquals(slices.size(), slicesCopy.size());
+        for (int i = 0; i < slices.size(); i++) {
+            assertEquals(slices.get(i), slicesCopy.get(i));
+        }
+
+        List<TableRecord> recordsCopy = iterateRecords(slicesCopy);
+        assertEquals(new HashSet<TableRecord>(records), new HashSet<TableRecord>(recordsCopy));
+        dump(recordsCopy);
+    }
+
+    private Dictionary<?>[] buildDictionary(List<List<String>> table, IIDesc desc) throws IOException{
+        SetMultimap<TblColRef, String> valueMap = HashMultimap.create();
+        Set<TblColRef> dimensionColumns = Sets.newHashSet();
+        for (int i = 0; i < desc.listAllColumns().size(); i++) {
+            if (!desc.isMetricsCol(i)) {
+                dimensionColumns.add(desc.listAllColumns().get(i));
+            }
+        }
+        for (List<String> row : table) {
+            for (int i = 0; i < row.size(); i++) {
+                String cell = row.get(i);
+                valueMap.put(desc.listAllColumns().get(i), cell);
+            }
+        }
+        Dictionary<?>[] result = new Dictionary<?>[desc.listAllColumns().size()];
+        for (TblColRef tblColRef : valueMap.keys()) {
+            result[desc.findColumn(tblColRef)] = DictionaryGenerator.buildDictionaryFromValueEnumerator(tblColRef.getType(),
+                    new IterableDictionaryValueEnumerator(Collections2.transform(valueMap.get(tblColRef), new Function<String, byte[]>() {
+                @Nullable
+                @Override
+                public byte[] apply(String input) {
+                    return input.getBytes();
+                }
+            })));
+        }
+        return result;
+    }
+
+    private List<TableRecord> loadRecordsSorted() throws IOException {
+        List<TableRecord> records = Lists.newArrayList();
+        for (String line : lines) {
+            String[] fields = line.split(",");
+            TableRecord rec = info.createTableRecord();
+            for (int col = 0; col < fields.length; col++) {
+                rec.setValueString(col, fields[col]);
+            }
+            records.add(rec);
+        }
+
+        Collections.sort(records, new Comparator<TableRecord>() {
+            @Override
+            public int compare(TableRecord a, TableRecord b) {
+                long x = a.getTimestamp() - b.getTimestamp();
+                if (x > 0)
+                    return 1;
+                else if (x == 0)
+                    return 0;
+                else
+                    return -1;
+            }
+        });
+
+        return records;
+    }
+
+    private List<Slice> buildTimeSlices(List<TableRecord> records) throws IOException {
+        ShardingSliceBuilder builder = new ShardingSliceBuilder(info);
+        List<Slice> slices = Lists.newArrayList();
+        for (TableRecord rec : records) {
+            //here assume there less records than slice size for each shard
+            Slice slice = builder.append(rec);
+            if (slice != null) {
+                slice.setLocalDictionaries(dictionaryMap);
+                slices.add(slice);
+            }
+        }
+        List<Slice> finals = builder.close();
+        for (Slice slice : finals) {
+            slice.setLocalDictionaries(dictionaryMap);
+        }
+        slices.addAll(finals);
+
+        Collections.sort(slices);
+        return slices;
+    }
+
+    private List<IIRow> encodeKVs(IIKeyValueCodec codec, List<Slice> slices) {
+
+        List<IIRow> kvs = Lists.newArrayList();
+        for (Slice slice : slices) {
+            kvs.addAll(codec.encodeKeyValue(slice));
+        }
+        return kvs;
+    }
+
+    private List<Slice> decodeKVs(IIKeyValueCodec codec, List<IIRow> kvs) {
+        List<Slice> slices = Lists.newArrayList();
+        for (Slice slice : codec.decodeKeyValue(kvs)) {
+            slices.add(slice);
+        }
+        return slices;
+    }
+
+    private List<TableRecord> iterateRecords(List<Slice> slices) {
+        List<TableRecord> records = Lists.newArrayList();
+        for (Slice slice : slices) {
+            for (RawTableRecord rec : slice) {
+                records.add(new TableRecord((RawTableRecord) rec.clone(), info));
+            }
+        }
+        return records;
+    }
+
+    private void dump(Iterable<TableRecord> records) {
+        for (TableRecord rec : records) {
+            System.out.println(rec.toString());
+        }
+    }
+
+}

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIInstanceTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIInstanceTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIInstanceTest.java
deleted file mode 100644
index 8a0c2ba..0000000
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIInstanceTest.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.invertedindex.invertedindex;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.invertedindex.IIDescManager;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- */
-public class IIInstanceTest extends LocalFileMetadataTestCase {
-    @Before
-    public void setup() {
-        createTestMetadata();
-    }
-
-    @After
-    public void clean() {
-        cleanupTestMetadata();
-    }
-
-    @Test
-    public void testGetIIsByDesc() throws IOException {
-        IIManager mgr = IIManager.getInstance(getTestConfig());
-
-        List<IIInstance> iiInstances = mgr.getIIsByDesc("test_kylin_ii_left_join_desc");
-
-        Assert.assertTrue(iiInstances.size() > 0);
-
-    }
-
-    @Test
-    public void testCreateIIInstance() throws IOException {
-
-        IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_left_join_desc");
-
-        IIInstance ii = IIInstance.create("new ii", "default", iiDesc);
-
-        IIManager iiMgr = IIManager.getInstance(getTestConfig());
-
-        List<IIInstance> allIIList = iiMgr.listAllIIs();
-
-        iiMgr.createII(ii);
-
-        Assert.assertNotNull(iiMgr.getII("new ii"));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/InvertedIndexLocalTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/InvertedIndexLocalTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/InvertedIndexLocalTest.java
deleted file mode 100644
index 6e05759..0000000
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/InvertedIndexLocalTest.java
+++ /dev/null
@@ -1,262 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.invertedindex.invertedindex;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.dict.DictionaryGenerator;
-import org.apache.kylin.dict.IterableDictionaryValueEnumerator;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.index.CompressedValueContainer;
-import org.apache.kylin.invertedindex.index.RawTableRecord;
-import org.apache.kylin.invertedindex.index.ShardingSliceBuilder;
-import org.apache.kylin.invertedindex.index.Slice;
-import org.apache.kylin.invertedindex.index.TableRecord;
-import org.apache.kylin.invertedindex.index.TableRecordInfo;
-import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.apache.kylin.invertedindex.model.IIRow;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.SetMultimap;
-import com.google.common.collect.Sets;
-
-public class InvertedIndexLocalTest extends LocalFileMetadataTestCase {
-
-    IIInstance ii;
-    TableRecordInfo info;
-    List<String> lines;
-    private Dictionary<?>[] dictionaryMap;
-
-    @Before
-    public void setUp() throws Exception {
-        this.createTestMetadata();
-        this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii_left_join");
-
-        File file = new File(LOCALMETA_TEST_DATA, "data/flatten_data_for_ii.csv");
-        FileInputStream in = new FileInputStream(file);
-        this.lines = IOUtils.readLines(in, "UTF-8");
-        in.close();
-
-        dictionaryMap = buildDictionary(Lists.transform(lines, new Function<String, List<String>>() {
-            @Nullable
-            @Override
-            public List<String> apply(@Nullable String input) {
-                return Lists.newArrayList(input.split(","));
-            }
-        }), ii.getDescriptor());
-        this.info = new TableRecordInfo(ii.getDescriptor(), dictionaryMap);
-    }
-
-    @After
-    public void after() throws Exception {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void testCompressedValueContainer() {
-        // create container
-        CompressedValueContainer container = new CompressedValueContainer(info.getDigest(), 0, 500);
-        Dictionary<String> dict = info.dict(0);
-
-        byte[] buf = new byte[dict.getSizeOfId()];
-        ImmutableBytesWritable bytes = new ImmutableBytesWritable(buf);
-
-        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
-            BytesUtil.writeUnsigned(v, buf, 0, dict.getSizeOfId());
-            container.append(bytes);
-        }
-        BytesUtil.writeUnsigned(Dictionary.NULL_ID[dict.getSizeOfId()], buf, 0, dict.getSizeOfId());
-        container.append(bytes);
-        container.closeForChange();
-
-        // copy by serialization
-        ImmutableBytesWritable copy = container.toBytes();
-        CompressedValueContainer container2 = new CompressedValueContainer(info.getDigest(), 0, 500);
-        container2.fromBytes(copy);
-
-        // check the copy
-        int i = 0;
-        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
-            container2.getValueAt(i++, bytes);
-            int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength());
-            assertEquals(v, value);
-        }
-        container2.getValueAt(i++, bytes);
-        int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength());
-        assertEquals(Dictionary.NULL_ID[dict.getSizeOfId()], value);
-        assertEquals(container, container2);
-    }
-
-    @Test
-    public void testCodec() throws IOException {
-        List<TableRecord> records = loadRecordsSorted();
-        System.out.println(records.size() + " records");
-        List<Slice> slices = buildTimeSlices(records);
-        System.out.println(slices.size() + " slices");
-
-        IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest());
-        List<IIRow> kvs = encodeKVs(codec, slices);
-        System.out.println(kvs.size() + " KV pairs");
-
-        List<Slice> slicesCopy = decodeKVs(codec, kvs);
-        assertEquals(slices.size(), slicesCopy.size());
-        for (int i = 0; i < slices.size(); i++) {
-            assertEquals(slices.get(i), slicesCopy.get(i));
-        }
-
-        List<TableRecord> recordsCopy = iterateRecords(slicesCopy);
-        assertEquals(new HashSet<TableRecord>(records), new HashSet<TableRecord>(recordsCopy));
-        dump(recordsCopy);
-    }
-
-    private Dictionary<?>[] buildDictionary(List<List<String>> table, IIDesc desc) throws IOException{
-        SetMultimap<TblColRef, String> valueMap = HashMultimap.create();
-        Set<TblColRef> dimensionColumns = Sets.newHashSet();
-        for (int i = 0; i < desc.listAllColumns().size(); i++) {
-            if (!desc.isMetricsCol(i)) {
-                dimensionColumns.add(desc.listAllColumns().get(i));
-            }
-        }
-        for (List<String> row : table) {
-            for (int i = 0; i < row.size(); i++) {
-                String cell = row.get(i);
-                valueMap.put(desc.listAllColumns().get(i), cell);
-            }
-        }
-        Dictionary<?>[] result = new Dictionary<?>[desc.listAllColumns().size()];
-        for (TblColRef tblColRef : valueMap.keys()) {
-            result[desc.findColumn(tblColRef)] = DictionaryGenerator.buildDictionaryFromValueEnumerator(tblColRef.getType(),
-                    new IterableDictionaryValueEnumerator(Collections2.transform(valueMap.get(tblColRef), new Function<String, byte[]>() {
-                @Nullable
-                @Override
-                public byte[] apply(String input) {
-                    return input.getBytes();
-                }
-            })));
-        }
-        return result;
-    }
-
-    private List<TableRecord> loadRecordsSorted() throws IOException {
-        List<TableRecord> records = Lists.newArrayList();
-        for (String line : lines) {
-            String[] fields = line.split(",");
-            TableRecord rec = info.createTableRecord();
-            for (int col = 0; col < fields.length; col++) {
-                rec.setValueString(col, fields[col]);
-            }
-            records.add(rec);
-        }
-
-        Collections.sort(records, new Comparator<TableRecord>() {
-            @Override
-            public int compare(TableRecord a, TableRecord b) {
-                long x = a.getTimestamp() - b.getTimestamp();
-                if (x > 0)
-                    return 1;
-                else if (x == 0)
-                    return 0;
-                else
-                    return -1;
-            }
-        });
-
-        return records;
-    }
-
-    private List<Slice> buildTimeSlices(List<TableRecord> records) throws IOException {
-        ShardingSliceBuilder builder = new ShardingSliceBuilder(info);
-        List<Slice> slices = Lists.newArrayList();
-        for (TableRecord rec : records) {
-            //here assume there less records than slice size for each shard
-            Slice slice = builder.append(rec);
-            if (slice != null) {
-                slice.setLocalDictionaries(dictionaryMap);
-                slices.add(slice);
-            }
-        }
-        List<Slice> finals = builder.close();
-        for (Slice slice : finals) {
-            slice.setLocalDictionaries(dictionaryMap);
-        }
-        slices.addAll(finals);
-
-        Collections.sort(slices);
-        return slices;
-    }
-
-    private List<IIRow> encodeKVs(IIKeyValueCodec codec, List<Slice> slices) {
-
-        List<IIRow> kvs = Lists.newArrayList();
-        for (Slice slice : slices) {
-            kvs.addAll(codec.encodeKeyValue(slice));
-        }
-        return kvs;
-    }
-
-    private List<Slice> decodeKVs(IIKeyValueCodec codec, List<IIRow> kvs) {
-        List<Slice> slices = Lists.newArrayList();
-        for (Slice slice : codec.decodeKeyValue(kvs)) {
-            slices.add(slice);
-        }
-        return slices;
-    }
-
-    private List<TableRecord> iterateRecords(List<Slice> slices) {
-        List<TableRecord> records = Lists.newArrayList();
-        for (Slice slice : slices) {
-            for (RawTableRecord rec : slice) {
-                records.add(new TableRecord((RawTableRecord) rec.clone(), info));
-            }
-        }
-        return records;
-    }
-
-    private void dump(Iterable<TableRecord> records) {
-        for (TableRecord rec : records) {
-            System.out.println(rec.toString());
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/LZFTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/LZFTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/LZFTest.java
deleted file mode 100644
index d39c421..0000000
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/LZFTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.invertedindex.invertedindex;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.junit.Test;
-
-import com.ning.compress.lzf.LZFDecoder;
-import com.ning.compress.lzf.LZFEncoder;
-
-/**
- */
-public class LZFTest {
-    @Test
-    public void test() throws IOException {
-
-        byte[] raw = new byte[] { 1, 2, 3, 3, 2, 23 };
-        byte[] data = LZFEncoder.encode(raw);
-
-        byte[] data2 = new byte[data.length * 2];
-        java.lang.System.arraycopy(data, 0, data2, 0, data.length);
-        ImmutableBytesWritable bytes = new ImmutableBytesWritable();
-        bytes.set(data2, 0, data.length);
-
-        try {
-            byte[] uncompressed = LZFDecoder.decode(bytes.get(), bytes.getOffset(), bytes.getLength());
-        } catch (IOException e) {
-            throw new RuntimeException("LZF decode failure", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
new file mode 100644
index 0000000..27a4f71
--- /dev/null
+++ b/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
@@ -0,0 +1,45 @@
+package org.apache.kylin.invertedindex.measure;
+
+import org.apache.kylin.invertedindex.measure.FixedPointLongCodec;
+import org.apache.kylin.metadata.model.DataType;
+import org.junit.Test;
+
+/**
+ */
+public class FixedPointLongCodecTest {
+
+    @Test
+    public void testEncode1() {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        long x = codec.getValueIgnoringDecimalPoint("12.12345");
+        org.junit.Assert.assertEquals(1212345, x);
+    }
+
+    @Test
+    public void testEncode2() {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        long x = codec.getValueIgnoringDecimalPoint("12.1234");
+        org.junit.Assert.assertEquals(1212340, x);
+    }
+
+    @Test
+    public void testEncode3() {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        long x = codec.getValueIgnoringDecimalPoint("12.123456");
+        org.junit.Assert.assertEquals(1212345, x);
+    }
+
+    @Test
+    public void testEncode4() {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        long x = codec.getValueIgnoringDecimalPoint("12");
+        org.junit.Assert.assertEquals(1200000, x);
+    }
+
+    @Test
+    public void testDecode1() {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        String x = codec.restoreDecimalPoint(1212345);
+        org.junit.Assert.assertEquals("12.12345", x);
+    }
+}



[04/13] incubator-kylin git commit: KYLIN-976 very initial

Posted by li...@apache.org.
KYLIN-976 very initial


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

Branch: refs/heads/KYLIN-976
Commit: 1218bbde487e973de0391162204d73c76f1a9e81
Parents: 6515b0a
Author: Li, Yang <ya...@ebay.com>
Authored: Tue Nov 24 13:15:54 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Fri Nov 27 14:47:05 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/util/DoubleMutable.java |  68 +++++
 .../apache/kylin/common/util/LongMutable.java   |  70 +++++
 .../apache/kylin/aggregation/Aggregation.java   |  42 +++
 .../kylin/aggregation/DataTypeSerializer.java   | 118 +++++++++
 .../kylin/aggregation/MeasureAggregator.java    | 133 ++++++++++
 .../kylin/aggregation/MeasureAggregators.java   |  81 ++++++
 .../apache/kylin/aggregation/MeasureCodec.java  |  78 ++++++
 .../basic/BigDecimalMaxAggregator.java          |  54 ++++
 .../basic/BigDecimalMinAggregator.java          |  55 ++++
 .../aggregation/basic/BigDecimalSerializer.java | 111 ++++++++
 .../basic/BigDecimalSumAggregator.java          |  51 ++++
 .../aggregation/basic/DateTimeSerializer.java   |  65 +++++
 .../aggregation/basic/DoubleMaxAggregator.java  |  54 ++++
 .../aggregation/basic/DoubleMinAggregator.java  |  54 ++++
 .../aggregation/basic/DoubleSerializer.java     |  84 ++++++
 .../aggregation/basic/DoubleSumAggregator.java  |  51 ++++
 .../aggregation/basic/LongMaxAggregator.java    |  54 ++++
 .../aggregation/basic/LongMinAggregator.java    |  54 ++++
 .../kylin/aggregation/basic/LongSerializer.java |  91 +++++++
 .../aggregation/basic/LongSumAggregator.java    |  51 ++++
 .../aggregation/basic/StringSerializer.java     |  56 ++++
 .../kylin/aggregation/hllc/HLLCAggregator.java  |  64 +++++
 .../kylin/aggregation/hllc/HLLCSerializer.java  |  98 +++++++
 .../kylin/aggregation/hllc/LDCAggregator.java   |  63 +++++
 .../kylin/aggregation/topn/TopNAggregator.java  |  66 +++++
 .../aggregation/topn/TopNCounterSerializer.java | 117 +++++++++
 .../kylin/cube/gridtable/CubeCodeSystem.java    |   6 +-
 .../cube/gridtable/TrimmedCubeCodeSystem.java   |   6 +-
 .../cube/inmemcubing/DoggedCubeBuilder.java     |   2 +-
 .../cube/inmemcubing/InMemCubeBuilder.java      |   2 +-
 .../InMemCubeBuilderInputConverter.java         |   5 +-
 .../kylin/gridtable/GTAggregateScanner.java     |   2 +-
 .../kylin/gridtable/GTSampleCodeSystem.java     |   4 +-
 .../apache/kylin/gridtable/IGTCodeSystem.java   |   2 +-
 .../apache/kylin/gridtable/UnitTestSupport.java |   2 +-
 .../basic/BigDecimalSerializerTest.java         |  53 ++++
 .../topn/TopNCounterSerializerTest.java         |  61 +++++
 .../gridtable/AggregationCacheMemSizeTest.java  |  14 +-
 .../kylin/gridtable/DictGridTableTest.java      |   2 +-
 .../kylin/gridtable/SimpleGridTableTest.java    |   2 +-
 .../gridtable/SimpleInvertedIndexTest.java      |   5 +-
 .../metadata/measure/MeasureCodecTest.java      |   3 +
 .../measure/BigDecimalMaxAggregator.java        |  52 ----
 .../measure/BigDecimalMinAggregator.java        |  53 ----
 .../measure/BigDecimalSumAggregator.java        |  49 ----
 .../metadata/measure/DoubleMaxAggregator.java   |  51 ----
 .../metadata/measure/DoubleMinAggregator.java   |  51 ----
 .../kylin/metadata/measure/DoubleMutable.java   |  68 -----
 .../metadata/measure/DoubleSumAggregator.java   |  48 ----
 .../kylin/metadata/measure/HLLCAggregator.java  |  63 -----
 .../kylin/metadata/measure/LDCAggregator.java   |  60 -----
 .../metadata/measure/LongMaxAggregator.java     |  51 ----
 .../metadata/measure/LongMinAggregator.java     |  51 ----
 .../kylin/metadata/measure/LongMutable.java     |  70 -----
 .../metadata/measure/LongSumAggregator.java     |  48 ----
 .../metadata/measure/MeasureAggregator.java     | 121 ---------
 .../metadata/measure/MeasureAggregators.java    |  81 ------
 .../kylin/metadata/measure/MeasureCodec.java    |  79 ------
 .../kylin/metadata/measure/TopNAggregator.java  |  66 -----
 .../measure/fixedlen/FixedHLLCodec.java         |  80 ------
 .../measure/fixedlen/FixedLenMeasureCodec.java  |  49 ----
 .../measure/fixedlen/FixedPointLongCodec.java   | 117 ---------
 .../serializer/BigDecimalSerializer.java        | 110 --------
 .../measure/serializer/DataTypeSerializer.java  | 111 --------
 .../measure/serializer/DateTimeSerializer.java  |  64 -----
 .../measure/serializer/DoubleSerializer.java    |  83 ------
 .../measure/serializer/HLLCSerializer.java      |  97 -------
 .../measure/serializer/LongSerializer.java      |  90 -------
 .../measure/serializer/StringSerializer.java    |  55 ----
 .../serializer/TopNCounterSerializer.java       | 116 --------
 .../apache/kylin/metadata/model/DataType.java   |   2 +-
 .../fixedlen/FixedPointLongCodecTest.java       |  44 ----
 .../serializer/BigDecimalSerializerTest.java    |  52 ----
 .../serializer/TopNCounterSerializerTest.java   |  61 -----
 .../org/apache/kylin/storage/tuple/Tuple.java   |   4 +-
 .../engine/mr/steps/BaseCuboidMapperBase.java   |   3 +-
 .../kylin/engine/mr/steps/CuboidReducer.java    |   4 +-
 .../engine/mr/steps/InMemCuboidReducer.java     |   4 +-
 .../mr/steps/MergeCuboidFromStorageMapper.java  |   2 +-
 .../engine/mr/steps/MergeCuboidMapper.java      |   2 +-
 .../kylin/engine/mr/steps/CubeReducerTest.java  |   4 +-
 .../apache/kylin/engine/spark/SparkCubing.java  |   8 +-
 .../invertedindex/index/RawTableRecord.java     |   4 +-
 .../kylin/invertedindex/index/TableRecord.java  |   2 +-
 .../invertedindex/index/TableRecordInfo.java    |   2 +-
 .../index/TableRecordInfoDigest.java            |   4 +-
 .../invertedindex/measure/FixedHLLCodec.java    |  80 ++++++
 .../measure/FixedLenMeasureCodec.java           |  49 ++++
 .../measure/FixedPointLongCodec.java            | 117 +++++++++
 .../invertedindex/model/IIKeyValueCodec.java    |   2 +-
 .../kylin/invertedindex/IIDescManagerTest.java  | 104 ++++++++
 .../apache/kylin/invertedindex/IIDescTest.java  |  67 +++++
 .../kylin/invertedindex/IIInstanceTest.java     |  74 ++++++
 .../invertedindex/InvertedIndexLocalTest.java   | 262 +++++++++++++++++++
 .../org/apache/kylin/invertedindex/LZFTest.java |  49 ++++
 .../invertedindex/IIDescManagerTest.java        | 104 --------
 .../invertedindex/invertedindex/IIDescTest.java |  67 -----
 .../invertedindex/IIInstanceTest.java           |  74 ------
 .../invertedindex/InvertedIndexLocalTest.java   | 262 -------------------
 .../invertedindex/invertedindex/LZFTest.java    |  49 ----
 .../measure/FixedPointLongCodecTest.java        |  45 ++++
 .../common/coprocessor/AggregationCache.java    |   2 +-
 .../observer/AggregationScanner.java            |   2 +-
 .../observer/ObserverAggregationCache.java      |   2 +-
 .../observer/ObserverAggregators.java           |   4 +-
 .../endpoint/EndpointAggregationCache.java      |   2 +-
 .../endpoint/EndpointAggregators.java           |   6 +-
 .../ii/coprocessor/endpoint/IIEndpoint.java     |   3 +-
 .../storage/hbase/steps/CubeHFileMapper.java    |   2 +-
 .../hbase/steps/HBaseMROutput2Transition.java   |   2 +-
 .../storage/hbase/steps/KeyValueCreator.java    |   2 +-
 .../storage/hbase/steps/RowValueDecoder.java    |   6 +-
 .../observer/AggregateRegionObserverTest.java   |   2 +-
 .../endpoint/EndpointAggregationTest.java       |   7 +-
 .../hbase/steps/CubeHFileMapper2Test.java       |   2 +-
 .../hbase/steps/RowValueDecoderTest.java        |   4 +-
 116 files changed, 2924 insertions(+), 2819 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-common/src/main/java/org/apache/kylin/common/util/DoubleMutable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/DoubleMutable.java b/core-common/src/main/java/org/apache/kylin/common/util/DoubleMutable.java
new file mode 100644
index 0000000..520cd74
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/util/DoubleMutable.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.common.util;
+
+import java.io.Serializable;
+
+@SuppressWarnings("serial")
+public class DoubleMutable implements Comparable<DoubleMutable>, Serializable {
+
+    private double v;
+
+    public DoubleMutable() {
+        this(0);
+    }
+
+    public DoubleMutable(double v) {
+        set(v);
+    }
+
+    public double get() {
+        return v;
+    }
+
+    public void set(double v) {
+        this.v = v;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof DoubleMutable)) {
+            return false;
+        }
+        DoubleMutable other = (DoubleMutable) o;
+        return this.v == other.v;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) Double.doubleToLongBits(v);
+    }
+
+    @Override
+    public int compareTo(DoubleMutable o) {
+        return (v < o.v ? -1 : (v == o.v ? 0 : 1));
+    }
+
+    @Override
+    public String toString() {
+        return Double.toString(v);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-common/src/main/java/org/apache/kylin/common/util/LongMutable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/LongMutable.java b/core-common/src/main/java/org/apache/kylin/common/util/LongMutable.java
new file mode 100644
index 0000000..238bb86
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/util/LongMutable.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.common.util;
+
+import java.io.Serializable;
+
+@SuppressWarnings("serial")
+public class LongMutable implements Comparable<LongMutable>, Serializable {
+
+    private long v;
+
+    public LongMutable() {
+        this(0);
+    }
+
+    public LongMutable(long v) {
+        set(v);
+    }
+
+    public long get() {
+        return v;
+    }
+
+    public void set(long v) {
+        this.v = v;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof LongMutable)) {
+            return false;
+        }
+        LongMutable other = (LongMutable) o;
+        return this.v == other.v;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) v;
+    }
+
+    @Override
+    public int compareTo(LongMutable o) {
+        long thisValue = this.v;
+        long thatValue = o.v;
+        return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
+    }
+
+    @Override
+    public String toString() {
+        return Long.toString(v);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/Aggregation.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/Aggregation.java b/core-cube/src/main/java/org/apache/kylin/aggregation/Aggregation.java
new file mode 100644
index 0000000..193c5de
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/Aggregation.java
@@ -0,0 +1,42 @@
+package org.apache.kylin.aggregation;
+
+import java.util.List;
+
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+abstract public class Aggregation {
+    
+    /* ============================================================================
+     * Define
+     * ---------------------------------------------------------------------------- */
+    
+    abstract public DataType getAggregationDataType();
+    
+    abstract public DataType getResultDataType();
+    
+    abstract public void validate(MeasureDesc measureDesc) throws IllegalArgumentException;
+    
+    /* ============================================================================
+     * Build
+     * ---------------------------------------------------------------------------- */
+    
+    abstract public DataTypeSerializer<?> getSeralizer();
+    
+    abstract public MeasureAggregator<?> getAggregator();
+ 
+    abstract public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc);
+    
+    abstract public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts);
+
+    /* ============================================================================
+     * Cube Selection
+     * ---------------------------------------------------------------------------- */
+    
+    /* ============================================================================
+     * Query
+     * ---------------------------------------------------------------------------- */
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/DataTypeSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/DataTypeSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/DataTypeSerializer.java
new file mode 100644
index 0000000..df6833c
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/DataTypeSerializer.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.aggregation;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kylin.aggregation.basic.BigDecimalSerializer;
+import org.apache.kylin.aggregation.basic.DateTimeSerializer;
+import org.apache.kylin.aggregation.basic.DoubleSerializer;
+import org.apache.kylin.aggregation.basic.LongSerializer;
+import org.apache.kylin.aggregation.basic.StringSerializer;
+import org.apache.kylin.aggregation.hllc.HLLCSerializer;
+import org.apache.kylin.aggregation.topn.TopNCounterSerializer;
+import org.apache.kylin.common.util.BytesSerializer;
+import org.apache.kylin.metadata.model.DataType;
+
+import com.google.common.collect.Maps;
+
+/**
+ * @author yangli9
+ * 
+ * Note: the implementations MUST be thread-safe.
+ * 
+ */
+abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
+
+    final static Map<String, Class<?>> implementations;
+    static {
+        HashMap<String, Class<?>> impl = Maps.newHashMap();
+        impl.put("varchar", StringSerializer.class);
+        impl.put("decimal", BigDecimalSerializer.class);
+        impl.put("double", DoubleSerializer.class);
+        impl.put("float", DoubleSerializer.class);
+        impl.put("bigint", LongSerializer.class);
+        impl.put("long", LongSerializer.class);
+        impl.put("integer", LongSerializer.class);
+        impl.put("int", LongSerializer.class);
+        impl.put("smallint", LongSerializer.class);
+        impl.put("date", DateTimeSerializer.class);
+        impl.put("datetime", DateTimeSerializer.class);
+        impl.put("timestamp", DateTimeSerializer.class);
+        implementations = Collections.unmodifiableMap(impl);
+
+    }
+
+    public static DataTypeSerializer<?> create(String dataType) {
+        return create(DataType.getInstance(dataType));
+    }
+
+    public static DataTypeSerializer<?> create(DataType type) {
+        if (type.isHLLC()) {
+            return new HLLCSerializer(type);
+        }
+
+        if (type.isTopN()) {
+            return new TopNCounterSerializer(type);
+        }
+
+        Class<?> clz = implementations.get(type.getName());
+        if (clz == null)
+            throw new RuntimeException("No MeasureSerializer for type " + type);
+
+        try {
+            return (DataTypeSerializer<?>) clz.getConstructor(DataType.class).newInstance(type);
+        } catch (Exception e) {
+            throw new RuntimeException(e); // never happen
+        }
+    }
+    
+    /** peek into buffer and return the length of serialization */
+    abstract public int peekLength(ByteBuffer in);
+
+    /** return the max number of bytes to the longest serialization */
+    abstract public int maxLength();
+
+    /** get an estimate of size in bytes of the serialized data */
+    abstract public int getStorageBytesEstimate();
+
+    /** convert from String to obj (string often come as byte[] in mapred) */
+    abstract public T valueOf(byte[] value);
+
+    /** convert from String to obj */
+    public T valueOf(String value) {
+        try {
+            return valueOf(value.getBytes("UTF-8"));
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e); // never happen
+        }
+    }
+
+    /** convert from obj to string */
+    public String toString(T value) {
+        if (value == null)
+            return "NULL";
+        else
+            return value.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
new file mode 100644
index 0000000..9c8945d
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.aggregation;
+
+import org.apache.kylin.aggregation.basic.BigDecimalMaxAggregator;
+import org.apache.kylin.aggregation.basic.BigDecimalMinAggregator;
+import org.apache.kylin.aggregation.basic.BigDecimalSumAggregator;
+import org.apache.kylin.aggregation.basic.DoubleMaxAggregator;
+import org.apache.kylin.aggregation.basic.DoubleMinAggregator;
+import org.apache.kylin.aggregation.basic.DoubleSumAggregator;
+import org.apache.kylin.aggregation.basic.LongMaxAggregator;
+import org.apache.kylin.aggregation.basic.LongMinAggregator;
+import org.apache.kylin.aggregation.basic.LongSumAggregator;
+import org.apache.kylin.aggregation.hllc.HLLCAggregator;
+import org.apache.kylin.aggregation.hllc.LDCAggregator;
+import org.apache.kylin.aggregation.topn.TopNAggregator;
+import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.metadata.model.FunctionDesc;
+
+import java.io.Serializable;
+
+/**
+ */
+@SuppressWarnings("serial")
+abstract public class MeasureAggregator<V> implements Serializable {
+
+    public static MeasureAggregator<?> create(String funcName, String returnType) {
+        if (FunctionDesc.FUNC_SUM.equalsIgnoreCase(funcName) || FunctionDesc.FUNC_COUNT.equalsIgnoreCase(funcName)) {
+            if (isInteger(returnType))
+                return new LongSumAggregator();
+            else if (isBigDecimal(returnType))
+                return new BigDecimalSumAggregator();
+            else if (isDouble(returnType))
+                return new DoubleSumAggregator();
+        } else if (FunctionDesc.FUNC_COUNT_DISTINCT.equalsIgnoreCase(funcName)) {
+            DataType hllcType = DataType.getInstance(returnType);
+            if (hllcType.isHLLC())
+                return new HLLCAggregator(hllcType.getPrecision());
+            else
+                return new LDCAggregator();
+        } else if (FunctionDesc.FUNC_MAX.equalsIgnoreCase(funcName)) {
+            if (isInteger(returnType))
+                return new LongMaxAggregator();
+            else if (isBigDecimal(returnType))
+                return new BigDecimalMaxAggregator();
+            else if (isDouble(returnType))
+                return new DoubleMaxAggregator();
+        } else if (FunctionDesc.FUNC_MIN.equalsIgnoreCase(funcName)) {
+            if (isInteger(returnType))
+                return new LongMinAggregator();
+            else if (isBigDecimal(returnType))
+                return new BigDecimalMinAggregator();
+            else if (isDouble(returnType))
+                return new DoubleMinAggregator();
+        } else if (FunctionDesc.FUNC_TOP_N.equalsIgnoreCase(funcName)) {
+            return new TopNAggregator();
+        }
+        throw new IllegalArgumentException("No aggregator for func '" + funcName + "' and return type '" + returnType + "'");
+    }
+
+    public static boolean isBigDecimal(String type) {
+        return type.startsWith("decimal");
+    }
+
+    public static boolean isDouble(String type) {
+        return "double".equalsIgnoreCase(type) || "float".equalsIgnoreCase(type) || "real".equalsIgnoreCase(type);
+    }
+
+    public static boolean isInteger(String type) {
+        return "long".equalsIgnoreCase(type) || "bigint".equalsIgnoreCase(type) || "int".equalsIgnoreCase(type) || "integer".equalsIgnoreCase(type);
+    }
+
+    public static int guessBigDecimalMemBytes() {
+        // 116 returned by AggregationCacheMemSizeTest
+        return 8 // aggregator obj shell
+        + 8 // ref to BigDecimal
+        + 8 // BigDecimal obj shell
+        + 100; // guess of BigDecimal internal
+    }
+
+    public static int guessDoubleMemBytes() {
+        // 29 to 44 returned by AggregationCacheMemSizeTest
+        return 44;
+        /*
+        return 8 // aggregator obj shell
+        + 8 // ref to DoubleWritable
+        + 8 // DoubleWritable obj shell
+        + 8; // size of double
+        */
+    }
+
+    public static int guessLongMemBytes() {
+        // 29 to 44 returned by AggregationCacheMemSizeTest
+        return 44;
+        /*
+        return 8 // aggregator obj shell
+        + 8 // ref to LongWritable
+        + 8 // LongWritable obj shell
+        + 8; // size of long
+        */
+    }
+
+    // ============================================================================
+
+    @SuppressWarnings("rawtypes")
+    public void setDependentAggregator(MeasureAggregator agg) {
+    }
+
+    abstract public void reset();
+
+    abstract public void aggregate(V value);
+
+    abstract public V getState();
+
+    // get an estimate of memory consumption UPPER BOUND
+    abstract public int getMemBytesEstimate();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregators.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregators.java b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregators.java
new file mode 100644
index 0000000..3aa575b
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregators.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.aggregation;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+/**
+ */
+@SuppressWarnings({ "rawtypes", "unchecked", "serial" })
+public class MeasureAggregators implements Serializable {
+
+    private final MeasureAggregator[] aggs;
+    private final int descLength;
+
+    public MeasureAggregators(Collection<MeasureDesc> measureDescs) {
+        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
+    }
+
+    public MeasureAggregators(MeasureDesc... measureDescs) {
+        descLength = measureDescs.length;
+        aggs = new MeasureAggregator[descLength];
+
+        Map<String, Integer> measureIndexMap = new HashMap<String, Integer>();
+        for (int i = 0; i < descLength; i++) {
+            FunctionDesc func = measureDescs[i].getFunction();
+            aggs[i] = MeasureAggregator.create(func.getExpression(), func.getReturnType());
+            measureIndexMap.put(measureDescs[i].getName(), i);
+        }
+        // fill back dependent aggregator
+        for (int i = 0; i < descLength; i++) {
+            String depMsrRef = measureDescs[i].getDependentMeasureRef();
+            if (depMsrRef != null) {
+                int index = measureIndexMap.get(depMsrRef);
+                aggs[i].setDependentAggregator(aggs[index]);
+            }
+        }
+    }
+
+    public void reset() {
+        for (int i = 0; i < aggs.length; i++) {
+            aggs[i].reset();
+        }
+    }
+    
+    public void aggregate(Object[] values) {
+        assert values.length == descLength;
+
+        for (int i = 0; i < descLength; i++) {
+            aggs[i].aggregate(values[i]);
+        }
+    }
+
+    public void collectStates(Object[] states) {
+        for (int i = 0; i < descLength; i++) {
+            states[i] = aggs[i].getState();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java
new file mode 100644
index 0000000..cbcb3a8
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.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.aggregation;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class MeasureCodec {
+
+    int nMeasures;
+    DataTypeSerializer[] serializers;
+
+    public MeasureCodec(Collection<MeasureDesc> measureDescs) {
+        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
+    }
+
+    public MeasureCodec(MeasureDesc... measureDescs) {
+        String[] dataTypes = new String[measureDescs.length];
+        for (int i = 0; i < dataTypes.length; i++) {
+            dataTypes[i] = measureDescs[i].getFunction().getReturnType();
+        }
+        init(dataTypes);
+    }
+
+    public MeasureCodec(String... dataTypes) {
+        init(dataTypes);
+    }
+
+    private void init(String[] dataTypes) {
+        nMeasures = dataTypes.length;
+        serializers = new DataTypeSerializer[nMeasures];
+
+        for (int i = 0; i < nMeasures; i++) {
+            serializers[i] = DataTypeSerializer.create(dataTypes[i]);
+        }
+    }
+
+    public DataTypeSerializer getSerializer(int idx) {
+        return serializers[idx];
+    }
+
+    public void decode(ByteBuffer buf, Object[] result) {
+        assert result.length == nMeasures;
+        for (int i = 0; i < nMeasures; i++) {
+            result[i] = serializers[i].deserialize(buf);
+        }
+    }
+
+    public void encode(Object[] values, ByteBuffer out) {
+        assert values.length == nMeasures;
+        for (int i = 0; i < nMeasures; i++) {
+            serializers[i].serialize(values[i], out);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMaxAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMaxAggregator.java
new file mode 100644
index 0000000..ca044d0
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMaxAggregator.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.aggregation.basic;
+
+import java.math.BigDecimal;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class BigDecimalMaxAggregator extends MeasureAggregator<BigDecimal> {
+
+    BigDecimal max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(BigDecimal value) {
+        if (max == null)
+            max = value;
+        else if (max.compareTo(value) < 0)
+            max = value;
+    }
+
+    @Override
+    public BigDecimal getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessBigDecimalMemBytes();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMinAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMinAggregator.java
new file mode 100644
index 0000000..3c3c85e
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalMinAggregator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.aggregation.basic;
+
+import java.math.BigDecimal;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class BigDecimalMinAggregator extends MeasureAggregator<BigDecimal> {
+
+    BigDecimal max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(BigDecimal value) {
+        if (max == null)
+            max = value;
+        else if (max.compareTo(value) > 0)
+            max = value;
+    }
+
+    @Override
+    public BigDecimal getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessBigDecimalMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSerializer.java
new file mode 100644
index 0000000..9f7c3cf
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSerializer.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.aggregation.basic;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.model.DataType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class BigDecimalSerializer extends DataTypeSerializer<BigDecimal> {
+
+    private static final Logger logger = LoggerFactory.getLogger(BigDecimalSerializer.class);
+
+    final DataType type;
+    final int maxLength;
+
+    int avoidVerbose = 0;
+
+    public BigDecimalSerializer(DataType type) {
+        this.type = type;
+        // see serialize(): 1 byte scale, 1 byte length, assume every 2 digits takes 1 byte
+        this.maxLength = 1 + 1 + (type.getPrecision() + 1) / 2;
+    }
+
+    @Override
+    public void serialize(BigDecimal value, ByteBuffer out) {
+        if (value.scale() > type.getScale()) {
+            if (avoidVerbose % 10000 == 0) {
+                logger.warn("value's scale has exceeded the " + type.getScale() + ", cut it off, to ensure encoded value do not exceed maxLength " + maxLength + " times:" + (avoidVerbose++));
+            }
+            value = value.setScale(type.getScale(), BigDecimal.ROUND_HALF_EVEN);
+        }
+        byte[] bytes = value.unscaledValue().toByteArray();
+        if (bytes.length + 2 > maxLength) {
+            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
+        }
+
+        BytesUtil.writeVInt(value.scale(), out);
+        BytesUtil.writeVInt(bytes.length, out);
+        out.put(bytes);
+    }
+
+    @Override
+    public BigDecimal deserialize(ByteBuffer in) {
+        int scale = BytesUtil.readVInt(in);
+        int n = BytesUtil.readVInt(in);
+
+        byte[] bytes = new byte[n];
+        in.get(bytes);
+
+        return new BigDecimal(new BigInteger(bytes), scale);
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+
+        @SuppressWarnings("unused")
+        int scale = BytesUtil.readVInt(in);
+        int n = BytesUtil.readVInt(in);
+        int len = in.position() - mark + n;
+
+        in.position(mark);
+        return len;
+    }
+
+    @Override
+    public int maxLength() {
+        return maxLength;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+    @Override
+    public BigDecimal valueOf(byte[] value) {
+        if (value == null)
+            return new BigDecimal(0);
+        else
+            return new BigDecimal(Bytes.toString(value));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSumAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSumAggregator.java
new file mode 100644
index 0000000..19aef3c
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSumAggregator.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.aggregation.basic;
+
+import java.math.BigDecimal;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class BigDecimalSumAggregator extends MeasureAggregator<BigDecimal> {
+
+    BigDecimal sum = new BigDecimal(0);
+
+    @Override
+    public void reset() {
+        sum = new BigDecimal(0);
+    }
+
+    @Override
+    public void aggregate(BigDecimal value) {
+        sum = sum.add(value);
+    }
+
+    @Override
+    public BigDecimal getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessBigDecimalMemBytes();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DateTimeSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DateTimeSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DateTimeSerializer.java
new file mode 100644
index 0000000..0bf4aba
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DateTimeSerializer.java
@@ -0,0 +1,65 @@
+package org.apache.kylin.aggregation.basic;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.common.util.LongMutable;
+import org.apache.kylin.metadata.model.DataType;
+
+public class DateTimeSerializer extends DataTypeSerializer<LongMutable> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+
+    public DateTimeSerializer(DataType type) {
+    }
+
+    @Override
+    public void serialize(LongMutable value, ByteBuffer out) {
+        out.putLong(value.get());
+    }
+
+    private LongMutable current() {
+        LongMutable l = current.get();
+        if (l == null) {
+            l = new LongMutable();
+            current.set(l);
+        }
+        return l;
+    }
+
+    @Override
+    public LongMutable deserialize(ByteBuffer in) {
+        LongMutable l = current();
+        l.set(in.getLong());
+        return l;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return 8;
+    }
+
+    @Override
+    public int maxLength() {
+        return 8;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+    @Override
+    public LongMutable valueOf(byte[] value) {
+        LongMutable l = current();
+        if (value == null)
+            l.set(0L);
+        else
+            l.set(DateFormat.stringToMillis(Bytes.toString(value)));
+        return l;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java
new file mode 100644
index 0000000..f09614d
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.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.aggregation.basic;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.util.DoubleMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class DoubleMaxAggregator extends MeasureAggregator<DoubleMutable> {
+
+    DoubleMutable max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(DoubleMutable value) {
+        if (max == null)
+            max = new DoubleMutable(value.get());
+        else if (max.get() < value.get())
+            max.set(value.get());
+    }
+
+    @Override
+    public DoubleMutable getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessDoubleMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java
new file mode 100644
index 0000000..b93c15c
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.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.aggregation.basic;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.util.DoubleMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class DoubleMinAggregator extends MeasureAggregator<DoubleMutable> {
+
+    DoubleMutable min = null;
+
+    @Override
+    public void reset() {
+        min = null;
+    }
+
+    @Override
+    public void aggregate(DoubleMutable value) {
+        if (min == null)
+            min = new DoubleMutable(value.get());
+        else if (min.get() > value.get())
+            min.set(value.get());
+    }
+
+    @Override
+    public DoubleMutable getState() {
+        return min;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessDoubleMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSerializer.java
new file mode 100644
index 0000000..f207054
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSerializer.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.aggregation.basic;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.DoubleMutable;
+import org.apache.kylin.metadata.model.DataType;
+
+/**
+ */
+public class DoubleSerializer extends DataTypeSerializer<DoubleMutable> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<DoubleMutable> current = new ThreadLocal<DoubleMutable>();
+
+    public DoubleSerializer(DataType type) {
+    }
+
+    @Override
+    public void serialize(DoubleMutable value, ByteBuffer out) {
+        out.putDouble(value.get());
+    }
+
+    private DoubleMutable current() {
+        DoubleMutable d = current.get();
+        if (d == null) {
+            d = new DoubleMutable();
+            current.set(d);
+        }
+        return d;
+    }
+
+    @Override
+    public DoubleMutable deserialize(ByteBuffer in) {
+        DoubleMutable d = current();
+        d.set(in.getDouble());
+        return d;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return 8;
+    }
+
+    @Override
+    public int maxLength() {
+        return 8;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+    @Override
+    public DoubleMutable valueOf(byte[] value) {
+        DoubleMutable d = current();
+        if (value == null)
+            d.set(0d);
+        else
+            d.set(Double.parseDouble(Bytes.toString(value)));
+        return d;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java
new file mode 100644
index 0000000..298cec6
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.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.aggregation.basic;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.util.DoubleMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class DoubleSumAggregator extends MeasureAggregator<DoubleMutable> {
+
+    DoubleMutable sum = new DoubleMutable();
+
+    @Override
+    public void reset() {
+        sum.set(0.0);
+    }
+
+    @Override
+    public void aggregate(DoubleMutable value) {
+        sum.set(sum.get() + value.get());
+    }
+
+    @Override
+    public DoubleMutable getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessDoubleMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java
new file mode 100644
index 0000000..71d95f2
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.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.aggregation.basic;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.util.LongMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class LongMaxAggregator extends MeasureAggregator<LongMutable> {
+
+    LongMutable max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(LongMutable value) {
+        if (max == null)
+            max = new LongMutable(value.get());
+        else if (max.get() < value.get())
+            max.set(value.get());
+    }
+
+    @Override
+    public LongMutable getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java
new file mode 100644
index 0000000..d1e93f2
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.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.aggregation.basic;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.util.LongMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class LongMinAggregator extends MeasureAggregator<LongMutable> {
+
+    LongMutable min = null;
+
+    @Override
+    public void reset() {
+        min = null;
+    }
+
+    @Override
+    public void aggregate(LongMutable value) {
+        if (min == null)
+            min = new LongMutable(value.get());
+        else if (min.get() > value.get())
+            min.set(value.get());
+    }
+
+    @Override
+    public LongMutable getState() {
+        return min;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSerializer.java
new file mode 100644
index 0000000..202596d
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSerializer.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.aggregation.basic;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.LongMutable;
+import org.apache.kylin.metadata.model.DataType;
+
+/**
+ */
+public class LongSerializer extends DataTypeSerializer<LongMutable> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+
+    public LongSerializer(DataType type) {
+    }
+
+    @Override
+    public void serialize(LongMutable value, ByteBuffer out) {
+        BytesUtil.writeVLong(value.get(), out);
+    }
+
+    private LongMutable current() {
+        LongMutable l = current.get();
+        if (l == null) {
+            l = new LongMutable();
+            current.set(l);
+        }
+        return l;
+    }
+
+    @Override
+    public LongMutable deserialize(ByteBuffer in) {
+        LongMutable l = current();
+        l.set(BytesUtil.readVLong(in));
+        return l;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+
+        BytesUtil.readVLong(in);
+        int len = in.position() - mark;
+
+        in.position(mark);
+        return len;
+    }
+
+    @Override
+    public int maxLength() {
+        return 9; // vlong: 1 + 8
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 5;
+    }
+
+    @Override
+    public LongMutable valueOf(byte[] value) {
+        LongMutable l = current();
+        if (value == null)
+            l.set(0L);
+        else
+            l.set(Long.parseLong(Bytes.toString(value)));
+        return l;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java
new file mode 100644
index 0000000..c85c83c
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.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.aggregation.basic;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.util.LongMutable;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class LongSumAggregator extends MeasureAggregator<LongMutable> {
+
+    LongMutable sum = new LongMutable();
+
+    @Override
+    public void reset() {
+        sum.set(0);
+    }
+
+    @Override
+    public void aggregate(LongMutable value) {
+        sum.set(sum.get() + value.get());
+    }
+
+    @Override
+    public LongMutable getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/basic/StringSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/StringSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/StringSerializer.java
new file mode 100644
index 0000000..e84278d
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/StringSerializer.java
@@ -0,0 +1,56 @@
+package org.apache.kylin.aggregation.basic;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.model.DataType;
+
+public class StringSerializer extends DataTypeSerializer<String> {
+
+    final DataType type;
+    final int maxLength;
+
+    public StringSerializer(DataType type) {
+        this.type = type;
+        // see serialize(): 2 byte length, rest is String.toBytes()
+        this.maxLength = 2 + type.getPrecision();
+    }
+
+    @Override
+    public void serialize(String value, ByteBuffer out) {
+        int start = out.position();
+
+        BytesUtil.writeUTFString(value, out);
+
+        if (out.position() - start > maxLength)
+            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
+    }
+
+    @Override
+    public String deserialize(ByteBuffer in) {
+        return BytesUtil.readUTFString(in);
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return BytesUtil.peekByteArrayLength(in);
+    }
+
+    @Override
+    public int maxLength() {
+        return maxLength;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return maxLength;
+    }
+
+    @Override
+    public String valueOf(byte[] value) {
+        return Bytes.toString(value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregator.java
new file mode 100644
index 0000000..8f85fe8
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.aggregation.hllc;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class HLLCAggregator extends MeasureAggregator<HyperLogLogPlusCounter> {
+
+    final int precision;
+    HyperLogLogPlusCounter sum = null;
+
+    public HLLCAggregator(int precision) {
+        this.precision = precision;
+    }
+
+    @Override
+    public void reset() {
+        sum = null;
+    }
+
+    @Override
+    public void aggregate(HyperLogLogPlusCounter value) {
+        if (sum == null)
+            sum = new HyperLogLogPlusCounter(value);
+        else
+            sum.merge(value);
+    }
+
+    @Override
+    public HyperLogLogPlusCounter getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        // 1024 + 60 returned by AggregationCacheMemSizeTest
+        return 8 // aggregator obj shell
+                + 4 // precision
+                + 8 // ref to HLLC
+                + 8 // HLLC obj shell
+                + 32 + (1 << precision); // HLLC internal
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java
new file mode 100644
index 0000000..f7804f4
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.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.aggregation.hllc;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.metadata.model.DataType;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class HLLCSerializer extends DataTypeSerializer<HyperLogLogPlusCounter> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<HyperLogLogPlusCounter> current = new ThreadLocal<HyperLogLogPlusCounter>();
+
+    private int precision;
+
+    public HLLCSerializer(DataType type) {
+        this.precision = type.getPrecision();
+    }
+
+    @Override
+    public void serialize(HyperLogLogPlusCounter value, ByteBuffer out) {
+        try {
+            value.writeRegisters(out);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private HyperLogLogPlusCounter current() {
+        HyperLogLogPlusCounter hllc = current.get();
+        if (hllc == null) {
+            hllc = new HyperLogLogPlusCounter(precision);
+            current.set(hllc);
+        }
+        return hllc;
+    }
+
+    @Override
+    public HyperLogLogPlusCounter deserialize(ByteBuffer in) {
+        HyperLogLogPlusCounter hllc = current();
+        try {
+            hllc.readRegisters(in);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return hllc;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return current().peekLength(in);
+    }
+
+    @Override
+    public int maxLength() {
+        return current().maxLength();
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return current().maxLength();
+    }
+
+    @Override
+    public HyperLogLogPlusCounter valueOf(byte[] value) {
+        HyperLogLogPlusCounter hllc = current();
+        hllc.clear();
+        if (value == null)
+            hllc.add("__nUlL__");
+        else
+            hllc.add(value);
+        return hllc;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java
new file mode 100644
index 0000000..643bcae
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.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.aggregation.hllc;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.util.LongMutable;
+
+/**
+ * Long Distinct Count
+ */
+@SuppressWarnings("serial")
+public class LDCAggregator extends MeasureAggregator<LongMutable> {
+
+    private static LongMutable ZERO = new LongMutable(0);
+
+    private HLLCAggregator hllAgg = null;
+    private LongMutable state = new LongMutable(0);
+
+    @SuppressWarnings("rawtypes")
+    public void setDependentAggregator(MeasureAggregator agg) {
+        this.hllAgg = (HLLCAggregator) agg;
+    }
+
+    @Override
+    public void reset() {
+    }
+
+    @Override
+    public void aggregate(LongMutable value) {
+    }
+
+    @Override
+    public LongMutable getState() {
+        if (hllAgg == null) {
+            return ZERO;
+        } else {
+            state.set(hllAgg.getState().getCountEstimate());
+            return state;
+        }
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregator.java
new file mode 100644
index 0000000..4f6c7ee
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregator.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.aggregation.topn;
+
+import java.util.Map;
+
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.topn.TopNCounter;
+import org.apache.kylin.common.util.ByteArray;
+
+import com.google.common.collect.Maps;
+
+/**
+ * 
+ */
+@SuppressWarnings("serial")
+public class TopNAggregator extends MeasureAggregator<TopNCounter<ByteArray>> {
+
+    int capacity = 0;
+    TopNCounter<ByteArray> sum = null;
+    Map<ByteArray, Double> sanityCheckMap;
+
+    @Override
+    public void reset() {
+        sum = null;
+    }
+
+    @Override
+    public void aggregate(TopNCounter<ByteArray> value) {
+        if (sum == null) {
+            capacity = value.getCapacity();
+            sum = new TopNCounter<>(capacity);
+            sanityCheckMap = Maps.newHashMap();
+        }
+        sum.merge(value);
+    }
+
+    @Override
+    public TopNCounter<ByteArray> getState() {
+        
+        //sum.retain(capacity);
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        return 8 * capacity / 4;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
new file mode 100644
index 0000000..8c44f8f
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.aggregation.topn;
+
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.common.topn.Counter;
+import org.apache.kylin.common.topn.DoubleDeltaSerializer;
+import org.apache.kylin.common.topn.TopNCounter;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.model.DataType;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * 
+ */
+public class TopNCounterSerializer extends DataTypeSerializer<TopNCounter<ByteArray>> {
+
+    private DoubleDeltaSerializer dds = new DoubleDeltaSerializer(3);
+
+    private int precision;
+
+    public TopNCounterSerializer(DataType dataType) {
+        this.precision = dataType.getPrecision();
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+        @SuppressWarnings("unused")
+        int capacity = in.getInt();
+        int size = in.getInt();
+        int keyLength = in.getInt();
+        dds.deserialize(in);
+        int len = in.position() - mark + keyLength * size;
+        in.position(mark);
+        return len;
+    }
+
+    @Override
+    public int maxLength() {
+        return precision * TopNCounter.EXTRA_SPACE_RATE * (4 + 8);
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return precision * TopNCounter.EXTRA_SPACE_RATE * 8;
+    }
+
+    @Override
+    public TopNCounter<ByteArray> valueOf(byte[] value) {
+        ByteBuffer buffer = ByteBuffer.wrap(value);
+        int sizeOfId = buffer.getInt();
+        int keyEncodedValue = buffer.getInt();
+        double counter = buffer.getDouble();
+
+        ByteArray key = new ByteArray(sizeOfId);
+        BytesUtil.writeUnsigned(keyEncodedValue, key.array(), 0, sizeOfId);
+
+        TopNCounter<ByteArray> topNCounter = new TopNCounter<ByteArray>(precision * TopNCounter.EXTRA_SPACE_RATE);
+        topNCounter.offer(key, counter);
+        return topNCounter;
+    }
+
+    @Override
+    public void serialize(TopNCounter<ByteArray> value, ByteBuffer out) {
+        double[] counters = value.getCounters();
+        List<ByteArray> peek = value.peek(1);
+        int keyLength = peek.size() > 0 ? peek.get(0).length() : 0;
+        out.putInt(value.getCapacity());
+        out.putInt(value.size());
+        out.putInt(keyLength);
+        dds.serialize(counters, out);
+        Iterator<Counter<ByteArray>> iterator = value.iterator();
+        while (iterator.hasNext()) {
+            out.put(iterator.next().getItem().array());
+        }
+    }
+
+    @Override
+    public TopNCounter<ByteArray> deserialize(ByteBuffer in) {
+        int capacity = in.getInt();
+        int size = in.getInt();
+        int keyLength = in.getInt();
+        double[] counters = dds.deserialize(in);
+
+        TopNCounter<ByteArray> counter = new TopNCounter<ByteArray>(capacity);
+        ByteArray byteArray;
+        for (int i = 0; i < size; i++) {
+            byteArray = new ByteArray(keyLength);
+            in.get(byteArray.array());
+            counter.offerToHead(byteArray, counters[i]);
+        }
+
+        return counter;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 65d639f..b88f9df 100644
--- 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
@@ -6,6 +6,9 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map;
 
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.aggregation.basic.StringSerializer;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.ImmutableBitSet;
@@ -15,9 +18,6 @@ import org.apache.kylin.gridtable.DefaultGTComparator;
 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.apache.kylin.metadata.measure.serializer.StringSerializer;
 
 /**
  * defines how column values will be encoded to/ decoded from GTRecord 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
index e4f32fb..26f1636 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
@@ -23,13 +23,13 @@ package org.apache.kylin.cube.gridtable;
 import java.nio.ByteBuffer;
 import java.util.Map;
 
+import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.aggregation.basic.StringSerializer;
 import org.apache.kylin.common.util.ImmutableBitSet;
 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.apache.kylin.metadata.measure.serializer.StringSerializer;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
 public class TrimmedCubeCodeSystem implements IGTCodeSystem {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index 7fe2122..ce912a3 100644
--- 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
@@ -29,6 +29,7 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.kylin.aggregation.MeasureAggregators;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.MemoryBudgetController;
@@ -37,7 +38,6 @@ 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.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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 e9d940a..8c6146b 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
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.TopNCounter;
+import org.apache.kylin.common.util.DoubleMutable;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.MemoryBudgetController;
 import org.apache.kylin.common.util.Pair;
@@ -46,7 +47,6 @@ 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.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
index 69a9fc9..951c054 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
@@ -18,15 +18,16 @@
 package org.apache.kylin.cube.inmemcubing;
 
 import com.google.common.base.Preconditions;
+
+import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.LongMutable;
 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.dict.Dictionary;
 import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.GTRecord;
-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;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1218bbde/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
index cd2881e..01696e8 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
@@ -7,10 +7,10 @@ import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.SortedMap;
 
+import org.apache.kylin.aggregation.MeasureAggregator;
 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.metadata.measure.MeasureAggregator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 



[11/13] incubator-kylin git commit: KYLIN-976 Add ingester; Build part done, in-mem cube test pass

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java
deleted file mode 100644
index 7430c4e..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.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.aggregation.basic;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.DoubleMutable;
-
-/**
- */
-@SuppressWarnings("serial")
-public class DoubleMinAggregator extends MeasureAggregator<DoubleMutable> {
-
-    DoubleMutable min = null;
-
-    @Override
-    public void reset() {
-        min = null;
-    }
-
-    @Override
-    public void aggregate(DoubleMutable value) {
-        if (min == null)
-            min = new DoubleMutable(value.get());
-        else if (min.get() > value.get())
-            min.set(value.get());
-    }
-
-    @Override
-    public DoubleMutable getState() {
-        return min;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessDoubleMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java
deleted file mode 100644
index 6e66c1b..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.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.aggregation.basic;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.DoubleMutable;
-
-/**
- */
-@SuppressWarnings("serial")
-public class DoubleSumAggregator extends MeasureAggregator<DoubleMutable> {
-
-    DoubleMutable sum = new DoubleMutable();
-
-    @Override
-    public void reset() {
-        sum.set(0.0);
-    }
-
-    @Override
-    public void aggregate(DoubleMutable value) {
-        sum.set(sum.get() + value.get());
-    }
-
-    @Override
-    public DoubleMutable getState() {
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessDoubleMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java
deleted file mode 100644
index 7fdf3d8..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.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.aggregation.basic;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.LongMutable;
-
-/**
- */
-@SuppressWarnings("serial")
-public class LongMaxAggregator extends MeasureAggregator<LongMutable> {
-
-    LongMutable max = null;
-
-    @Override
-    public void reset() {
-        max = null;
-    }
-
-    @Override
-    public void aggregate(LongMutable value) {
-        if (max == null)
-            max = new LongMutable(value.get());
-        else if (max.get() < value.get())
-            max.set(value.get());
-    }
-
-    @Override
-    public LongMutable getState() {
-        return max;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessLongMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java
deleted file mode 100644
index 22ae865..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.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.aggregation.basic;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.LongMutable;
-
-/**
- */
-@SuppressWarnings("serial")
-public class LongMinAggregator extends MeasureAggregator<LongMutable> {
-
-    LongMutable min = null;
-
-    @Override
-    public void reset() {
-        min = null;
-    }
-
-    @Override
-    public void aggregate(LongMutable value) {
-        if (min == null)
-            min = new LongMutable(value.get());
-        else if (min.get() > value.get())
-            min.set(value.get());
-    }
-
-    @Override
-    public LongMutable getState() {
-        return min;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessLongMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java
deleted file mode 100644
index 38d728a..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.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.aggregation.basic;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.LongMutable;
-
-/**
- */
-@SuppressWarnings("serial")
-public class LongSumAggregator extends MeasureAggregator<LongMutable> {
-
-    LongMutable sum = new LongMutable();
-
-    @Override
-    public void reset() {
-        sum.set(0);
-    }
-
-    @Override
-    public void aggregate(LongMutable value) {
-        sum.set(sum.get() + value.get());
-    }
-
-    @Override
-    public LongMutable getState() {
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessLongMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregation.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregation.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregation.java
deleted file mode 100644
index d5ceba5..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregation.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.aggregation.hllc;
-
-import java.util.List;
-
-import org.apache.kylin.aggregation.AggregationType;
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.DataTypeSerializer;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-
-public class HLLCAggregation extends AggregationType {
-    
-    private final DataType dataType;
-    
-    public HLLCAggregation(String dataType) {
-        this.dataType = DataType.getType(dataType);
-        
-        if (this.dataType.getPrecision() < 10 || this.dataType.getPrecision() > 16)
-            throw new IllegalArgumentException("HLLC precision must be between 10 and 16");
-    }
-
-    @Override
-    public DataType getAggregationDataType() {
-        return dataType;
-    }
-
-    @Override
-    public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer() {
-        return HLLCSerializer.class;
-    }
-    
-    @Override
-    public void validate(MeasureDesc measureDesc) throws IllegalArgumentException {
-        // TODO Auto-generated method stub
-        
-    }
-
-    @Override
-    public MeasureAggregator<?> newAggregator() {
-      if (dataType.isHLLC())
-          return new HLLCAggregator(dataType.getPrecision());
-      else
-          return new LDCAggregator();
-    }
-
-    @Override
-    public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregationFactory.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregationFactory.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregationFactory.java
deleted file mode 100644
index 18c021d..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregationFactory.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.aggregation.hllc;
-
-import org.apache.kylin.aggregation.AggregationType;
-import org.apache.kylin.aggregation.IAggregationFactory;
-import org.apache.kylin.metadata.model.FunctionDesc;
-
-public class HLLCAggregationFactory implements IAggregationFactory {
-
-    @Override
-    public AggregationType createAggregationType(String funcName, String dataType) {
-        if (FunctionDesc.FUNC_COUNT_DISTINCT.equalsIgnoreCase(funcName) == false)
-            throw new IllegalArgumentException();
-        
-        return new HLLCAggregation(dataType);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregator.java
deleted file mode 100644
index 8f85fe8..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregator.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.aggregation.hllc;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-
-/**
- */
-@SuppressWarnings("serial")
-public class HLLCAggregator extends MeasureAggregator<HyperLogLogPlusCounter> {
-
-    final int precision;
-    HyperLogLogPlusCounter sum = null;
-
-    public HLLCAggregator(int precision) {
-        this.precision = precision;
-    }
-
-    @Override
-    public void reset() {
-        sum = null;
-    }
-
-    @Override
-    public void aggregate(HyperLogLogPlusCounter value) {
-        if (sum == null)
-            sum = new HyperLogLogPlusCounter(value);
-        else
-            sum.merge(value);
-    }
-
-    @Override
-    public HyperLogLogPlusCounter getState() {
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        // 1024 + 60 returned by AggregationCacheMemSizeTest
-        return 8 // aggregator obj shell
-                + 4 // precision
-                + 8 // ref to HLLC
-                + 8 // HLLC obj shell
-                + 32 + (1 << precision); // HLLC internal
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java
deleted file mode 100644
index 5612892..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.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.aggregation.hllc;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.DataTypeSerializer;
-import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-
-/**
- * @author yangli9
- * 
- */
-public class HLLCSerializer extends DataTypeSerializer<HyperLogLogPlusCounter> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<HyperLogLogPlusCounter> current = new ThreadLocal<HyperLogLogPlusCounter>();
-
-    private int precision;
-
-    public HLLCSerializer(DataType type) {
-        this.precision = type.getPrecision();
-    }
-
-    @Override
-    public void serialize(HyperLogLogPlusCounter value, ByteBuffer out) {
-        try {
-            value.writeRegisters(out);
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private HyperLogLogPlusCounter current() {
-        HyperLogLogPlusCounter hllc = current.get();
-        if (hllc == null) {
-            hllc = new HyperLogLogPlusCounter(precision);
-            current.set(hllc);
-        }
-        return hllc;
-    }
-
-    @Override
-    public HyperLogLogPlusCounter deserialize(ByteBuffer in) {
-        HyperLogLogPlusCounter hllc = current();
-        try {
-            hllc.readRegisters(in);
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-        return hllc;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return current().peekLength(in);
-    }
-
-    @Override
-    public int maxLength() {
-        return current().maxLength();
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return current().maxLength();
-    }
-
-    @Override
-    public HyperLogLogPlusCounter valueOf(byte[] value) {
-        HyperLogLogPlusCounter hllc = current();
-        hllc.clear();
-        if (value == null)
-            hllc.add("__nUlL__");
-        else
-            hllc.add(value);
-        return hllc;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java
deleted file mode 100644
index 151c1ee..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.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.aggregation.hllc;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.LongMutable;
-
-/**
- * Long Distinct Count
- */
-@SuppressWarnings("serial")
-public class LDCAggregator extends MeasureAggregator<LongMutable> {
-
-    private static LongMutable ZERO = new LongMutable(0);
-
-    private HLLCAggregator hllAgg = null;
-    private LongMutable state = new LongMutable(0);
-
-    @SuppressWarnings("rawtypes")
-    public void setDependentAggregator(MeasureAggregator agg) {
-        this.hllAgg = (HLLCAggregator) agg;
-    }
-
-    @Override
-    public void reset() {
-    }
-
-    @Override
-    public void aggregate(LongMutable value) {
-    }
-
-    @Override
-    public LongMutable getState() {
-        if (hllAgg == null) {
-            return ZERO;
-        } else {
-            state.set(hllAgg.getState().getCountEstimate());
-            return state;
-        }
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return guessLongMemBytes();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregation.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregation.java b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregation.java
deleted file mode 100644
index 251abd9..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregation.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.aggregation.topn;
-
-import java.util.List;
-
-import org.apache.kylin.aggregation.AggregationType;
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.aggregation.hllc.HLLCSerializer;
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.DataTypeSerializer;
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-
-public class TopNAggregation extends AggregationType {
-
-    private final DataType dataType;
-
-    public TopNAggregation(String dataType) {
-        this.dataType = DataType.getType(dataType);
-        
-        if (this.dataType.getPrecision() < 1 || this.dataType.getPrecision() > 1000)
-            throw new IllegalArgumentException("TopN precision must be between 1 and 1000");
-    }
-
-    @Override
-    public DataType getAggregationDataType() {
-        return dataType;
-    }
-
-    @Override
-    public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer() {
-        return HLLCSerializer.class;
-    }
-    
-    @Override
-    public void validate(MeasureDesc measureDesc) throws IllegalArgumentException {
-        // TODO Auto-generated method stub
-        
-    }
-
-    @Override
-    public MeasureAggregator<?> newAggregator() {
-        return new TopNAggregator();
-    }
-
-    @Override
-    public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregationFactory.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregationFactory.java b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregationFactory.java
deleted file mode 100644
index 1ea22c8..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregationFactory.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.aggregation.topn;
-
-import org.apache.kylin.aggregation.AggregationType;
-import org.apache.kylin.aggregation.IAggregationFactory;
-import org.apache.kylin.metadata.model.FunctionDesc;
-
-public class TopNAggregationFactory implements IAggregationFactory {
-
-    @Override
-    public AggregationType createAggregationType(String funcName, String dataType) {
-        if (FunctionDesc.FUNC_TOP_N.equalsIgnoreCase(funcName) == false)
-            throw new IllegalArgumentException();
-        
-        return new TopNAggregation(dataType);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregator.java
deleted file mode 100644
index 4f6c7ee..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregator.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.aggregation.topn;
-
-import java.util.Map;
-
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.topn.TopNCounter;
-import org.apache.kylin.common.util.ByteArray;
-
-import com.google.common.collect.Maps;
-
-/**
- * 
- */
-@SuppressWarnings("serial")
-public class TopNAggregator extends MeasureAggregator<TopNCounter<ByteArray>> {
-
-    int capacity = 0;
-    TopNCounter<ByteArray> sum = null;
-    Map<ByteArray, Double> sanityCheckMap;
-
-    @Override
-    public void reset() {
-        sum = null;
-    }
-
-    @Override
-    public void aggregate(TopNCounter<ByteArray> value) {
-        if (sum == null) {
-            capacity = value.getCapacity();
-            sum = new TopNCounter<>(capacity);
-            sanityCheckMap = Maps.newHashMap();
-        }
-        sum.merge(value);
-    }
-
-    @Override
-    public TopNCounter<ByteArray> getState() {
-        
-        //sum.retain(capacity);
-        return sum;
-    }
-
-    @Override
-    public int getMemBytesEstimate() {
-        return 8 * capacity / 4;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
deleted file mode 100644
index 8088842..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.aggregation.topn;
-
-import java.nio.ByteBuffer;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.DataTypeSerializer;
-import org.apache.kylin.common.topn.Counter;
-import org.apache.kylin.common.topn.DoubleDeltaSerializer;
-import org.apache.kylin.common.topn.TopNCounter;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.BytesUtil;
-
-/**
- * 
- */
-public class TopNCounterSerializer extends DataTypeSerializer<TopNCounter<ByteArray>> {
-
-    private DoubleDeltaSerializer dds = new DoubleDeltaSerializer(3);
-
-    private int precision;
-
-    public TopNCounterSerializer(DataType dataType) {
-        this.precision = dataType.getPrecision();
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-        @SuppressWarnings("unused")
-        int capacity = in.getInt();
-        int size = in.getInt();
-        int keyLength = in.getInt();
-        dds.deserialize(in);
-        int len = in.position() - mark + keyLength * size;
-        in.position(mark);
-        return len;
-    }
-
-    @Override
-    public int maxLength() {
-        return precision * TopNCounter.EXTRA_SPACE_RATE * (4 + 8);
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return precision * TopNCounter.EXTRA_SPACE_RATE * 8;
-    }
-
-    @Override
-    public TopNCounter<ByteArray> valueOf(byte[] value) {
-        ByteBuffer buffer = ByteBuffer.wrap(value);
-        int sizeOfId = buffer.getInt();
-        int keyEncodedValue = buffer.getInt();
-        double counter = buffer.getDouble();
-
-        ByteArray key = new ByteArray(sizeOfId);
-        BytesUtil.writeUnsigned(keyEncodedValue, key.array(), 0, sizeOfId);
-
-        TopNCounter<ByteArray> topNCounter = new TopNCounter<ByteArray>(precision * TopNCounter.EXTRA_SPACE_RATE);
-        topNCounter.offer(key, counter);
-        return topNCounter;
-    }
-
-    @Override
-    public void serialize(TopNCounter<ByteArray> value, ByteBuffer out) {
-        double[] counters = value.getCounters();
-        List<ByteArray> peek = value.peek(1);
-        int keyLength = peek.size() > 0 ? peek.get(0).length() : 0;
-        out.putInt(value.getCapacity());
-        out.putInt(value.size());
-        out.putInt(keyLength);
-        dds.serialize(counters, out);
-        Iterator<Counter<ByteArray>> iterator = value.iterator();
-        while (iterator.hasNext()) {
-            out.put(iterator.next().getItem().array());
-        }
-    }
-
-    @Override
-    public TopNCounter<ByteArray> deserialize(ByteBuffer in) {
-        int capacity = in.getInt();
-        int size = in.getInt();
-        int keyLength = in.getInt();
-        double[] counters = dds.deserialize(in);
-
-        TopNCounter<ByteArray> counter = new TopNCounter<ByteArray>(capacity);
-        ByteArray byteArray;
-        for (int i = 0; i < size; i++) {
-            byteArray = new ByteArray(keyLength);
-            in.get(byteArray.array());
-            counter.offerToHead(byteArray, counters[i]);
-        }
-
-        return counter;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index 4592b15..d909a4c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -38,10 +38,10 @@ import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
 import org.apache.kylin.common.restclient.Broadcaster;
 import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.DimensionDesc;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.DictionaryInfo;
 import org.apache.kylin.dict.DictionaryManager;
 import org.apache.kylin.dict.DistinctColumnValuesProvider;
@@ -182,7 +182,8 @@ public class CubeManager implements IRealizationProvider {
     /**
      * return null if no dictionary for given column
      */
-    public Dictionary<?> getDictionary(CubeSegment cubeSeg, TblColRef col) {
+    @SuppressWarnings("unchecked")
+    public Dictionary<String> getDictionary(CubeSegment cubeSeg, TblColRef col) {
         DictionaryInfo info = null;
         try {
             DictionaryManager dictMgr = getDictionaryManager();
@@ -199,7 +200,7 @@ public class CubeManager implements IRealizationProvider {
             throw new IllegalStateException("Failed to get dictionary for cube segment" + cubeSeg + ", col" + col, e);
         }
 
-        return info.getDictionaryObject();
+        return (Dictionary<String>) info.getDictionaryObject();
     }
 
     public SnapshotTable buildSnapshotTable(CubeSegment cubeSeg, String lookupTable) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index 62df1e9..b29f83a 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -25,17 +25,17 @@ import java.util.TimeZone;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.ShardingHash;
 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.dict.Dictionary;
 import org.apache.kylin.dict.IDictionaryAware;
-import org.apache.kylin.metadata.model.IBuildable;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
-import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.metadata.realization.IRealizationSegment;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
@@ -43,8 +43,6 @@ import com.fasterxml.jackson.annotation.JsonBackReference;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Objects;
 import com.google.common.collect.Maps;
-import org.apache.kylin.metadata.realization.IRealization;
-import org.apache.kylin.metadata.realization.IRealizationSegment;
 
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, IRealizationSegment {
@@ -267,6 +265,14 @@ public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, I
     public void setStorageLocationIdentifier(String storageLocationIdentifier) {
         this.storageLocationIdentifier = storageLocationIdentifier;
     }
+    
+    public Map<TblColRef, Dictionary<String>> buildDictionaryMap() {
+        Map<TblColRef, Dictionary<String>> result = Maps.newHashMap();
+        for (TblColRef col : getCubeDesc().getAllColumnsNeedDictionary()) {
+            result.put(col, (Dictionary<String>) getDictionary(col));
+        }
+        return result;
+    }
 
     @Override
     public int getColumnLength(TblColRef col) {
@@ -279,7 +285,7 @@ public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, I
     }
 
     @Override
-    public Dictionary<?> getDictionary(TblColRef col) {
+    public Dictionary<String> getDictionary(TblColRef col) {
         return CubeManager.getInstance(this.getCubeInstance().getConfig()).getDictionary(this, col);
     }
 
@@ -427,4 +433,5 @@ public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, I
     public IJoinedFlatTableDesc getJoinedFlatTableDesc() {
         return new CubeJoinedFlatTableDesc(this.getCubeDesc(), this);
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 3619d69..7f38c26 100644
--- 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
@@ -1,23 +1,21 @@
 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.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.DataTypeSerializer;
-import org.apache.kylin.common.datatype.StringSerializer;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Dictionary;
 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.DefaultGTComparator;
 import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.IGTCodeSystem;
 import org.apache.kylin.gridtable.IGTComparator;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 
 /**
  * defines how column values will be encoded to/ decoded from GTRecord 
@@ -111,9 +109,6 @@ public class CubeCodeSystem implements IGTCodeSystem {
         if (serializer instanceof DictionarySerializer) {
             ((DictionarySerializer) serializer).serializeWithRounding(value, roundingFlag, buf);
         } else {
-            if ((value instanceof String) && (!(serializer instanceof StringSerializer || serializer instanceof FixLenSerializer))) {
-                value = serializer.valueOf((String) value);
-            }
             serializer.serialize(value, buf);
         }
     }
@@ -176,11 +171,6 @@ public class CubeCodeSystem implements IGTCodeSystem {
         }
 
         @Override
-        public Object valueOf(byte[] value) {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
         public void serialize(Object value, ByteBuffer out) {
             throw new UnsupportedOperationException();
         }
@@ -230,10 +220,6 @@ public class CubeCodeSystem implements IGTCodeSystem {
             return dictionary.getSizeOfId();
         }
 
-        @Override
-        public Object valueOf(byte[] value) {
-            throw new UnsupportedOperationException();
-        }
     }
 
     static class FixLenSerializer extends DataTypeSerializer {
@@ -306,16 +292,6 @@ public class CubeCodeSystem implements IGTCodeSystem {
             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/ce61309a/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
index aa0a530..05fc8a5 100644
--- 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
@@ -3,11 +3,11 @@ package org.apache.kylin.cube.gridtable;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.kylin.common.util.Dictionary;
 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;
 
@@ -16,24 +16,25 @@ import com.google.common.collect.Maps;
 @SuppressWarnings("rawtypes")
 public class CubeGridTable {
 
-    public static Map<TblColRef, Dictionary<?>> getDimensionToDictionaryMap(CubeSegment cubeSeg, long cuboidId) {
+    public static Map<TblColRef, Dictionary<String>> 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();
+        Map<TblColRef, Dictionary<String>> dictionaryMap = Maps.newHashMap();
         List<TblColRef> dimCols = Cuboid.findById(cubeDesc, cuboidId).getColumns();
         for (TblColRef col : dimCols) {
-            Dictionary<?> dictionary = cubeMgr.getDictionary(cubeSeg, col);
+            Dictionary<String> dictionary = cubeMgr.getDictionary(cubeSeg, col);
             if (dictionary != null) {
                 dictionaryMap.put(col, dictionary);
             }
         }
+        
         return dictionaryMap;
     }
 
     public static GTInfo newGTInfo(CubeSegment cubeSeg, long cuboidId) throws NotEnoughGTInfoException {
-        Map<TblColRef, Dictionary<?>> dictionaryMap = getDimensionToDictionaryMap(cubeSeg, cuboidId);
+        Map<TblColRef, Dictionary<String>> dictionaryMap = getDimensionToDictionaryMap(cubeSeg, cuboidId);
         Cuboid cuboid = Cuboid.findById(cubeSeg.getCubeDesc(), cuboidId);
         for (TblColRef dim : cuboid.getColumns()) {
             if (cubeSeg.getCubeDesc().getRowkey().isUseDictionary(dim)) {
@@ -47,7 +48,7 @@ public class CubeGridTable {
         return newGTInfo(cubeSeg.getCubeDesc(), cuboidId, dictionaryMap);
     }
 
-    public static GTInfo newGTInfo(CubeDesc cubeDesc, long cuboidId, Map<TblColRef, Dictionary<?>> dictionaryMap) {
+    public static GTInfo newGTInfo(CubeDesc cubeDesc, long cuboidId, Map<TblColRef, Dictionary<String>> dictionaryMap) {
         Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidId);
         CuboidToGridTableMapping mapping = new CuboidToGridTableMapping(cuboid);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 36db773..2152301 100644
--- 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
@@ -6,11 +6,11 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
 import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/NotEnoughGTInfoException.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/NotEnoughGTInfoException.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/NotEnoughGTInfoException.java
index 045b11e..9bbcf75 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/NotEnoughGTInfoException.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/NotEnoughGTInfoException.java
@@ -18,5 +18,6 @@
 
 package org.apache.kylin.cube.gridtable;
 
+@SuppressWarnings("serial")
 public class NotEnoughGTInfoException extends Exception {
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
index d30186e..c4d0a7e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
@@ -23,12 +23,12 @@ package org.apache.kylin.cube.gridtable;
 import java.nio.ByteBuffer;
 import java.util.Map;
 
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.DataTypeSerializer;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.IGTCodeSystem;
 import org.apache.kylin.gridtable.IGTComparator;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
 public class TrimmedCubeCodeSystem implements IGTCodeSystem {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 0a35559..58f94c1 100644
--- 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
@@ -21,8 +21,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
 
+import org.apache.kylin.common.util.Dictionary;
 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;
@@ -39,12 +39,12 @@ abstract public class AbstractInMemCubeBuilder {
     private static Logger logger = LoggerFactory.getLogger(AbstractInMemCubeBuilder.class);
 
     final protected CubeDesc cubeDesc;
-    final protected Map<TblColRef, Dictionary<?>> dictionaryMap;
+    final protected Map<TblColRef, Dictionary<String>> dictionaryMap;
 
     protected int taskThreadCount = 4;
     protected int reserveMemoryMB = 200;
 
-    public AbstractInMemCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<?>> dictionaryMap) {
+    public AbstractInMemCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
         if (cubeDesc == null)
             throw new NullPointerException();
         if (dictionaryMap == null)

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index ce912a3..5b6131f 100644
--- 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
@@ -29,15 +29,15 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.kylin.aggregation.MeasureAggregators;
 import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Dictionary;
 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.gridtable.GTRecord;
 import org.apache.kylin.gridtable.GTScanRequest;
 import org.apache.kylin.gridtable.IGTScanner;
+import org.apache.kylin.measure.MeasureAggregators;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -54,7 +54,7 @@ public class DoggedCubeBuilder extends AbstractInMemCubeBuilder {
     private int splitRowThreshold = Integer.MAX_VALUE;
     private int unitRows = 1000;
 
-    public DoggedCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<?>> dictionaryMap) {
+    public DoggedCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
         super(cubeDesc, dictionaryMap);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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 a393179..5c59de7 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
@@ -28,9 +28,9 @@ import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.kylin.common.datatype.DoubleMutable;
 import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.TopNCounter;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.MemoryBudgetController;
 import org.apache.kylin.common.util.Pair;
@@ -38,8 +38,6 @@ 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.dict.Dictionary;
 import org.apache.kylin.gridtable.GTAggregateScanner;
 import org.apache.kylin.gridtable.GTBuilder;
 import org.apache.kylin.gridtable.GTInfo;
@@ -47,6 +45,7 @@ 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.datatype.DoubleMutable;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
@@ -66,7 +65,6 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
     private final CuboidScheduler cuboidScheduler;
     private final long baseCuboidId;
     private final int totalCuboidCount;
-    private final CubeJoinedFlatTableDesc intermediateTableDesc;
     private final String[] metricsAggrFuncs;
     private final MeasureDesc[] measureDescs;
     private final int measureCount;
@@ -81,12 +79,11 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
     private Object[] totalSumForSanityCheck;
     private ICuboidCollector resultCollector;
 
-    public InMemCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<?>> dictionaryMap) {
+    public InMemCubeBuilder(CubeDesc cubeDesc, Map<TblColRef, Dictionary<String>> 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.measureCount = cubeDesc.getMeasures().size();
         this.measureDescs = cubeDesc.getMeasures().toArray(new MeasureDesc[measureCount]);
@@ -100,8 +97,6 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         this.metricsAggrFuncs = metricsAggrFuncsList.toArray(new String[metricsAggrFuncsList.size()]);
     }
 
-    
-
     private GridTable newGridTableByCuboidID(long cuboidID) throws IOException {
         GTInfo info = CubeGridTable.newGTInfo(cubeDesc, cuboidID, dictionaryMap);
 
@@ -114,7 +109,6 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         return gridTable;
     }
 
-
     @Override
     public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
         ConcurrentNavigableMap<Long, CuboidResult> result = build(input);
@@ -447,7 +441,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         return updateCuboidResult(cuboidId, newGridTable, count, timeSpent, 0);
     }
 
-    //@SuppressWarnings("unused")
+    @SuppressWarnings({ "unused", "rawtypes", "unchecked" })
     private void sanityCheck(long parentId, long cuboidId, Object[] totalSum) {
         // double sum introduces error and causes result not exactly equal
         for (int i = 0; i < totalSum.length; i++) {
@@ -508,9 +502,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
             this.info = info;
             this.input = input;
             this.record = new GTRecord(info);
-            this.inMemCubeBuilderInputConverter = new InMemCubeBuilderInputConverter(cubeDesc, 
-                    InMemCubeBuilderUtils.createTopNLiteralColDictionaryMap(cubeDesc, intermediateTableDesc, dictionaryMap), 
-                    info);
+            this.inMemCubeBuilderInputConverter = new InMemCubeBuilderInputConverter(cubeDesc, dictionaryMap, info);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
index bf4278a..fed9479 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
@@ -17,49 +17,39 @@
 */
 package org.apache.kylin.cube.inmemcubing;
 
-import com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
 
-import org.apache.kylin.aggregation.MeasureCodec;
-import org.apache.kylin.common.datatype.LongMutable;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.measure.MeasureIngester;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
+import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TblColRef;
 
 /**
  */
 public class InMemCubeBuilderInputConverter {
 
-    private static final LongMutable ONE = new LongMutable(1l);
-    
-    private final CubeDesc cubeDesc;
     private final CubeJoinedFlatTableDesc intermediateTableDesc;
     private final MeasureDesc[] measureDescs;
-    private final MeasureCodec measureCodec;
+    private final MeasureIngester<?>[] measureIngesters;
     private final int measureCount;
-    private final ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
-    private final Map<Integer, Dictionary<String>> topNLiteralColDictMap;
+    private final Map<TblColRef, Dictionary<String>> dictionaryMap;
     private final GTInfo gtInfo;
     
 
-    public InMemCubeBuilderInputConverter(CubeDesc cubeDesc, Map<Integer, Dictionary<String>> topNLiteralColDictMap, GTInfo gtInfo) {
-        this.cubeDesc = cubeDesc;
+    public InMemCubeBuilderInputConverter(CubeDesc cubeDesc, Map<TblColRef, Dictionary<String>> dictionaryMap, GTInfo gtInfo) {
         this.gtInfo = gtInfo;
         this.intermediateTableDesc = new CubeJoinedFlatTableDesc(cubeDesc, null);
         this.measureCount = cubeDesc.getMeasures().size();
         this.measureDescs = cubeDesc.getMeasures().toArray(new MeasureDesc[measureCount]);
-        this.measureCodec = new MeasureCodec(cubeDesc.getMeasures());
-        this.topNLiteralColDictMap = Preconditions.checkNotNull(topNLiteralColDictMap, "topNLiteralColDictMap cannot be null");
+        this.measureIngesters = MeasureIngester.create(cubeDesc.getMeasures());
+        this.dictionaryMap = dictionaryMap;
     }
     
     public final GTRecord convert(List<String> row) {
@@ -89,59 +79,38 @@ public class InMemCubeBuilderInputConverter {
     }
 
     private Object[] buildValue(List<String> row) {
-
         Object[] values = new Object[measureCount];
         for (int i = 0; i < measureCount; i++) {
-            MeasureDesc measureDesc = measureDescs[i];
-            int[] flatTableIdx = intermediateTableDesc.getMeasureColumnIndexes()[i];
-            FunctionDesc function = cubeDesc.getMeasures().get(i).getFunction();
-            if (flatTableIdx == null) {
-                values[i] = measureCodec.getSerializer(i).valueOf(measureDesc.getFunction().getParameter().getValue());
-            } else if (function.isCount() || function.isHolisticCountDistinct()) {
+            values[i] = buildValueOf(i, row);
+        }
+        return values;
+    }
+    
+    private Object buildValueOf(int idxOfMeasure, List<String> row) {
+        MeasureDesc measure = measureDescs[idxOfMeasure];
+        FunctionDesc function = measure.getFunction();
+        int[] colIdxOnFlatTable = intermediateTableDesc.getMeasureColumnIndexes()[idxOfMeasure];
+        
+        int paramCount = function.getParameterCount();
+        String[] inputToMeasure = new String[paramCount];
+
+        // pick up parameter values
+        ParameterDesc param = function.getParameter();
+        int paramColIdx = 0; // index among parameters of column type
+        for (int i = 0; i < paramCount; i++, param = param.getNextParameter()) {
+            String value;
+            if (function.isCount() || function.isHolisticCountDistinct()) {
                 // note for holistic count distinct, this value will be ignored
-                values[i] = ONE;
-            } else if (function.isTopN()) {
-                // encode the key column with dict, and get the counter column;
-                int keyColIndex = flatTableIdx[flatTableIdx.length - 1];
-                Dictionary<String> literalColDict = topNLiteralColDictMap.get(keyColIndex);
-                int keyColEncoded = literalColDict.getIdFromValue(row.get(keyColIndex));
-                valueBuf.clear();
-                valueBuf.putInt(literalColDict.getSizeOfId());
-                valueBuf.putInt(keyColEncoded);
-                if (flatTableIdx.length == 1) {
-                    // only literalCol, use 1.0 as counter
-                    valueBuf.putDouble(1.0);
-                } else {
-                    // get the counter column value
-                    valueBuf.putDouble(Double.valueOf(row.get(flatTableIdx[0])));
-                }
-
-                values[i] = measureCodec.getSerializer(i).valueOf(valueBuf.array());
-
-            } else if (flatTableIdx.length == 1) {
-                values[i] = measureCodec.getSerializer(i).valueOf(toBytes(row.get(flatTableIdx[0])));
+                value = "1";
+            } else if (param.isColumnType()) {
+                value = row.get(colIdxOnFlatTable[paramColIdx++]);
             } 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;
-                    }
-                }
-                values[i] = measureCodec.getSerializer(i).valueOf(result);
+                value = param.getValue();
             }
+            inputToMeasure[i] = value;
         }
-        return values;
-    }
-
-    private byte[] toBytes(String v) {
-        return v == null ? null : Bytes.toBytes(v);
+        
+        return measureIngesters[idxOfMeasure].valueOf(inputToMeasure, measure, dictionaryMap);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderUtils.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderUtils.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderUtils.java
index 9d819a4..e8fa6d0 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderUtils.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderUtils.java
@@ -18,11 +18,12 @@
 package org.apache.kylin.cube.inmemcubing;
 
 import com.google.common.collect.Maps;
+
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java
index 4316376..62432f7 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java
@@ -21,10 +21,10 @@ package org.apache.kylin.cube.kv;
 import java.util.Map;
 
 import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java
index 7fedd90..ba15b48 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java
@@ -22,7 +22,7 @@ import java.util.Arrays;
 
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.dict.IDictionaryAware;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java
index a21fe9f..fea3736 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java
@@ -21,7 +21,7 @@ package org.apache.kylin.cube.kv;
 import java.util.Collection;
 import java.util.Comparator;
 
-import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataType;
 
 /**
  * @author yangli9

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index ef563ed..3e8ee13 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -41,6 +41,7 @@ import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.CaseInsensitiveStringMap;
 import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
@@ -818,7 +819,8 @@ public class CubeDesc extends RootPersistentEntity {
         }
 
         for (MeasureDesc measure : measures) {
-            result.addAll(measure.getColumnsNeedDictionary());
+            MeasureType aggrType = MeasureType.create(measure.getFunction());
+            result.addAll(aggrType.getColumnsNeedDictionary(measure));
         }
         return result;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
index 8e36009..1121621 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
@@ -26,12 +26,12 @@ import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.validation.IValidatorRule;
 import org.apache.kylin.cube.model.validation.ResultLevel;
 import org.apache.kylin.cube.model.validation.ValidateContext;
 import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java b/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
index 0cfd020..3e79226 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
@@ -50,9 +50,11 @@ import com.google.common.collect.Maps;
 import com.google.common.hash.HashFunction;
 import com.google.common.hash.Hasher;
 import com.google.common.hash.Hashing;
+
 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.Dictionary;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
@@ -146,7 +148,7 @@ public class CubingUtils {
         return result;
     }
 
-    public static Map<TblColRef, Dictionary<?>> buildDictionary(final CubeInstance cubeInstance, Iterable<List<String>> recordList) throws IOException {
+    public static Map<TblColRef, Dictionary<String>> buildDictionary(final CubeInstance cubeInstance, Iterable<List<String>> recordList) throws IOException {
         final List<TblColRef> columnsNeedToBuildDictionary = cubeInstance.getDescriptor().listDimensionColumnsExcludingDerived();
         final HashMap<Integer, TblColRef> tblColRefMap = Maps.newHashMap();
         int index = 0;
@@ -154,7 +156,7 @@ public class CubingUtils {
             tblColRefMap.put(index++, column);
         }
 
-        HashMap<TblColRef, Dictionary<?>> result = Maps.newHashMap();
+        HashMap<TblColRef, Dictionary<String>> result = Maps.newHashMap();
 
         HashMultimap<TblColRef, String> valueMap = HashMultimap.create();
         for (List<String> row : recordList) {
@@ -173,18 +175,19 @@ public class CubingUtils {
                     return input == null ? null : input.getBytes();
                 }
             });
-            final Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(tblColRef.getType(), new IterableDictionaryValueEnumerator(bytes));
+            final Dictionary<String> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(tblColRef.getType(), new IterableDictionaryValueEnumerator(bytes));
             result.put(tblColRef, dict);
         }
         return result;
     }
 
-    public static Map<TblColRef, Dictionary<?>> writeDictionary(CubeSegment cubeSegment, Map<TblColRef, Dictionary<?>> dictionaryMap, long startOffset, long endOffset) {
-        Map<TblColRef, Dictionary<?>> realDictMap = Maps.newHashMap();
+    @SuppressWarnings("unchecked")
+    public static Map<TblColRef, Dictionary<String>> writeDictionary(CubeSegment cubeSegment, Map<TblColRef, Dictionary<String>> dictionaryMap, long startOffset, long endOffset) {
+        Map<TblColRef, Dictionary<String>> realDictMap = Maps.newHashMap();
 
-        for (Map.Entry<TblColRef, Dictionary<?>> entry : dictionaryMap.entrySet()) {
+        for (Map.Entry<TblColRef, Dictionary<String>> entry : dictionaryMap.entrySet()) {
             final TblColRef tblColRef = entry.getKey();
-            final Dictionary<?> dictionary = entry.getValue();
+            final Dictionary<String> dictionary = entry.getValue();
             ReadableTable.TableSignature signature = new ReadableTable.TableSignature();
             signature.setLastModifiedTime(System.currentTimeMillis());
             signature.setPath(String.format("streaming_%s_%s", startOffset, endOffset));
@@ -195,7 +198,7 @@ public class CubingUtils {
             try {
                 DictionaryInfo realDict = dictionaryManager.trySaveNewDict(dictionary, dictInfo);
                 cubeSegment.putDictResPath(tblColRef, realDict.getResourcePath());
-                realDictMap.put(tblColRef, realDict.getDictionaryObject());
+                realDictMap.put(tblColRef, (Dictionary<String>) realDict.getDictionaryObject());
             } catch (IOException e) {
                 logger.error("error save dictionary for column:" + tblColRef, e);
                 throw new RuntimeException("error save dictionary for column:" + tblColRef, e);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/main/java/org/apache/kylin/gridtable/DefaultGTComparator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/DefaultGTComparator.java b/core-cube/src/main/java/org/apache/kylin/gridtable/DefaultGTComparator.java
index f7623e3..714571f 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/DefaultGTComparator.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/DefaultGTComparator.java
@@ -1,7 +1,7 @@
 package org.apache.kylin.gridtable;
 
 import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 
 public class DefaultGTComparator implements IGTComparator {
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 01696e8..eb8d212 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
@@ -7,10 +7,10 @@ import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.SortedMap;
 
-import org.apache.kylin.aggregation.MeasureAggregator;
 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.measure.MeasureAggregator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 229c679..d3a03d1 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTInfo.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTInfo.java
@@ -5,10 +5,10 @@ import java.util.BitSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.gridtable.CubeCodeSystem;
 import org.apache.kylin.cube.util.KryoUtils;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 public class GTInfo {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 2783f55..b3133be 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java
@@ -2,9 +2,9 @@ package org.apache.kylin.gridtable;
 
 import java.nio.ByteBuffer;
 
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.DataTypeSerializer;
 import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 0e61cf2..3a22091 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/IGTCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTCodeSystem.java
@@ -2,8 +2,8 @@ package org.apache.kylin.gridtable;
 
 import java.nio.ByteBuffer;
 
-import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.common.util.ImmutableBitSet;
+import org.apache.kylin.measure.MeasureAggregator;
 
 public interface IGTCodeSystem {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index f8d7f30..ff71b4f 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
@@ -22,11 +22,11 @@ import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.List;
 
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.LongMutable;
 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.datatype.DataType;
+import org.apache.kylin.metadata.datatype.LongMutable;
 
 public class UnitTestSupport {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java b/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
deleted file mode 100644
index 8ae44b6..0000000
--- a/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-package org.apache.kylin.aggregation.topn;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.topn.TopNCounter;
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * 
- */
-public class TopNCounterSerializerTest {
-
-    private static TopNCounterSerializer serializer = new TopNCounterSerializer(DataType.getType("topn(10)"));
-
-    @Test
-    public void testSerialization() {
-        TopNCounter<ByteArray> vs = new TopNCounter<ByteArray>(50);
-        Integer[] stream = { 1, 1, 2, 9, 1, 2, 3, 7, 7, 1, 3, 1, 1 };
-        for (Integer i : stream) {
-            vs.offer(new ByteArray(Bytes.toBytes(i)));
-        }
-
-        ByteBuffer out = ByteBuffer.allocate(1024);
-        serializer.serialize(vs, out);
-        
-        byte[] copyBytes = new byte[out.position()];
-        System.arraycopy(out.array(), 0, copyBytes, 0, out.position());
-
-        ByteBuffer in = ByteBuffer.wrap(copyBytes);
-        TopNCounter<ByteArray> vsNew = serializer.deserialize(in);
-
-        Assert.assertEquals(vs.toString(), vsNew.toString());
-
-    }
-    
-    @Test
-    public void testValueOf() {
-
-        TopNCounter<ByteArray> origin = new TopNCounter<ByteArray>(10);
-        ByteArray key = new ByteArray(1);
-        ByteBuffer byteBuffer = key.asBuffer();
-        BytesUtil.writeVLong(20l, byteBuffer);
-        origin.offer(key, 1.0);
-
-        byteBuffer = ByteBuffer.allocate(1024);
-        byteBuffer.putInt(1);
-        byteBuffer.putInt(20);
-        byteBuffer.putDouble(1.0);
-        TopNCounter<ByteArray> counter = serializer.valueOf(byteBuffer.array());
-
-
-        Assert.assertEquals(origin.toString(), counter.toString());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java b/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java
index d7feb56..bce228d 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java
@@ -23,10 +23,10 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.HashSet;
 
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.DictionaryInfo;
 import org.apache.kylin.dict.DictionaryManager;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 935e840..c25bad7 100644
--- 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
@@ -26,10 +26,10 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Dictionary;
 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.gridtable.GTRecord;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.junit.AfterClass;
@@ -51,7 +51,7 @@ public class DoggedCubeBuilderStressTest extends LocalFileMetadataTestCase {
 
     private static CubeInstance cube;
     private static String flatTable;
-    private static Map<TblColRef, Dictionary<?>> dictionaryMap;
+    private static Map<TblColRef, Dictionary<String>> dictionaryMap;
 
     @BeforeClass
     public static void before() throws IOException {



[08/13] incubator-kylin git commit: KYLIN-976 Add ingester; Build part done, in-mem cube test pass

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 55235be..e79bd3a 100644
--- 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
@@ -31,13 +31,13 @@ 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.aggregation.MeasureCodec;
-import org.apache.kylin.common.datatype.LongMutable;
 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.measure.MeasureCodec;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 5145efc..5db3631 100644
--- 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
@@ -29,19 +29,19 @@ 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.datatype.DataType;
+import org.apache.kylin.common.util.Dictionary;
 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;
 import org.apache.kylin.cube.CubeUpdate;
-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.IterableDictionaryValueEnumerator;
 import org.apache.kylin.dict.TrieDictionary;
 import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.project.ProjectManager;
 import org.apache.kylin.source.ReadableTable.TableSignature;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
index 3d2badd..99b85e9 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
@@ -39,12 +39,11 @@ import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.aggregation.MeasureAggregators;
-import org.apache.kylin.aggregation.MeasureCodec;
 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.ClassUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -57,12 +56,13 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.*;
 import org.apache.kylin.cube.util.CubingUtils;
 import org.apache.kylin.dict.*;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.engine.spark.cube.BufferedCuboidWriter;
 import org.apache.kylin.engine.spark.cube.DefaultTupleConverter;
 import org.apache.kylin.engine.spark.util.IteratorUtils;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
+import org.apache.kylin.measure.MeasureAggregators;
+import org.apache.kylin.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -162,7 +162,7 @@ public class SparkCubing extends AbstractApplication {
             tblColRefMap.put(rowKeyColumnIndex, col);
         }
 
-        Map<TblColRef, Dictionary<?>> dictionaryMap = Maps.newHashMap();
+        Map<TblColRef, Dictionary<String>> dictionaryMap = Maps.newHashMap();
         for (Map.Entry<Integer, TblColRef> entry : tblColRefMap.entrySet()) {
             final String column = columns[entry.getKey()];
             final TblColRef tblColRef = entry.getValue();
@@ -309,12 +309,12 @@ public class SparkCubing extends AbstractApplication {
         for (TblColRef tblColRef : baseCuboidColumn) {
             columnLengthMap.put(tblColRef, cubeSegment.getColumnLength(tblColRef));
         }
-        final Map<TblColRef, Dictionary<?>> dictionaryMap = Maps.newHashMap();
+        final Map<TblColRef, Dictionary<String>> dictionaryMap = Maps.newHashMap();
         for (DimensionDesc dim : cubeDesc.getDimensions()) {
             // dictionary
             for (TblColRef col : dim.getColumnRefs()) {
                 if (cubeDesc.getRowkey().isUseDictionary(col)) {
-                    Dictionary<?> dict = cubeSegment.getDictionary(col);
+                    Dictionary<String> dict = cubeSegment.getDictionary(col);
                     if (dict == null) {
                         System.err.println("Dictionary for " + col + " was not found.");
                     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java
index 4a3b8b8..1965f8c 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java
@@ -36,8 +36,8 @@ package org.apache.kylin.engine.streaming;
 import java.util.Map;
 
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.StreamingBatch;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.engine.streaming.util.StreamingUtils;
 import org.apache.kylin.metadata.model.IBuildable;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -72,7 +72,7 @@ public class OneOffStreamingBuilder {
                 StreamingBatch streamingBatch = streamingInput.getBatchWithTimeWindow(streamingConfig, -1, startTime, endTime);
                 final IBuildable buildable = streamingBatchBuilder.createBuildable(streamingBatch);
                 final Map<Long, HyperLogLogPlusCounter> samplingResult = streamingBatchBuilder.sampling(streamingBatch);
-                final Map<TblColRef, Dictionary<?>> dictionaryMap = streamingBatchBuilder.buildDictionary(streamingBatch, buildable);
+                final Map<TblColRef, Dictionary<String>> dictionaryMap = streamingBatchBuilder.buildDictionary(streamingBatch, buildable);
                 streamingBatchBuilder.build(streamingBatch, dictionaryMap, streamingOutput.getCuboidWriter(buildable));
                 streamingOutput.output(buildable, samplingResult);
                 streamingBatchBuilder.commit(buildable);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java
index 93cda2d..91cdea9 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java
@@ -36,9 +36,9 @@ package org.apache.kylin.engine.streaming;
 import java.util.Map;
 
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.StreamingBatch;
 import org.apache.kylin.cube.inmemcubing.ICuboidWriter;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.metadata.model.IBuildable;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -50,9 +50,9 @@ public interface StreamingBatchBuilder {
 
     Map<Long, HyperLogLogPlusCounter> sampling(StreamingBatch streamingBatch);
 
-    Map<TblColRef, Dictionary<?>> buildDictionary(StreamingBatch streamingBatch, IBuildable buildable);
+    Map<TblColRef, Dictionary<String>> buildDictionary(StreamingBatch streamingBatch, IBuildable buildable);
 
-    void build(StreamingBatch streamingBatch, Map<TblColRef, Dictionary<?>> dictionaryMap, ICuboidWriter cuboidWriter);
+    void build(StreamingBatch streamingBatch, Map<TblColRef, Dictionary<String>> dictionaryMap, ICuboidWriter cuboidWriter);
 
     void commit(IBuildable buildable);
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java
index ae72218..044dcca 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java
@@ -53,7 +53,7 @@ import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.inmemcubing.ICuboidWriter;
 import org.apache.kylin.cube.inmemcubing.InMemCubeBuilder;
 import org.apache.kylin.cube.util.CubingUtils;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.StreamingBatch;
 import org.apache.kylin.engine.streaming.StreamingBatchBuilder;
 import org.apache.kylin.common.util.StreamingMessage;
@@ -79,7 +79,7 @@ public class StreamingCubeBuilder implements StreamingBatchBuilder {
     }
 
     @Override
-    public void build(StreamingBatch streamingBatch, Map<TblColRef, Dictionary<?>> dictionaryMap, ICuboidWriter cuboidWriter) {
+    public void build(StreamingBatch streamingBatch, Map<TblColRef, Dictionary<String>> dictionaryMap, ICuboidWriter cuboidWriter) {
         try {
 
             CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
@@ -130,10 +130,10 @@ public class StreamingCubeBuilder implements StreamingBatchBuilder {
     }
 
     @Override
-    public Map<TblColRef, Dictionary<?>> buildDictionary(StreamingBatch streamingBatch, IBuildable buildable) {
+    public Map<TblColRef, Dictionary<String>> buildDictionary(StreamingBatch streamingBatch, IBuildable buildable) {
         final CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
         final CubeInstance cubeInstance = cubeManager.reloadCubeLocal(cubeName);
-        final Map<TblColRef, Dictionary<?>> dictionaryMap;
+        final Map<TblColRef, Dictionary<String>> dictionaryMap;
         try {
             dictionaryMap = CubingUtils.buildDictionary(cubeInstance,
                     Lists.transform(streamingBatch.getMessages(), new Function<StreamingMessage, List<String>>() {
@@ -143,7 +143,7 @@ public class StreamingCubeBuilder implements StreamingBatchBuilder {
                             return input.getData();
                         }
                     }));
-            Map<TblColRef, Dictionary<?>> realDictMap = CubingUtils.writeDictionary((CubeSegment) buildable,
+            Map<TblColRef, Dictionary<String>> realDictMap = CubingUtils.writeDictionary((CubeSegment) buildable,
                     dictionaryMap,
                     streamingBatch.getTimeRange().getFirst(),
                     streamingBatch.getTimeRange().getSecond());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java
index c3ca464..dedf8de 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IISegment.java
@@ -22,7 +22,7 @@ import java.text.SimpleDateFormat;
 import java.util.List;
 import java.util.TimeZone;
 
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.dict.IDictionaryAware;
 import org.apache.kylin.invertedindex.index.TableRecordInfo;
 import org.apache.kylin.invertedindex.model.IIDesc;
@@ -36,6 +36,7 @@ import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonBackReference;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Objects;
+
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.metadata.realization.IRealizationSegment;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
index c32b767..e395544 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
@@ -24,10 +24,11 @@ import java.util.Arrays;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 
 import com.ning.compress.lzf.LZFDecoder;
 import com.ning.compress.lzf.LZFEncoder;
+
 import it.uniroma3.mat.extendedset.intset.ConciseSet;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
index aad1fa2..2d635ab 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
@@ -21,11 +21,11 @@ package org.apache.kylin.invertedindex.index;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
+import org.apache.kylin.metadata.datatype.LongMutable;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
index 16fb3a8..dc2c5c4 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
@@ -21,9 +21,10 @@ package org.apache.kylin.invertedindex.index;
 import java.util.Iterator;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 
 import com.google.common.base.Objects;
+
 import it.uniroma3.mat.extendedset.intset.ConciseSet;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/SliceBuilder.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/SliceBuilder.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/SliceBuilder.java
index 4b9d7f5..792a0cc 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/SliceBuilder.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/SliceBuilder.java
@@ -19,7 +19,8 @@ package org.apache.kylin.invertedindex.index;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
-import org.apache.kylin.dict.Dictionary;
+
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.StreamingBatch;
 import org.apache.kylin.common.util.StreamingMessage;
 import org.apache.kylin.invertedindex.model.IIDesc;
@@ -28,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
+
 import java.io.IOException;
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
index 12fcf29..3ee34be 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
@@ -22,10 +22,10 @@ import java.util.Arrays;
 
 import org.apache.commons.lang.ObjectUtils;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.ShardingHash;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.metadata.datatype.LongMutable;
 
 /**
  * TableRecord extends RawTableRecord by decorating it with a TableRecordInfo.

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
index d9e5b26..628a08d 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
@@ -20,12 +20,12 @@ package org.apache.kylin.invertedindex.index;
 
 import java.util.List;
 
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.Array;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
 import org.apache.kylin.invertedindex.model.IIDesc;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
index afb3633..09e6e4b 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
@@ -21,11 +21,11 @@ package org.apache.kylin.invertedindex.index;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.LongMutable;
 
 import com.google.common.base.Objects;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
index 42e5185..8671f43 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
@@ -19,8 +19,8 @@ package org.apache.kylin.invertedindex.measure;
 
 import java.nio.ByteBuffer;
 
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
+import org.apache.kylin.metadata.datatype.DataType;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
index 4fa8714..c9a78a4 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
@@ -20,7 +20,7 @@ package org.apache.kylin.invertedindex.measure;
 
 import java.nio.ByteBuffer;
 
-import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataType;
 
 abstract public class FixedLenMeasureCodec<T> {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
index 2b65192..67e5158 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
@@ -20,9 +20,9 @@ package org.apache.kylin.invertedindex.measure;
 
 import java.nio.ByteBuffer;
 
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.LongMutable;
 
 public class FixedPointLongCodec extends FixedLenMeasureCodec<LongMutable> {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
index 0f29d95..73a605c 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
@@ -23,17 +23,17 @@ import java.util.Collection;
 import java.util.Iterator;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.dict.DictionarySerializer;
 import org.apache.kylin.invertedindex.index.ColumnValueContainer;
 import org.apache.kylin.invertedindex.index.CompressedValueContainer;
 import org.apache.kylin.invertedindex.index.Slice;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
+import org.apache.kylin.metadata.datatype.DataType;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/main/java/org/apache/kylin/invertedindex/util/IIDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/util/IIDictionaryBuilder.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/util/IIDictionaryBuilder.java
index 2474750..4187193 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/util/IIDictionaryBuilder.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/util/IIDictionaryBuilder.java
@@ -40,7 +40,7 @@ import java.util.List;
 
 import javax.annotation.Nullable;
 
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.dict.DictionaryGenerator;
 import org.apache.kylin.dict.IterableDictionaryValueEnumerator;
 import org.apache.kylin.invertedindex.model.IIDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
index f30a876..6047514 100644
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
+++ b/invertedindex/src/test/java/org/apache/kylin/invertedindex/IIInstanceTest.java
@@ -21,8 +21,8 @@ package org.apache.kylin.invertedindex;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.invertedindex.IIDescManager;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
index d72899b..8bd19ab 100644
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
+++ b/invertedindex/src/test/java/org/apache/kylin/invertedindex/InvertedIndexLocalTest.java
@@ -34,8 +34,8 @@ import javax.annotation.Nullable;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.DictionaryGenerator;
 import org.apache.kylin.dict.IterableDictionaryValueEnumerator;
 import org.apache.kylin.invertedindex.IIInstance;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
index 600d034..03b4068 100644
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
+++ b/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
@@ -1,6 +1,6 @@
 package org.apache.kylin.invertedindex.measure;
 
-import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java
index 1a6d944..06cdbba 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/AggregationCache.java
@@ -20,8 +20,8 @@ package org.apache.kylin.storage.hbase.common.coprocessor;
 
 import java.util.Map;
 
-import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.common.util.MemoryBudgetController;
+import org.apache.kylin.measure.MeasureAggregator;
 
 import com.google.common.collect.Maps;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/FilterDecorator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/FilterDecorator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/FilterDecorator.java
index 9b12bfd..c833781 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/FilterDecorator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/common/coprocessor/FilterDecorator.java
@@ -4,9 +4,9 @@ import java.util.Collection;
 import java.util.Set;
 
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.kv.RowKeyColumnIO;
 import org.apache.kylin.dict.DictCodeSystem;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.IDictionaryAware;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
index 4d34943..0cd9014 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
@@ -36,6 +36,7 @@ import java.util.TreeSet;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -46,7 +47,6 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
 import org.apache.kylin.cube.model.HBaseMappingDesc;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.lookup.LookupStringTable;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java
index 817a1e2..85f31c4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java
@@ -13,12 +13,12 @@ import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.RowKeyDecoder;
 import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.lookup.LookupStringTable;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
index 7a75bbc..d8b61b3 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorBehavior;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorFilter;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
index b75510f..6b27aaa 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggregationCache;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
index c9343fb..521f111 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
@@ -25,13 +25,13 @@ import java.util.Comparator;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorConstants;
 import org.apache.kylin.storage.hbase.steps.RowValueDecoder;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverTuple.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverTuple.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverTuple.java
index 4952931..06c6e2c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverTuple.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverTuple.java
@@ -19,7 +19,7 @@
 package org.apache.kylin.storage.hbase.cube.v1.coprocessor.observer;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.IEvaluatableTuple;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorRowType;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeTupleConverter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeTupleConverter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeTupleConverter.java
index c89cce2..8fa63d3 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeTupleConverter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeTupleConverter.java
@@ -30,12 +30,12 @@ import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.gridtable.CuboidToGridTableMapping;
 import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.lookup.LookupStringTable;
 import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.metadata.model.FunctionDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluator.java
index 87d1b49..1f024fe 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluator.java
@@ -20,7 +20,7 @@ package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint;
 
 import java.util.List;
 
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;
 import org.apache.kylin.metadata.filter.LogicalTupleFilter;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/ClearTextDictionary.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/ClearTextDictionary.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/ClearTextDictionary.java
index a14f3da..41a2584 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/ClearTextDictionary.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/ClearTextDictionary.java
@@ -36,7 +36,7 @@ package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint;
 
 import java.util.Map;
 
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.dict.IDictionaryAware;
 import org.apache.kylin.invertedindex.index.TableRecordInfo;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
index a9d3b10..5b82aa4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationCache.java
@@ -21,7 +21,7 @@ package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggregationCache;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
index 910ed31..036d9b8 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
@@ -22,9 +22,6 @@ import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.DataType;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
@@ -32,6 +29,9 @@ import org.apache.kylin.invertedindex.index.RawTableRecord;
 import org.apache.kylin.invertedindex.index.TableRecordInfo;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorConstants;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
index 046d8c1..0fe9898 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/IIEndpoint.java
@@ -36,18 +36,18 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.kylin.aggregation.MeasureAggregator;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.CompressionUtils;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.kv.RowKeyColumnIO;
-import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.TrieDictionary;
 import org.apache.kylin.invertedindex.index.RawTableRecord;
 import org.apache.kylin.invertedindex.index.Slice;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
 import org.apache.kylin.invertedindex.model.IIDesc;
 import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
+import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorConstants;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/LocalDictionary.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/LocalDictionary.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/LocalDictionary.java
index b44ae7e..94e5b33 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/LocalDictionary.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/LocalDictionary.java
@@ -1,6 +1,6 @@
 package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint;
 
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.dict.IDictionaryAware;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 2717040..a7d0776 100644
--- 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
@@ -45,7 +45,6 @@ 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.datatype.DataType;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.ByteArray;
@@ -60,6 +59,7 @@ 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.engine.mr.common.CuboidShardUtil;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 26876ec..9c4a3bb 100644
--- 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
@@ -25,7 +25,6 @@ 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.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.CubeDesc;
@@ -34,6 +33,7 @@ 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.measure.MeasureCodec;
 
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
index d93b108..3720123 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 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.aggregation.MeasureCodec;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeSegment;
@@ -59,6 +58,7 @@ import org.apache.kylin.engine.mr.IMROutput2;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.steps.MergeCuboidMapper;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index 4037f90..dc35c56 100644
--- 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
@@ -5,11 +5,11 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.io.Text;
-import org.apache.kylin.aggregation.MeasureCodec;
 import org.apache.kylin.common.util.Bytes;
 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.measure.MeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index b247ad2..617af76 100644
--- 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
@@ -23,11 +23,11 @@ import java.util.BitSet;
 import java.util.Collection;
 
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.kylin.aggregation.MeasureCodec;
-import org.apache.kylin.common.datatype.DoubleMutable;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
+import org.apache.kylin.measure.MeasureCodec;
+import org.apache.kylin.metadata.datatype.DoubleMutable;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
index 873fb37..cd4e33d 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
@@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluatorTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluatorTest.java
index da1cf71..61df5cf 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluatorTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/BitMapFilterEvaluatorTest.java
@@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.ArrayList;
 
-import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;
@@ -36,6 +36,7 @@ import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.BitMapFilterEvalua
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
+
 import it.uniroma3.mat.extendedset.intset.ConciseSet;
 
 public class BitMapFilterEvaluatorTest {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
index e2e5724..04e1f7f 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
@@ -20,14 +20,14 @@ package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint;
 
 import com.google.common.collect.Lists;
 
-import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.invertedindex.index.TableRecordInfo;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/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
index cf706e4..b715498 100644
--- 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
@@ -29,13 +29,13 @@ 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.aggregation.MeasureCodec;
 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.engine.mr.HadoopUtil;
+import org.apache.kylin.measure.MeasureCodec;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ce61309a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
index 276465e..7abdc26 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
@@ -25,14 +25,14 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 
 import org.apache.hadoop.io.LongWritable;
-import org.apache.kylin.aggregation.MeasureCodec;
-import org.apache.kylin.common.datatype.LongMutable;
 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.cube.model.HBaseColumnDesc;
+import org.apache.kylin.measure.MeasureCodec;
 import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.junit.After;


[06/13] incubator-kylin git commit: KYLIN-976 AggregationType interface, serializer and aggregator

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSerializer.java
deleted file mode 100644
index 202596d..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSerializer.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.aggregation.basic;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.aggregation.DataTypeSerializer;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.LongMutable;
-import org.apache.kylin.metadata.model.DataType;
-
-/**
- */
-public class LongSerializer extends DataTypeSerializer<LongMutable> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
-
-    public LongSerializer(DataType type) {
-    }
-
-    @Override
-    public void serialize(LongMutable value, ByteBuffer out) {
-        BytesUtil.writeVLong(value.get(), out);
-    }
-
-    private LongMutable current() {
-        LongMutable l = current.get();
-        if (l == null) {
-            l = new LongMutable();
-            current.set(l);
-        }
-        return l;
-    }
-
-    @Override
-    public LongMutable deserialize(ByteBuffer in) {
-        LongMutable l = current();
-        l.set(BytesUtil.readVLong(in));
-        return l;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-
-        BytesUtil.readVLong(in);
-        int len = in.position() - mark;
-
-        in.position(mark);
-        return len;
-    }
-
-    @Override
-    public int maxLength() {
-        return 9; // vlong: 1 + 8
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 5;
-    }
-
-    @Override
-    public LongMutable valueOf(byte[] value) {
-        LongMutable l = current();
-        if (value == null)
-            l.set(0L);
-        else
-            l.set(Long.parseLong(Bytes.toString(value)));
-        return l;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java
index c85c83c..38d728a 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongSumAggregator.java
@@ -19,7 +19,7 @@
 package org.apache.kylin.aggregation.basic;
 
 import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.util.LongMutable;
+import org.apache.kylin.common.datatype.LongMutable;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/StringSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/StringSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/StringSerializer.java
deleted file mode 100644
index e84278d..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/StringSerializer.java
+++ /dev/null
@@ -1,56 +0,0 @@
-package org.apache.kylin.aggregation.basic;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.aggregation.DataTypeSerializer;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.model.DataType;
-
-public class StringSerializer extends DataTypeSerializer<String> {
-
-    final DataType type;
-    final int maxLength;
-
-    public StringSerializer(DataType type) {
-        this.type = type;
-        // see serialize(): 2 byte length, rest is String.toBytes()
-        this.maxLength = 2 + type.getPrecision();
-    }
-
-    @Override
-    public void serialize(String value, ByteBuffer out) {
-        int start = out.position();
-
-        BytesUtil.writeUTFString(value, out);
-
-        if (out.position() - start > maxLength)
-            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
-    }
-
-    @Override
-    public String deserialize(ByteBuffer in) {
-        return BytesUtil.readUTFString(in);
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return BytesUtil.peekByteArrayLength(in);
-    }
-
-    @Override
-    public int maxLength() {
-        return maxLength;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return maxLength;
-    }
-
-    @Override
-    public String valueOf(byte[] value) {
-        return Bytes.toString(value);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregation.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregation.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregation.java
new file mode 100644
index 0000000..d5ceba5
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregation.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.aggregation.hllc;
+
+import java.util.List;
+
+import org.apache.kylin.aggregation.AggregationType;
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.DataTypeSerializer;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class HLLCAggregation extends AggregationType {
+    
+    private final DataType dataType;
+    
+    public HLLCAggregation(String dataType) {
+        this.dataType = DataType.getType(dataType);
+        
+        if (this.dataType.getPrecision() < 10 || this.dataType.getPrecision() > 16)
+            throw new IllegalArgumentException("HLLC precision must be between 10 and 16");
+    }
+
+    @Override
+    public DataType getAggregationDataType() {
+        return dataType;
+    }
+
+    @Override
+    public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer() {
+        return HLLCSerializer.class;
+    }
+    
+    @Override
+    public void validate(MeasureDesc measureDesc) throws IllegalArgumentException {
+        // TODO Auto-generated method stub
+        
+    }
+
+    @Override
+    public MeasureAggregator<?> newAggregator() {
+      if (dataType.isHLLC())
+          return new HLLCAggregator(dataType.getPrecision());
+      else
+          return new LDCAggregator();
+    }
+
+    @Override
+    public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregationFactory.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregationFactory.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregationFactory.java
new file mode 100644
index 0000000..18c021d
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCAggregationFactory.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.aggregation.hllc;
+
+import org.apache.kylin.aggregation.AggregationType;
+import org.apache.kylin.aggregation.IAggregationFactory;
+import org.apache.kylin.metadata.model.FunctionDesc;
+
+public class HLLCAggregationFactory implements IAggregationFactory {
+
+    @Override
+    public AggregationType createAggregationType(String funcName, String dataType) {
+        if (FunctionDesc.FUNC_COUNT_DISTINCT.equalsIgnoreCase(funcName) == false)
+            throw new IllegalArgumentException();
+        
+        return new HLLCAggregation(dataType);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java
index f7804f4..5612892 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/HLLCSerializer.java
@@ -21,9 +21,9 @@ package org.apache.kylin.aggregation.hllc;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.kylin.aggregation.DataTypeSerializer;
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.DataTypeSerializer;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.metadata.model.DataType;
 
 /**
  * @author yangli9

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java
index 643bcae..151c1ee 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/hllc/LDCAggregator.java
@@ -19,7 +19,7 @@
 package org.apache.kylin.aggregation.hllc;
 
 import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.util.LongMutable;
+import org.apache.kylin.common.datatype.LongMutable;
 
 /**
  * Long Distinct Count

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregation.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregation.java b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregation.java
new file mode 100644
index 0000000..251abd9
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregation.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.aggregation.topn;
+
+import java.util.List;
+
+import org.apache.kylin.aggregation.AggregationType;
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.aggregation.hllc.HLLCSerializer;
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.DataTypeSerializer;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class TopNAggregation extends AggregationType {
+
+    private final DataType dataType;
+
+    public TopNAggregation(String dataType) {
+        this.dataType = DataType.getType(dataType);
+        
+        if (this.dataType.getPrecision() < 1 || this.dataType.getPrecision() > 1000)
+            throw new IllegalArgumentException("TopN precision must be between 1 and 1000");
+    }
+
+    @Override
+    public DataType getAggregationDataType() {
+        return dataType;
+    }
+
+    @Override
+    public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer() {
+        return HLLCSerializer.class;
+    }
+    
+    @Override
+    public void validate(MeasureDesc measureDesc) throws IllegalArgumentException {
+        // TODO Auto-generated method stub
+        
+    }
+
+    @Override
+    public MeasureAggregator<?> newAggregator() {
+        return new TopNAggregator();
+    }
+
+    @Override
+    public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregationFactory.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregationFactory.java b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregationFactory.java
new file mode 100644
index 0000000..1ea22c8
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNAggregationFactory.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.aggregation.topn;
+
+import org.apache.kylin.aggregation.AggregationType;
+import org.apache.kylin.aggregation.IAggregationFactory;
+import org.apache.kylin.metadata.model.FunctionDesc;
+
+public class TopNAggregationFactory implements IAggregationFactory {
+
+    @Override
+    public AggregationType createAggregationType(String funcName, String dataType) {
+        if (FunctionDesc.FUNC_TOP_N.equalsIgnoreCase(funcName) == false)
+            throw new IllegalArgumentException();
+        
+        return new TopNAggregation(dataType);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
index 8c44f8f..8088842 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/topn/TopNCounterSerializer.java
@@ -18,17 +18,17 @@
 
 package org.apache.kylin.aggregation.topn;
 
-import org.apache.kylin.aggregation.DataTypeSerializer;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.DataTypeSerializer;
 import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.DoubleDeltaSerializer;
 import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.model.DataType;
-
-import java.nio.ByteBuffer;
-import java.util.Iterator;
-import java.util.List;
 
 /**
  * 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index b88f9df..3619d69 100644
--- 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
@@ -6,9 +6,9 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map;
 
-import org.apache.kylin.aggregation.DataTypeSerializer;
 import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.aggregation.basic.StringSerializer;
+import org.apache.kylin.common.datatype.DataTypeSerializer;
+import org.apache.kylin.common.datatype.StringSerializer;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.ImmutableBitSet;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index 33db11b..aa0a530 100644
--- 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
@@ -11,7 +11,6 @@ import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.metadata.model.TblColRef;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
 @SuppressWarnings("rawtypes")

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index c95e932..36db773 100644
--- 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
@@ -6,11 +6,11 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.cuboid.Cuboid;
 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;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
index 26f1636..d30186e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
@@ -23,9 +23,8 @@ package org.apache.kylin.cube.gridtable;
 import java.nio.ByteBuffer;
 import java.util.Map;
 
-import org.apache.kylin.aggregation.DataTypeSerializer;
 import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.aggregation.basic.StringSerializer;
+import org.apache.kylin.common.datatype.DataTypeSerializer;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.IGTCodeSystem;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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 8c6146b..a393179 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
@@ -28,9 +28,9 @@ import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.kylin.common.datatype.DoubleMutable;
 import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.TopNCounter;
-import org.apache.kylin.common.util.DoubleMutable;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.MemoryBudgetController;
 import org.apache.kylin.common.util.Pair;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
index 951c054..bf4278a 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
@@ -20,8 +20,8 @@ package org.apache.kylin.cube.inmemcubing;
 import com.google.common.base.Preconditions;
 
 import org.apache.kylin.aggregation.MeasureCodec;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java
index d7a48d9..a21fe9f 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnOrder.java
@@ -21,7 +21,7 @@ package org.apache.kylin.cube.kv;
 import java.util.Collection;
 import java.util.Comparator;
 
-import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.common.datatype.DataType;
 
 /**
  * @author yangli9

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
index 1920fc7..8e36009 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
@@ -26,13 +26,13 @@ import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.validation.IValidatorRule;
 import org.apache.kylin.cube.model.validation.ResultLevel;
 import org.apache.kylin.cube.model.validation.ValidateContext;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
-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.ParameterDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index e3d3640..229c679 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTInfo.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTInfo.java
@@ -5,10 +5,10 @@ import java.util.BitSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.gridtable.CubeCodeSystem;
 import org.apache.kylin.cube.util.KryoUtils;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 public class GTInfo {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index be92f73..2783f55 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTSampleCodeSystem.java
@@ -2,8 +2,8 @@ package org.apache.kylin.gridtable;
 
 import java.nio.ByteBuffer;
 
-import org.apache.kylin.aggregation.DataTypeSerializer;
 import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.datatype.DataTypeSerializer;
 import org.apache.kylin.common.util.ImmutableBitSet;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index 9852acc..f8d7f30 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
@@ -22,11 +22,11 @@ import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.List;
 
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.gridtable.GTInfo.Builder;
-import org.apache.kylin.metadata.model.DataType;
 
 public class UnitTestSupport {
 
@@ -48,11 +48,11 @@ public class UnitTestSupport {
         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") //
+                DataType.getType("varchar(10)"), //
+                DataType.getType("varchar(10)"), //
+                DataType.getType("varchar(10)"), //
+                DataType.getType("bigint"), //
+                DataType.getType("decimal") //
         );
         builder.setPrimaryKey(setOf(0));
         builder.setColumnPreferIndex(setOf(0));

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/test/java/org/apache/kylin/aggregation/basic/BigDecimalSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/aggregation/basic/BigDecimalSerializerTest.java b/core-cube/src/test/java/org/apache/kylin/aggregation/basic/BigDecimalSerializerTest.java
deleted file mode 100644
index 27aa07c..0000000
--- a/core-cube/src/test/java/org/apache/kylin/aggregation/basic/BigDecimalSerializerTest.java
+++ /dev/null
@@ -1,53 +0,0 @@
-package org.apache.kylin.aggregation.basic;
-
-import static org.junit.Assert.assertEquals;
-
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.aggregation.basic.BigDecimalSerializer;
-import org.apache.kylin.metadata.model.DataType;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- */
-public class BigDecimalSerializerTest {
-
-    private static BigDecimalSerializer bigDecimalSerializer;
-
-    @BeforeClass
-    public static void beforeClass() {
-        bigDecimalSerializer = new BigDecimalSerializer(DataType.getInstance("decimal"));
-    }
-
-    @Test
-    public void testNormal() {
-        BigDecimal input = new BigDecimal("1234.1234");
-        ByteBuffer buffer = ByteBuffer.allocate(256);
-        buffer.mark();
-        bigDecimalSerializer.serialize(input, buffer);
-        buffer.reset();
-        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
-        assertEquals(input, output);
-    }
-
-    @Test
-    public void testScaleOutOfRange() {
-        BigDecimal input = new BigDecimal("1234.1234567890");
-        ByteBuffer buffer = ByteBuffer.allocate(256);
-        buffer.mark();
-        bigDecimalSerializer.serialize(input, buffer);
-        buffer.reset();
-        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
-        assertEquals(input.setScale(bigDecimalSerializer.type.getScale(), BigDecimal.ROUND_HALF_EVEN), output);
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testOutOfPrecision() {
-        BigDecimal input = new BigDecimal("66855344214907231736.4924");
-        ByteBuffer buffer = ByteBuffer.allocate(256);
-        bigDecimalSerializer.serialize(input, buffer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java b/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
index 8ee7b8d..8ae44b6 100644
--- a/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/aggregation/topn/TopNCounterSerializerTest.java
@@ -1,22 +1,21 @@
 package org.apache.kylin.aggregation.topn;
 
-import org.apache.kylin.aggregation.topn.TopNCounterSerializer;
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.topn.TopNCounter;
 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.metadata.model.DataType;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.nio.ByteBuffer;
-
 /**
  * 
  */
 public class TopNCounterSerializerTest {
 
-    private static TopNCounterSerializer serializer = new TopNCounterSerializer(DataType.getInstance("topn(10)"));
+    private static TopNCounterSerializer serializer = new TopNCounterSerializer(DataType.getType("topn(10)"));
 
     @Test
     public void testSerialization() {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index f407977..93f0419 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
@@ -28,10 +28,10 @@ import org.apache.kylin.aggregation.basic.BigDecimalSumAggregator;
 import org.apache.kylin.aggregation.basic.DoubleSumAggregator;
 import org.apache.kylin.aggregation.basic.LongSumAggregator;
 import org.apache.kylin.aggregation.hllc.HLLCAggregator;
+import org.apache.kylin.common.datatype.DoubleMutable;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.DoubleMutable;
-import org.apache.kylin.common.util.LongMutable;
 import org.junit.Test;
 
 public class AggregationCacheMemSizeTest {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index f5247e2..b3981e8 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/DictGridTableTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.kylin.gridtable;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.math.BigDecimal;
@@ -27,9 +27,10 @@ import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.gridtable.CubeCodeSystem;
 import org.apache.kylin.dict.Dictionary;
@@ -46,7 +47,6 @@ import org.apache.kylin.metadata.filter.LogicalTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
 import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.junit.Before;
@@ -408,11 +408,11 @@ public class DictGridTableTest {
         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") //
+                DataType.getType("timestamp"), //
+                DataType.getType("integer"), //
+                DataType.getType("varchar(10)"), //
+                DataType.getType("bigint"), //
+                DataType.getType("decimal") //
         );
         builder.setPrimaryKey(setOf(0, 1));
         builder.setColumnPreferIndex(setOf(0));

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index 361617a..4313f4b 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
@@ -26,8 +26,8 @@ import java.math.BigDecimal;
 import java.util.BitSet;
 import java.util.List;
 
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.gridtable.memstore.GTSimpleMemStore;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index a964e67..7f4da61 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
@@ -17,29 +17,28 @@
 
 package org.apache.kylin.gridtable;
 
-import static org.junit.Assert.assertEquals;
+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.aggregation.basic.StringSerializer;
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.LongMutable;
+import org.apache.kylin.common.datatype.StringSerializer;
 import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;
 import org.apache.kylin.metadata.filter.LogicalTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
 
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
 public class SimpleInvertedIndexTest {
 
     GTInfo info;
@@ -167,7 +166,7 @@ public class SimpleInvertedIndexTest {
     public static ConstantTupleFilter constFilter(int id) {
         byte[] space = new byte[10];
         ByteBuffer buf = ByteBuffer.wrap(space);
-        StringSerializer stringSerializer = new StringSerializer(DataType.getInstance("string"));
+        StringSerializer stringSerializer = new StringSerializer(DataType.getType("string"));
         stringSerializer.serialize("" + id, buf);
         ByteArray data = new ByteArray(buf.array(), buf.arrayOffset(), buf.position());
         return new ConstantTupleFilter(data);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
index 0e13d7e..4da5471 100644
--- a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
@@ -24,9 +24,9 @@ import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
 import org.apache.kylin.aggregation.MeasureCodec;
+import org.apache.kylin.common.datatype.DoubleMutable;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.common.util.DoubleMutable;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
index 8027f41..300c240 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
@@ -21,20 +21,19 @@ package org.apache.kylin.dict;
 import java.io.IOException;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.JsonUtil;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.source.ReadableTable;
-import org.apache.kylin.source.ReadableTable.TableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 
 /**
  * @author yangli9
@@ -105,7 +104,7 @@ public class DictionaryGenerator {
             logger.debug("Building dictionary object " + JsonUtil.writeValueAsString(info));
 
             columnValueEnumerator = new TableColumnValueEnumerator(inpTable.getReader(), info.getSourceColumnIndex());
-            return buildDictionaryFromValueEnumerator(DataType.getInstance(info.getDataType()), columnValueEnumerator);
+            return buildDictionaryFromValueEnumerator(DataType.getType(info.getDataType()), columnValueEnumerator);
         } finally {
             if (columnValueEnumerator != null)
                 columnValueEnumerator.close();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
index 981e19a..2f4b761 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
@@ -30,10 +30,10 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.DataModelDesc;
-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.source.ReadableTable;
@@ -218,7 +218,7 @@ public class DictionaryManager {
             logger.info("Use one of the merging dictionaries directly");
             return dicts.get(0);
         } else {
-            Dictionary<?> newDict = DictionaryGenerator.mergeDictionaries(DataType.getInstance(newDictInfo.getDataType()), dicts);
+            Dictionary<?> newDict = DictionaryGenerator.mergeDictionaries(DataType.getType(newDictInfo.getDataType()), dicts);
             return trySaveNewDict(newDict, newDictInfo);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
index e51153c..8c8dcfc 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
@@ -18,10 +18,9 @@
 
 package org.apache.kylin.dict;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
-import java.io.IOException;
+import java.io.IOException;
 import java.math.BigDecimal;
 import java.util.Collection;
 import java.util.Collections;
@@ -29,8 +28,8 @@ import java.util.List;
 import java.util.Random;
 import java.util.Set;
 
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.metadata.model.DataType;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
@@ -54,7 +53,7 @@ public class NumberDictionaryTest {
         }
 
         // check "" is treated as NULL, not a code of dictionary
-        Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(DataType.getInstance("integer"), new IterableDictionaryValueEnumerator(intBytes));
+        Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(DataType.getType("integer"), new IterableDictionaryValueEnumerator(intBytes));
         assertEquals(4, dict.getSize());
 
         final int id = ((NumberDictionary<String>) dict).getIdFromValue("");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java
index e4bdf3d..0e687f7 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TimeConditionLiteralsReplacer.java
@@ -5,8 +5,8 @@ import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.google.common.collect.Maps;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
index 6162477..f03e736 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
@@ -18,14 +18,15 @@
 
 package org.apache.kylin.metadata.model;
 
+import java.io.Serializable;
+
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.datatype.DataType;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonProperty;
 
-import java.io.Serializable;
-
 /**
  * Column Metadata from Source. All name should be uppercase.
  * <p/>
@@ -72,7 +73,7 @@ public class ColumnDesc implements Serializable {
     public void setDatatype(String datatype) {
         //logger.info("setting datatype to " + datatype);
         this.datatype = datatype;
-        type = DataType.getInstance(datatype);
+        type = DataType.getType(datatype);
     }
 
     public String getId() {
@@ -132,7 +133,7 @@ public class ColumnDesc implements Serializable {
         if (id != null)
             zeroBasedIndex = Integer.parseInt(id) - 1;
 
-        DataType normalized = DataType.getInstance(datatype);
+        DataType normalized = DataType.getType(datatype);
         if (normalized == null) {
             this.setDatatype(null);
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java
deleted file mode 100644
index 1333426..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java
+++ /dev/null
@@ -1,295 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.metadata.model;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.kylin.aggregation.DataTypeSerializer;
-
-/**
- */
-@SuppressWarnings("serial")
-public class DataType implements Serializable {
-
-    public static final String VALID_TYPES_STRING = "any|char|varchar|boolean|binary" //
-            + "|integer|tinyint|smallint|bigint|decimal|numeric|float|real|double" //
-            + "|date|time|datetime|timestamp|byte|int|short|long|string|hllc|topn" //
-            + "|" + TblColRef.InnerDataTypeEnum.LITERAL.getDataType() //
-            + "|" + TblColRef.InnerDataTypeEnum.DERIVED.getDataType();
-
-    private static final Pattern TYPE_PATTERN = Pattern.compile(
-    // standard sql types, ref:
-    // http://www.w3schools.com/sql/sql_datatypes_general.asp
-            "(" + VALID_TYPES_STRING + ")" + "\\s*" //
-                    + "(?:" + "[(]" + "([\\d\\s,]+)" + "[)]" + ")?", Pattern.CASE_INSENSITIVE);
-
-    public static final Set<String> INTEGER_FAMILY = new HashSet<String>();
-    public static final Set<String> NUMBER_FAMILY = new HashSet<String>();
-    public static final Set<String> DATETIME_FAMILY = new HashSet<String>();
-    public static final Set<String> STRING_FAMILY = new HashSet<String>();
-    private static final Set<Integer> HLLC_PRECISIONS = new HashSet<Integer>();
-    private static final Map<String, String> LEGACY_TYPE_MAP = new HashMap<String, String>();
-    static {
-        INTEGER_FAMILY.add("tinyint");
-        INTEGER_FAMILY.add("smallint");
-        INTEGER_FAMILY.add("integer");
-        INTEGER_FAMILY.add("bigint");
-
-        NUMBER_FAMILY.addAll(INTEGER_FAMILY);
-        NUMBER_FAMILY.add("float");
-        NUMBER_FAMILY.add("double");
-        NUMBER_FAMILY.add("decimal");
-        NUMBER_FAMILY.add("real");
-        NUMBER_FAMILY.add("numeric");
-
-        DATETIME_FAMILY.add("date");
-        DATETIME_FAMILY.add("time");
-        DATETIME_FAMILY.add("datetime");
-        DATETIME_FAMILY.add("timestamp");
-
-        STRING_FAMILY.add("varchar");
-        STRING_FAMILY.add("char");
-
-        LEGACY_TYPE_MAP.put("byte", "tinyint");
-        LEGACY_TYPE_MAP.put("int", "integer");
-        LEGACY_TYPE_MAP.put("short", "smallint");
-        LEGACY_TYPE_MAP.put("long", "bigint");
-        LEGACY_TYPE_MAP.put("string", "varchar");
-        LEGACY_TYPE_MAP.put("hllc10", "hllc(10)");
-        LEGACY_TYPE_MAP.put("hllc12", "hllc(12)");
-        LEGACY_TYPE_MAP.put("hllc14", "hllc(14)");
-        LEGACY_TYPE_MAP.put("hllc15", "hllc(15)");
-        LEGACY_TYPE_MAP.put("hllc16", "hllc(16)");
-
-        for (int i = 10; i <= 16; i++)
-            HLLC_PRECISIONS.add(i);
-    }
-
-    private static final ConcurrentMap<DataType, DataType> CACHE = new ConcurrentHashMap<DataType, DataType>();
-
-    public static final DataType ANY = DataType.getInstance("any");
-
-    public static DataType getInstance(String type) {
-        if (type == null)
-            return null;
-
-        DataType dataType = new DataType(type);
-        DataType cached = CACHE.get(dataType);
-        if (cached == null) {
-            CACHE.put(dataType, dataType);
-            cached = dataType;
-        }
-        return cached;
-    }
-
-    // ============================================================================
-
-    private String name;
-    private int precision;
-    private int scale;
-
-    DataType(String datatype) {
-        parseDataType(datatype);
-    }
-
-    private void parseDataType(String datatype) {
-        datatype = datatype.trim().toLowerCase();
-        datatype = replaceLegacy(datatype);
-
-        Matcher m = TYPE_PATTERN.matcher(datatype);
-        if (m.matches() == false)
-            throw new IllegalArgumentException("bad data type -- " + datatype + ", does not match " + TYPE_PATTERN);
-
-        name = replaceLegacy(m.group(1));
-        precision = -1;
-        scale = -1;
-
-        String leftover = m.group(2);
-        if (leftover != null) {
-            String[] parts = leftover.split("\\s*,\\s*");
-            for (int i = 0; i < parts.length; i++) {
-                int n;
-                try {
-                    n = Integer.parseInt(parts[i]);
-                } catch (NumberFormatException e) {
-                    throw new IllegalArgumentException("bad data type -- " + datatype + ", precision/scale not numeric");
-                }
-                if (i == 0)
-                    precision = n;
-                else if (i == 1)
-                    scale = n;
-                else
-                    throw new IllegalArgumentException("bad data type -- " + datatype + ", too many precision/scale parts");
-            }
-        }
-
-        // FIXME 256 for unknown string precision
-        if ((name.equals("char") || name.equals("varchar")) && precision == -1) {
-            precision = 256; // to save memory at frontend, e.g. tableau will
-                             // allocate memory according to this
-        }
-
-        // FIXME (19,4) for unknown decimal precision
-        if ((name.equals("decimal") || name.equals("numeric")) && precision == -1) {
-            precision = 19;
-            scale = 4;
-        }
-
-        if (isHLLC() && HLLC_PRECISIONS.contains(precision) == false)
-            throw new IllegalArgumentException("HLLC precision must be one of " + HLLC_PRECISIONS);
-    }
-
-    private String replaceLegacy(String str) {
-        String replace = LEGACY_TYPE_MAP.get(str);
-        return replace == null ? str : replace;
-    }
-
-    public int getStorageBytesEstimate() {
-        return DataTypeSerializer.create(this).getStorageBytesEstimate();
-    }
-
-    public boolean isStringFamily() {
-        return STRING_FAMILY.contains(name);
-    }
-
-    public boolean isIntegerFamily() {
-        return INTEGER_FAMILY.contains(name);
-    }
-
-    public boolean isNumberFamily() {
-        return NUMBER_FAMILY.contains(name);
-    }
-
-    public boolean isDateTimeFamily() {
-        return DATETIME_FAMILY.contains(name);
-    }
-
-    public boolean isDate() {
-        return name.equals("date");
-    }
-
-    public boolean isTime() {
-        return name.equals("time");
-    }
-
-    public boolean isTimestamp() {
-        return name.equals("timestamp");
-    }
-
-    public boolean isDatetime() {
-        return name.equals("datetime");
-    }
-
-    public boolean isTinyInt() {
-        return name.equals("tinyint");
-    }
-
-    public boolean isSmallInt() {
-        return name.equals("smallint");
-    }
-
-    public boolean isInt() {
-        return name.equals("integer");
-    }
-
-    public boolean isBigInt() {
-        return name.equals("bigint");
-    }
-
-    public boolean isFloat() {
-        return name.equals("float");
-    }
-
-    public boolean isDouble() {
-        return name.equals("double");
-    }
-
-    public boolean isDecimal() {
-        return name.equals("decimal");
-    }
-
-    public boolean isHLLC() {
-        return name.equals("hllc");
-    }
-    
-    public boolean isTopN() {
-        return name.equals("topn");
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public int getPrecision() {
-        return precision;
-    }
-
-    public int getScale() {
-        return scale;
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((name == null) ? 0 : name.hashCode());
-        result = prime * result + precision;
-        result = prime * result + scale;
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        DataType other = (DataType) obj;
-        if (name == null) {
-            if (other.name != null)
-                return false;
-        } else if (!name.equals(other.name))
-            return false;
-        if (precision != other.precision)
-            return false;
-        if (scale != other.scale)
-            return false;
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        if (precision < 0 && scale < 0)
-            return name;
-        else if (scale < 0)
-            return name + "(" + precision + ")";
-        else
-            return name + "(" + precision + "," + scale + ")";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
index b8cefa2..d22d0a3 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
@@ -21,6 +21,8 @@ package org.apache.kylin.metadata.model;
 import java.util.ArrayList;
 import java.util.Collection;
 
+import org.apache.kylin.common.datatype.DataType;
+
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -53,7 +55,7 @@ public class FunctionDesc {
 
     public void init(TableDesc factTable) {
         expression = expression.toUpperCase();
-        returnDataType = DataType.getInstance(returnType);
+        returnDataType = DataType.getType(returnType);
 
         for (ParameterDesc p = parameter; p != null; p = p.getNextParameter()) {
             p.setValue(p.getValue().toUpperCase());
@@ -69,6 +71,11 @@ public class FunctionDesc {
         }
 
         parameter.setColRefs(colRefs);
+        
+        // make sure sum/max/min returns the exact type as its input
+        if (isSum() || isMax() || isMin() && (colRefs.size() > 0)) {
+            setReturnType(colRefs.get(0).getDatatype());
+        }
     }
 
     public String getRewriteFieldName() {
@@ -183,7 +190,7 @@ public class FunctionDesc {
 
     public void setReturnType(String returnType) {
         this.returnType = returnType;
-        this.returnDataType = DataType.getInstance(returnType);
+        this.returnDataType = DataType.getType(returnType);
     }
 
     public TblColRef selectTblColRef(Collection<TblColRef> metricColumns, String factTableName) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
index ddb1e1a..de27145 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
@@ -18,10 +18,11 @@
 
 package org.apache.kylin.metadata.model;
 
-import org.apache.commons.lang.StringUtils;
-
 import java.io.Serializable;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.datatype.DataType;
+
 /**
  */
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java
index a4cdcbe..4a8c5d1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigestUtil.java
@@ -1,12 +1,12 @@
 package org.apache.kylin.metadata.realization;
 
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;
 import org.apache.kylin.metadata.filter.LogicalTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.google.common.base.Function;
@@ -52,9 +52,9 @@ public class SQLDigestUtil {
     //ts column type differentiate
     private static String formatTimeStr(DataType type, long ts) {
         String ret;
-        if (type == DataType.getInstance("date")) {
+        if (type == DataType.getType("date")) {
             ret = DateFormat.formatToDateStr(ts);
-        } else if (type == DataType.getInstance("long")) {
+        } else if (type == DataType.getType("long")) {
             ret = String.valueOf(ts);
         } else {
             throw new IllegalArgumentException("Illegal type for partition column " + type);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index da009df..4f011cf 100644
--- 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
@@ -24,11 +24,11 @@ import java.util.List;
 
 import net.sf.ehcache.pool.sizeof.annotations.IgnoreSizeOf;
 
+import org.apache.kylin.common.datatype.DoubleMutable;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.common.util.DoubleMutable;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index 5b2d20e..55235be 100644
--- 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
@@ -32,8 +32,8 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
 import org.apache.hadoop.mrunit.types.Pair;
 import org.apache.kylin.aggregation.MeasureCodec;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index 2acc751..5145efc 100644
--- 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
@@ -18,7 +18,7 @@
 
 package org.apache.kylin.engine.mr.steps;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
 import java.io.File;
 import java.io.IOException;
@@ -29,14 +29,19 @@ 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.datatype.DataType;
 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;
 import org.apache.kylin.cube.CubeUpdate;
-import org.apache.kylin.dict.*;
+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.IterableDictionaryValueEnumerator;
+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;
@@ -71,7 +76,7 @@ public class MergeCuboidMapperTest extends LocalFileMetadataTestCase {
         List<byte[]> values = new ArrayList<byte[]>();
         values.add(new byte[] { 101, 101, 101 });
         values.add(new byte[] { 102, 102, 102 });
-        Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(DataType.getInstance(newDictInfo.getDataType()), new IterableDictionaryValueEnumerator(values));
+        Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(DataType.getType(newDictInfo.getDataType()), new IterableDictionaryValueEnumerator(values));
         dictionaryManager.trySaveNewDict(dict, newDictInfo);
         ((TrieDictionary) dict).dump(System.out);
 
@@ -123,7 +128,7 @@ public class MergeCuboidMapperTest extends LocalFileMetadataTestCase {
                 values.add(new byte[] { 99, 99, 99 });
             else
                 values.add(new byte[] { 98, 98, 98 });
-            Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(DataType.getInstance(newDictInfo.getDataType()), new IterableDictionaryValueEnumerator(values));
+            Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueEnumerator(DataType.getType(newDictInfo.getDataType()), new IterableDictionaryValueEnumerator(values));
             dictionaryManager.trySaveNewDict(dict, newDictInfo);
             ((TrieDictionary) dict).dump(System.out);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
index 11b1897..aad1fa2 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/RawTableRecord.java
@@ -21,9 +21,9 @@ package org.apache.kylin.invertedindex.index;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
index 71d7bae..12fcf29 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
@@ -22,8 +22,8 @@ import java.util.Arrays;
 
 import org.apache.commons.lang.ObjectUtils;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.common.util.ShardingHash;
 import org.apache.kylin.dict.Dictionary;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
index 27519bc..d9e5b26 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfo.java
@@ -20,12 +20,12 @@ package org.apache.kylin.invertedindex.index;
 
 import java.util.List;
 
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
 import org.apache.kylin.invertedindex.model.IIDesc;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 /**
@@ -71,10 +71,10 @@ public class TableRecordInfo {
             isMetric[i] = desc.isMetricsCol(i);
             dataTypes[i] = tblColRef.getDatatype();
             if (isMetric[i]) {
-                lengths[i] = FixedLenMeasureCodec.get(DataType.getInstance(tblColRef.getColumnDesc().getDatatype())).getLength();
+                lengths[i] = FixedLenMeasureCodec.get(DataType.getType(tblColRef.getColumnDesc().getDatatype())).getLength();
             } else {
                 if (Array.isEmpty(dictionaryMap)) {
-                    final DataType dataType = DataType.getInstance(tblColRef.getColumnDesc().getDatatype());
+                    final DataType dataType = DataType.getType(tblColRef.getColumnDesc().getDatatype());
                     if (dataType.isNumberFamily()) {
                         lengths[i] = 16;
                     } else if (dataType.isStringFamily()) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
index bd27e38..afb3633 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecordInfoDigest.java
@@ -21,11 +21,11 @@ package org.apache.kylin.invertedindex.index;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
-import org.apache.kylin.metadata.model.DataType;
 
 import com.google.common.base.Objects;
 
@@ -55,7 +55,7 @@ public class TableRecordInfoDigest {
         this.measureCodecs = new FixedLenMeasureCodec[nColumns];
         for (int i = 0; i < isMetric.length; i++) {
             if (isMetric[i]) {
-                measureCodecs[i] = FixedLenMeasureCodec.get(DataType.getInstance(metricDataTypes[i]));
+                measureCodecs[i] = FixedLenMeasureCodec.get(DataType.getType(metricDataTypes[i]));
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
index de35f91..42e5185 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedHLLCodec.java
@@ -19,8 +19,8 @@ package org.apache.kylin.invertedindex.measure;
 
 import java.nio.ByteBuffer;
 
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
-import org.apache.kylin.metadata.model.DataType;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
index 35872be..4fa8714 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedLenMeasureCodec.java
@@ -18,10 +18,10 @@
 
 package org.apache.kylin.invertedindex.measure;
 
-import org.apache.kylin.metadata.model.DataType;
-
 import java.nio.ByteBuffer;
 
+import org.apache.kylin.common.datatype.DataType;
+
 abstract public class FixedLenMeasureCodec<T> {
 
     public static FixedLenMeasureCodec<?> get(DataType type) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
index a5d74f7..2b65192 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodec.java
@@ -20,9 +20,9 @@ package org.apache.kylin.invertedindex.measure;
 
 import java.nio.ByteBuffer;
 
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.LongMutable;
-import org.apache.kylin.metadata.model.DataType;
 
 public class FixedPointLongCodec extends FixedLenMeasureCodec<LongMutable> {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
index 4dd1723..0f29d95 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.Iterator;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.BytesUtil;
@@ -33,7 +34,6 @@ import org.apache.kylin.invertedindex.index.CompressedValueContainer;
 import org.apache.kylin.invertedindex.index.Slice;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
-import org.apache.kylin.metadata.model.DataType;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -111,11 +111,11 @@ public class IIKeyValueCodec implements KeyValueCodec {
         final boolean emptyDictionary = Array.isEmpty(dictionaries);
         for (int i = 0; i < nColumns; ++i) {
             if (isMetric[i]) {
-                final FixedLenMeasureCodec<?> fixedLenMeasureCodec = FixedLenMeasureCodec.get(DataType.getInstance(dataTypes[i]));
+                final FixedLenMeasureCodec<?> fixedLenMeasureCodec = FixedLenMeasureCodec.get(DataType.getType(dataTypes[i]));
                 lengths[i] = fixedLenMeasureCodec.getLength();
             } else {
                 if (emptyDictionary) {
-                    final DataType dataType = DataType.getInstance(dataTypes[i]);
+                    final DataType dataType = DataType.getType(dataTypes[i]);
                     if (dataType.isNumberFamily()) {
                         lengths[i] = 16;
                     } else if (dataType.isStringFamily()) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
index 27a4f71..600d034 100644
--- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
+++ b/invertedindex/src/test/java/org/apache/kylin/invertedindex/measure/FixedPointLongCodecTest.java
@@ -1,7 +1,6 @@
 package org.apache.kylin.invertedindex.measure;
 
-import org.apache.kylin.invertedindex.measure.FixedPointLongCodec;
-import org.apache.kylin.metadata.model.DataType;
+import org.apache.kylin.common.datatype.DataType;
 import org.junit.Test;
 
 /**
@@ -10,35 +9,35 @@ public class FixedPointLongCodecTest {
 
     @Test
     public void testEncode1() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
         long x = codec.getValueIgnoringDecimalPoint("12.12345");
         org.junit.Assert.assertEquals(1212345, x);
     }
 
     @Test
     public void testEncode2() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
         long x = codec.getValueIgnoringDecimalPoint("12.1234");
         org.junit.Assert.assertEquals(1212340, x);
     }
 
     @Test
     public void testEncode3() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
         long x = codec.getValueIgnoringDecimalPoint("12.123456");
         org.junit.Assert.assertEquals(1212345, x);
     }
 
     @Test
     public void testEncode4() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
         long x = codec.getValueIgnoringDecimalPoint("12");
         org.junit.Assert.assertEquals(1200000, x);
     }
 
     @Test
     public void testDecode1() {
-        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getType("decimal(18,5)"));
         String x = codec.restoreDecimalPoint(1212345);
         org.junit.Assert.assertEquals("12.12345", x);
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java
index 8813901..817a1e2 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeTupleConverter.java
@@ -12,7 +12,6 @@ import org.apache.kylin.common.topn.Counter;
 import org.apache.kylin.common.topn.TopNCounter;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -21,12 +20,9 @@ import org.apache.kylin.cube.kv.RowKeyDecoder;
 import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
 import org.apache.kylin.dict.Dictionary;
 import org.apache.kylin.dict.lookup.LookupStringTable;
-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 org.apache.kylin.metadata.tuple.ITuple;
-import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.hbase.steps.RowValueDecoder;
 import org.apache.kylin.storage.tuple.Tuple;
 import org.apache.kylin.storage.tuple.TupleInfo;



[05/13] incubator-kylin git commit: KYLIN-976 AggregationType interface, serializer and aggregator

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
index 8f0b17b..910ed31 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregators.java
@@ -23,15 +23,15 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.invertedindex.index.RawTableRecord;
 import org.apache.kylin.invertedindex.index.TableRecordInfo;
 import org.apache.kylin.invertedindex.index.TableRecordInfoDigest;
 import org.apache.kylin.invertedindex.measure.FixedLenMeasureCodec;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorConstants;
 
@@ -128,7 +128,7 @@ public class EndpointAggregators {
         this.metricValues = new Object[funcNames.length];
         this.measureSerializers = new FixedLenMeasureCodec[funcNames.length];
         for (int i = 0; i < this.measureSerializers.length; ++i) {
-            this.measureSerializers[i] = FixedLenMeasureCodec.get(DataType.getInstance(dataTypes[i]));
+            this.measureSerializers[i] = FixedLenMeasureCodec.get(DataType.getType(dataTypes[i]));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index e8b21d3..2717040 100644
--- 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
@@ -45,6 +45,7 @@ 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.datatype.DataType;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.ByteArray;
@@ -60,7 +61,6 @@ import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.CuboidShardUtil;
 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;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/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
index 54217ff..b247ad2 100644
--- 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
@@ -24,9 +24,9 @@ import java.util.Collection;
 
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.kylin.aggregation.MeasureCodec;
+import org.apache.kylin.common.datatype.DoubleMutable;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.DoubleMutable;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
index 08d4a49..873fb37 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.cube.kv.RowConstants;
 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/8f2a56cf/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
index 8e665f7..e2e5724 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/ii/coprocessor/endpoint/EndpointAggregationTest.java
@@ -21,8 +21,8 @@ package org.apache.kylin.storage.hbase.ii.coprocessor.endpoint;
 import com.google.common.collect.Lists;
 
 import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
index 4662406..276465e 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
@@ -26,8 +26,8 @@ import java.util.Arrays;
 
 import org.apache.hadoop.io.LongWritable;
 import org.apache.kylin.aggregation.MeasureCodec;
+import org.apache.kylin.common.datatype.LongMutable;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.common.util.LongMutable;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;


[07/13] incubator-kylin git commit: KYLIN-976 AggregationType interface, serializer and aggregator

Posted by li...@apache.org.
KYLIN-976 AggregationType interface, serializer and aggregator


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

Branch: refs/heads/KYLIN-976
Commit: 8f2a56cfb5f340cd19c67bf22c8dfbdf2f9458a1
Parents: 1218bbd
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Nov 25 11:13:18 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Fri Nov 27 14:47:09 2015 +0800

----------------------------------------------------------------------
 .../kylin/job/dataGen/FactTableGenerator.java   |   2 +-
 .../streaming/StreamingTableDataGenerator.java  |   2 +-
 .../common/datatype/BigDecimalSerializer.java   | 109 +++++++
 .../apache/kylin/common/datatype/DataType.java  | 289 ++++++++++++++++++
 .../common/datatype/DataTypeSerializer.java     | 108 +++++++
 .../common/datatype/DateTimeSerializer.java     |  62 ++++
 .../kylin/common/datatype/DoubleMutable.java    |  68 +++++
 .../kylin/common/datatype/DoubleSerializer.java |  81 +++++
 .../kylin/common/datatype/LongMutable.java      |  70 +++++
 .../kylin/common/datatype/LongSerializer.java   |  88 ++++++
 .../kylin/common/datatype/StringSerializer.java |  54 ++++
 .../kylin/common/debug/BackdoorToggles.java     |   5 -
 .../kylin/common/topn/DoublyLinkedList.java     |   2 -
 .../apache/kylin/common/util/DoubleMutable.java |  68 -----
 .../apache/kylin/common/util/LongMutable.java   |  70 -----
 .../datatype/BigDecimalSerializerTest.java      |  51 ++++
 .../apache/kylin/aggregation/Aggregation.java   |  42 ---
 .../kylin/aggregation/AggregationType.java      |  97 ++++++
 .../kylin/aggregation/DataTypeSerializer.java   | 118 --------
 .../kylin/aggregation/IAggregationFactory.java  |   6 +
 .../kylin/aggregation/MeasureAggregator.java    |  61 +---
 .../apache/kylin/aggregation/MeasureCodec.java  |   1 +
 .../aggregation/basic/BasicAggregation.java     | 128 ++++++++
 .../basic/BasicAggregationFactory.java          |  31 ++
 .../aggregation/basic/BigDecimalSerializer.java | 111 -------
 .../aggregation/basic/DateTimeSerializer.java   |  65 ----
 .../aggregation/basic/DoubleMaxAggregator.java  |   2 +-
 .../aggregation/basic/DoubleMinAggregator.java  |   2 +-
 .../aggregation/basic/DoubleSerializer.java     |  84 ------
 .../aggregation/basic/DoubleSumAggregator.java  |   2 +-
 .../aggregation/basic/LongMaxAggregator.java    |   2 +-
 .../aggregation/basic/LongMinAggregator.java    |   2 +-
 .../kylin/aggregation/basic/LongSerializer.java |  91 ------
 .../aggregation/basic/LongSumAggregator.java    |   2 +-
 .../aggregation/basic/StringSerializer.java     |  56 ----
 .../kylin/aggregation/hllc/HLLCAggregation.java |  78 +++++
 .../hllc/HLLCAggregationFactory.java            |  35 +++
 .../kylin/aggregation/hllc/HLLCSerializer.java  |   4 +-
 .../kylin/aggregation/hllc/LDCAggregator.java   |   2 +-
 .../kylin/aggregation/topn/TopNAggregation.java |  76 +++++
 .../topn/TopNAggregationFactory.java            |  35 +++
 .../aggregation/topn/TopNCounterSerializer.java |  12 +-
 .../kylin/cube/gridtable/CubeCodeSystem.java    |   4 +-
 .../kylin/cube/gridtable/CubeGridTable.java     |   1 -
 .../gridtable/CuboidToGridTableMapping.java     |   2 +-
 .../cube/gridtable/TrimmedCubeCodeSystem.java   |   3 +-
 .../cube/inmemcubing/InMemCubeBuilder.java      |   2 +-
 .../InMemCubeBuilderInputConverter.java         |   2 +-
 .../apache/kylin/cube/kv/RowKeyColumnOrder.java |   2 +-
 .../model/validation/rule/FunctionRule.java     |   2 +-
 .../java/org/apache/kylin/gridtable/GTInfo.java |   2 +-
 .../kylin/gridtable/GTSampleCodeSystem.java     |   2 +-
 .../apache/kylin/gridtable/UnitTestSupport.java |  14 +-
 .../basic/BigDecimalSerializerTest.java         |  53 ----
 .../topn/TopNCounterSerializerTest.java         |   9 +-
 .../gridtable/AggregationCacheMemSizeTest.java  |   4 +-
 .../kylin/gridtable/DictGridTableTest.java      |  16 +-
 .../kylin/gridtable/SimpleGridTableTest.java    |   2 +-
 .../gridtable/SimpleInvertedIndexTest.java      |  13 +-
 .../metadata/measure/MeasureCodecTest.java      |   4 +-
 .../apache/kylin/dict/DictionaryGenerator.java  |   9 +-
 .../apache/kylin/dict/DictionaryManager.java    |   4 +-
 .../apache/kylin/dict/NumberDictionaryTest.java |   9 +-
 .../filter/TimeConditionLiteralsReplacer.java   |   2 +-
 .../apache/kylin/metadata/model/ColumnDesc.java |   9 +-
 .../apache/kylin/metadata/model/DataType.java   | 295 -------------------
 .../kylin/metadata/model/FunctionDesc.java      |  11 +-
 .../apache/kylin/metadata/model/TblColRef.java  |   5 +-
 .../metadata/realization/SQLDigestUtil.java     |   6 +-
 .../org/apache/kylin/storage/tuple/Tuple.java   |   4 +-
 .../kylin/engine/mr/steps/CubeReducerTest.java  |   2 +-
 .../engine/mr/steps/MergeCuboidMapperTest.java  |  15 +-
 .../invertedindex/index/RawTableRecord.java     |   2 +-
 .../kylin/invertedindex/index/TableRecord.java  |   2 +-
 .../invertedindex/index/TableRecordInfo.java    |   6 +-
 .../index/TableRecordInfoDigest.java            |   6 +-
 .../invertedindex/measure/FixedHLLCodec.java    |   2 +-
 .../measure/FixedLenMeasureCodec.java           |   4 +-
 .../measure/FixedPointLongCodec.java            |   4 +-
 .../invertedindex/model/IIKeyValueCodec.java    |   6 +-
 .../measure/FixedPointLongCodecTest.java        |  13 +-
 .../hbase/cube/v1/CubeTupleConverter.java       |   4 -
 .../endpoint/EndpointAggregators.java           |   6 +-
 .../storage/hbase/steps/CreateHTableJob.java    |   2 +-
 .../storage/hbase/steps/RowValueDecoder.java    |   4 +-
 .../observer/AggregateRegionObserverTest.java   |   2 +-
 .../endpoint/EndpointAggregationTest.java       |   2 +-
 .../hbase/steps/RowValueDecoderTest.java        |   2 +-
 88 files changed, 1601 insertions(+), 1248 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java b/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
index 5a0fee7..705c175 100644
--- a/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
+++ b/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
@@ -37,6 +37,7 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.Array;
 import org.apache.kylin.cube.CubeInstance;
@@ -45,7 +46,6 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.DimensionDesc;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java b/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java
index 075a048..04a5b5b 100644
--- a/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java
+++ b/assembly/src/test/java/org/apache/kylin/job/streaming/StreamingTableDataGenerator.java
@@ -5,9 +5,9 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.datatype.DataType;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/datatype/BigDecimalSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/BigDecimalSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/BigDecimalSerializer.java
new file mode 100644
index 0000000..f19ce14
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/datatype/BigDecimalSerializer.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.common.datatype;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class BigDecimalSerializer extends DataTypeSerializer<BigDecimal> {
+
+    private static final Logger logger = LoggerFactory.getLogger(BigDecimalSerializer.class);
+
+    final DataType type;
+    final int maxLength;
+
+    int avoidVerbose = 0;
+
+    public BigDecimalSerializer(DataType type) {
+        this.type = type;
+        // see serialize(): 1 byte scale, 1 byte length, assume every 2 digits takes 1 byte
+        this.maxLength = 1 + 1 + (type.getPrecision() + 1) / 2;
+    }
+
+    @Override
+    public void serialize(BigDecimal value, ByteBuffer out) {
+        if (value.scale() > type.getScale()) {
+            if (avoidVerbose % 10000 == 0) {
+                logger.warn("value's scale has exceeded the " + type.getScale() + ", cut it off, to ensure encoded value do not exceed maxLength " + maxLength + " times:" + (avoidVerbose++));
+            }
+            value = value.setScale(type.getScale(), BigDecimal.ROUND_HALF_EVEN);
+        }
+        byte[] bytes = value.unscaledValue().toByteArray();
+        if (bytes.length + 2 > maxLength) {
+            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
+        }
+
+        BytesUtil.writeVInt(value.scale(), out);
+        BytesUtil.writeVInt(bytes.length, out);
+        out.put(bytes);
+    }
+
+    @Override
+    public BigDecimal deserialize(ByteBuffer in) {
+        int scale = BytesUtil.readVInt(in);
+        int n = BytesUtil.readVInt(in);
+
+        byte[] bytes = new byte[n];
+        in.get(bytes);
+
+        return new BigDecimal(new BigInteger(bytes), scale);
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+
+        @SuppressWarnings("unused")
+        int scale = BytesUtil.readVInt(in);
+        int n = BytesUtil.readVInt(in);
+        int len = in.position() - mark + n;
+
+        in.position(mark);
+        return len;
+    }
+
+    @Override
+    public int maxLength() {
+        return maxLength;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+    @Override
+    public BigDecimal valueOf(byte[] value) {
+        if (value == null)
+            return new BigDecimal(0);
+        else
+            return new BigDecimal(Bytes.toString(value));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/datatype/DataType.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DataType.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DataType.java
new file mode 100644
index 0000000..36fad00
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/datatype/DataType.java
@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.common.datatype;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ */
+@SuppressWarnings("serial")
+public class DataType implements Serializable {
+
+    // standard sql types, ref: http://www.w3schools.com/sql/sql_datatypes_general.asp
+    public static final String VALID_TYPES_STRING = "any|char|varchar|boolean|binary" //
+            + "|integer|tinyint|smallint|bigint|decimal|numeric|float|real|double" //
+            + "|date|time|datetime|timestamp|byte|int|short|long|string";
+
+    private static final String TYPE_PATTEN_TAIL = "\\s*" //
+            + "(?:" + "[(]" + "([\\d\\s,]+)" + "[)]" + ")?";
+
+    private static final Pattern TYPE_PATTERN = Pattern.compile( //
+            "(" + VALID_TYPES_STRING + ")" + TYPE_PATTEN_TAIL, Pattern.CASE_INSENSITIVE);
+
+    private static final Pattern CUSTOM_TYPE_PATTERN = Pattern.compile( //
+            "(" + ".*?" + ")" + TYPE_PATTEN_TAIL, Pattern.CASE_INSENSITIVE);
+    
+    public static final Set<String> INTEGER_FAMILY = new HashSet<String>();
+    public static final Set<String> NUMBER_FAMILY = new HashSet<String>();
+    public static final Set<String> DATETIME_FAMILY = new HashSet<String>();
+    public static final Set<String> STRING_FAMILY = new HashSet<String>();
+    private static final Map<String, String> LEGACY_TYPE_MAP = new HashMap<String, String>();
+    static {
+        INTEGER_FAMILY.add("tinyint");
+        INTEGER_FAMILY.add("smallint");
+        INTEGER_FAMILY.add("integer");
+        INTEGER_FAMILY.add("bigint");
+
+        NUMBER_FAMILY.addAll(INTEGER_FAMILY);
+        NUMBER_FAMILY.add("float");
+        NUMBER_FAMILY.add("double");
+        NUMBER_FAMILY.add("decimal");
+        NUMBER_FAMILY.add("real");
+        NUMBER_FAMILY.add("numeric");
+
+        DATETIME_FAMILY.add("date");
+        DATETIME_FAMILY.add("time");
+        DATETIME_FAMILY.add("datetime");
+        DATETIME_FAMILY.add("timestamp");
+
+        STRING_FAMILY.add("varchar");
+        STRING_FAMILY.add("char");
+
+        LEGACY_TYPE_MAP.put("byte", "tinyint");
+        LEGACY_TYPE_MAP.put("int", "integer");
+        LEGACY_TYPE_MAP.put("short", "smallint");
+        LEGACY_TYPE_MAP.put("long", "bigint");
+        LEGACY_TYPE_MAP.put("string", "varchar");
+        LEGACY_TYPE_MAP.put("hllc10", "hllc(10)");
+        LEGACY_TYPE_MAP.put("hllc12", "hllc(12)");
+        LEGACY_TYPE_MAP.put("hllc14", "hllc(14)");
+        LEGACY_TYPE_MAP.put("hllc15", "hllc(15)");
+        LEGACY_TYPE_MAP.put("hllc16", "hllc(16)");
+    }
+
+    private static final ConcurrentMap<DataType, DataType> CACHE = new ConcurrentHashMap<DataType, DataType>();
+
+    public static final DataType ANY = DataType.getType("any");
+
+    public static DataType getType(String type) {
+        return getTypeInner(type, false);
+    }
+    
+    public static DataType getCustomType(String type) {
+        return getTypeInner(type, true);
+    }
+    
+    private static DataType getTypeInner(String type, boolean isCustom) {
+        if (type == null)
+            return null;
+
+        DataType dataType = new DataType(type, isCustom);
+        DataType cached = CACHE.get(dataType);
+        if (cached == null) {
+            CACHE.put(dataType, dataType);
+            cached = dataType;
+        }
+        return cached;
+    }
+
+    // ============================================================================
+
+    private String name;
+    private int precision;
+    private int scale;
+
+    DataType(String datatype, boolean isCustom) {
+        datatype = datatype.trim().toLowerCase();
+        datatype = replaceLegacy(datatype);
+
+        Pattern pattern = isCustom ? CUSTOM_TYPE_PATTERN : TYPE_PATTERN;
+        Matcher m = pattern.matcher(datatype);
+        if (m.matches() == false)
+            throw new IllegalArgumentException("bad data type -- " + datatype + ", does not match " + pattern);
+
+        name = replaceLegacy(m.group(1));
+        precision = -1;
+        scale = -1;
+
+        String leftover = m.group(2);
+        if (leftover != null) {
+            String[] parts = leftover.split("\\s*,\\s*");
+            for (int i = 0; i < parts.length; i++) {
+                int n;
+                try {
+                    n = Integer.parseInt(parts[i]);
+                } catch (NumberFormatException e) {
+                    throw new IllegalArgumentException("bad data type -- " + datatype + ", precision/scale not numeric");
+                }
+                if (i == 0)
+                    precision = n;
+                else if (i == 1)
+                    scale = n;
+                else
+                    throw new IllegalArgumentException("bad data type -- " + datatype + ", too many precision/scale parts");
+            }
+        }
+
+        // FIXME 256 for unknown string precision
+        if ((name.equals("char") || name.equals("varchar")) && precision == -1) {
+            precision = 256; // to save memory at frontend, e.g. tableau will
+                             // allocate memory according to this
+        }
+
+        // FIXME (19,4) for unknown decimal precision
+        if ((name.equals("decimal") || name.equals("numeric")) && precision == -1) {
+            precision = 19;
+            scale = 4;
+        }
+    }
+
+    private String replaceLegacy(String str) {
+        String replace = LEGACY_TYPE_MAP.get(str);
+        return replace == null ? str : replace;
+    }
+
+    public int getStorageBytesEstimate() {
+        return DataTypeSerializer.create(this).getStorageBytesEstimate();
+    }
+
+    public boolean isStringFamily() {
+        return STRING_FAMILY.contains(name);
+    }
+
+    public boolean isIntegerFamily() {
+        return INTEGER_FAMILY.contains(name);
+    }
+
+    public boolean isNumberFamily() {
+        return NUMBER_FAMILY.contains(name);
+    }
+
+    public boolean isDateTimeFamily() {
+        return DATETIME_FAMILY.contains(name);
+    }
+
+    public boolean isDate() {
+        return name.equals("date");
+    }
+
+    public boolean isTime() {
+        return name.equals("time");
+    }
+
+    public boolean isTimestamp() {
+        return name.equals("timestamp");
+    }
+
+    public boolean isDatetime() {
+        return name.equals("datetime");
+    }
+
+    public boolean isTinyInt() {
+        return name.equals("tinyint");
+    }
+
+    public boolean isSmallInt() {
+        return name.equals("smallint");
+    }
+
+    public boolean isInt() {
+        return name.equals("integer");
+    }
+
+    public boolean isBigInt() {
+        return name.equals("bigint");
+    }
+
+    public boolean isFloat() {
+        return name.equals("float");
+    }
+
+    public boolean isDouble() {
+        return name.equals("double");
+    }
+
+    public boolean isDecimal() {
+        return name.equals("decimal");
+    }
+
+    public boolean isHLLC() {
+        return name.equals("hllc");
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public int getPrecision() {
+        return precision;
+    }
+
+    public int getScale() {
+        return scale;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        result = prime * result + precision;
+        result = prime * result + scale;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        DataType other = (DataType) obj;
+        if (name == null) {
+            if (other.name != null)
+                return false;
+        } else if (!name.equals(other.name))
+            return false;
+        if (precision != other.precision)
+            return false;
+        if (scale != other.scale)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        if (precision < 0 && scale < 0)
+            return name;
+        else if (scale < 0)
+            return name + "(" + precision + ")";
+        else
+            return name + "(" + precision + "," + scale + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/datatype/DataTypeSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DataTypeSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DataTypeSerializer.java
new file mode 100644
index 0000000..99fa3fd
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/datatype/DataTypeSerializer.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.common.datatype;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kylin.common.util.BytesSerializer;
+
+import com.google.common.collect.Maps;
+
+/**
+ * @author yangli9
+ * 
+ * Note: the implementations MUST be thread-safe.
+ */
+abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
+
+    final static Map<String, Class<?>> implementations;
+    static {
+        HashMap<String, Class<?>> impl = Maps.newHashMap();
+        impl.put("varchar", StringSerializer.class);
+        impl.put("decimal", BigDecimalSerializer.class);
+        impl.put("double", DoubleSerializer.class);
+        impl.put("float", DoubleSerializer.class);
+        impl.put("bigint", LongSerializer.class);
+        impl.put("long", LongSerializer.class);
+        impl.put("integer", LongSerializer.class);
+        impl.put("int", LongSerializer.class);
+        impl.put("smallint", LongSerializer.class);
+        impl.put("date", DateTimeSerializer.class);
+        impl.put("datetime", DateTimeSerializer.class);
+        impl.put("timestamp", DateTimeSerializer.class);
+        implementations = Collections.unmodifiableMap(impl);
+    }
+    
+    public static boolean hasRegistered(String dataTypeName) {
+        return implementations.containsKey(dataTypeName);
+    }
+    
+    public static void register(String dataTypeName, Class<? extends DataTypeSerializer<?>> impl) {
+        implementations.put(dataTypeName, impl);
+    }
+
+    public static DataTypeSerializer<?> create(String dataType) {
+        return create(DataType.getType(dataType));
+    }
+
+    public static DataTypeSerializer<?> create(DataType type) {
+        Class<?> clz = implementations.get(type.getName());
+        if (clz == null)
+            throw new RuntimeException("No DataTypeSerializer for type " + type);
+
+        try {
+            return (DataTypeSerializer<?>) clz.getConstructor(DataType.class).newInstance(type);
+        } catch (Exception e) {
+            throw new RuntimeException(e); // never happen
+        }
+    }
+    
+    /** peek into buffer and return the length of serialization */
+    abstract public int peekLength(ByteBuffer in);
+
+    /** return the max number of bytes to the longest serialization */
+    abstract public int maxLength();
+
+    /** get an estimate of size in bytes of the serialized data */
+    abstract public int getStorageBytesEstimate();
+
+    /** convert from String to obj (string often come as byte[] in mapred) */
+    abstract public T valueOf(byte[] value);
+
+    /** convert from String to obj */
+    public T valueOf(String value) {
+        try {
+            return valueOf(value.getBytes("UTF-8"));
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e); // never happen
+        }
+    }
+
+    /** convert from obj to string */
+    public String toString(T value) {
+        if (value == null)
+            return "NULL";
+        else
+            return value.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/datatype/DateTimeSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DateTimeSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DateTimeSerializer.java
new file mode 100644
index 0000000..d3ef7cd
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/datatype/DateTimeSerializer.java
@@ -0,0 +1,62 @@
+package org.apache.kylin.common.datatype;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.DateFormat;
+
+public class DateTimeSerializer extends DataTypeSerializer<LongMutable> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+
+    public DateTimeSerializer(DataType type) {
+    }
+
+    @Override
+    public void serialize(LongMutable value, ByteBuffer out) {
+        out.putLong(value.get());
+    }
+
+    private LongMutable current() {
+        LongMutable l = current.get();
+        if (l == null) {
+            l = new LongMutable();
+            current.set(l);
+        }
+        return l;
+    }
+
+    @Override
+    public LongMutable deserialize(ByteBuffer in) {
+        LongMutable l = current();
+        l.set(in.getLong());
+        return l;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return 8;
+    }
+
+    @Override
+    public int maxLength() {
+        return 8;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+    @Override
+    public LongMutable valueOf(byte[] value) {
+        LongMutable l = current();
+        if (value == null)
+            l.set(0L);
+        else
+            l.set(DateFormat.stringToMillis(Bytes.toString(value)));
+        return l;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleMutable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleMutable.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleMutable.java
new file mode 100644
index 0000000..a32fbd8
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleMutable.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.common.datatype;
+
+import java.io.Serializable;
+
+@SuppressWarnings("serial")
+public class DoubleMutable implements Comparable<DoubleMutable>, Serializable {
+
+    private double v;
+
+    public DoubleMutable() {
+        this(0);
+    }
+
+    public DoubleMutable(double v) {
+        set(v);
+    }
+
+    public double get() {
+        return v;
+    }
+
+    public void set(double v) {
+        this.v = v;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof DoubleMutable)) {
+            return false;
+        }
+        DoubleMutable other = (DoubleMutable) o;
+        return this.v == other.v;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) Double.doubleToLongBits(v);
+    }
+
+    @Override
+    public int compareTo(DoubleMutable o) {
+        return (v < o.v ? -1 : (v == o.v ? 0 : 1));
+    }
+
+    @Override
+    public String toString() {
+        return Double.toString(v);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleSerializer.java
new file mode 100644
index 0000000..f128576
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/datatype/DoubleSerializer.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.common.datatype;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+
+/**
+ */
+public class DoubleSerializer extends DataTypeSerializer<DoubleMutable> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<DoubleMutable> current = new ThreadLocal<DoubleMutable>();
+
+    public DoubleSerializer(DataType type) {
+    }
+
+    @Override
+    public void serialize(DoubleMutable value, ByteBuffer out) {
+        out.putDouble(value.get());
+    }
+
+    private DoubleMutable current() {
+        DoubleMutable d = current.get();
+        if (d == null) {
+            d = new DoubleMutable();
+            current.set(d);
+        }
+        return d;
+    }
+
+    @Override
+    public DoubleMutable deserialize(ByteBuffer in) {
+        DoubleMutable d = current();
+        d.set(in.getDouble());
+        return d;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return 8;
+    }
+
+    @Override
+    public int maxLength() {
+        return 8;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+    @Override
+    public DoubleMutable valueOf(byte[] value) {
+        DoubleMutable d = current();
+        if (value == null)
+            d.set(0d);
+        else
+            d.set(Double.parseDouble(Bytes.toString(value)));
+        return d;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/datatype/LongMutable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/LongMutable.java b/core-common/src/main/java/org/apache/kylin/common/datatype/LongMutable.java
new file mode 100644
index 0000000..72b540c
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/datatype/LongMutable.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.common.datatype;
+
+import java.io.Serializable;
+
+@SuppressWarnings("serial")
+public class LongMutable implements Comparable<LongMutable>, Serializable {
+
+    private long v;
+
+    public LongMutable() {
+        this(0);
+    }
+
+    public LongMutable(long v) {
+        set(v);
+    }
+
+    public long get() {
+        return v;
+    }
+
+    public void set(long v) {
+        this.v = v;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof LongMutable)) {
+            return false;
+        }
+        LongMutable other = (LongMutable) o;
+        return this.v == other.v;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) v;
+    }
+
+    @Override
+    public int compareTo(LongMutable o) {
+        long thisValue = this.v;
+        long thatValue = o.v;
+        return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
+    }
+
+    @Override
+    public String toString() {
+        return Long.toString(v);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/datatype/LongSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/LongSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/LongSerializer.java
new file mode 100644
index 0000000..5c5e10f
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/datatype/LongSerializer.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.common.datatype;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+
+/**
+ */
+public class LongSerializer extends DataTypeSerializer<LongMutable> {
+
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+
+    public LongSerializer(DataType type) {
+    }
+
+    @Override
+    public void serialize(LongMutable value, ByteBuffer out) {
+        BytesUtil.writeVLong(value.get(), out);
+    }
+
+    private LongMutable current() {
+        LongMutable l = current.get();
+        if (l == null) {
+            l = new LongMutable();
+            current.set(l);
+        }
+        return l;
+    }
+
+    @Override
+    public LongMutable deserialize(ByteBuffer in) {
+        LongMutable l = current();
+        l.set(BytesUtil.readVLong(in));
+        return l;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+
+        BytesUtil.readVLong(in);
+        int len = in.position() - mark;
+
+        in.position(mark);
+        return len;
+    }
+
+    @Override
+    public int maxLength() {
+        return 9; // vlong: 1 + 8
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 5;
+    }
+
+    @Override
+    public LongMutable valueOf(byte[] value) {
+        LongMutable l = current();
+        if (value == null)
+            l.set(0L);
+        else
+            l.set(Long.parseLong(Bytes.toString(value)));
+        return l;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/datatype/StringSerializer.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/datatype/StringSerializer.java b/core-common/src/main/java/org/apache/kylin/common/datatype/StringSerializer.java
new file mode 100644
index 0000000..b3562eb
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/datatype/StringSerializer.java
@@ -0,0 +1,54 @@
+package org.apache.kylin.common.datatype;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+
+public class StringSerializer extends DataTypeSerializer<String> {
+
+    final DataType type;
+    final int maxLength;
+
+    public StringSerializer(DataType type) {
+        this.type = type;
+        // see serialize(): 2 byte length, rest is String.toBytes()
+        this.maxLength = 2 + type.getPrecision();
+    }
+
+    @Override
+    public void serialize(String value, ByteBuffer out) {
+        int start = out.position();
+
+        BytesUtil.writeUTFString(value, out);
+
+        if (out.position() - start > maxLength)
+            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
+    }
+
+    @Override
+    public String deserialize(ByteBuffer in) {
+        return BytesUtil.readUTFString(in);
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return BytesUtil.peekByteArrayLength(in);
+    }
+
+    @Override
+    public int maxLength() {
+        return maxLength;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return maxLength;
+    }
+
+    @Override
+    public String valueOf(byte[] value) {
+        return Bytes.toString(value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java b/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java
index 7304b8b..f3745d7 100644
--- a/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java
+++ b/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java
@@ -20,15 +20,10 @@ package org.apache.kylin.common.debug;
 
 import java.util.Map;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  */
 public class BackdoorToggles {
 
-    private static final Logger logger = LoggerFactory.getLogger(BackdoorToggles.class);
-
     private static final ThreadLocal<Map<String, String>> _backdoorToggles = new ThreadLocal<Map<String, String>>();
 
     public static void setToggles(Map<String, String> toggles) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/topn/DoublyLinkedList.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/topn/DoublyLinkedList.java b/core-common/src/main/java/org/apache/kylin/common/topn/DoublyLinkedList.java
index 1520ce1..4a3d6e8 100644
--- a/core-common/src/main/java/org/apache/kylin/common/topn/DoublyLinkedList.java
+++ b/core-common/src/main/java/org/apache/kylin/common/topn/DoublyLinkedList.java
@@ -18,8 +18,6 @@
 
 package org.apache.kylin.common.topn;
 
-import java.util.ConcurrentModificationException;
-import java.util.Iterator;
 
 /**
  * Modified from DoublyLinkedList.java in https://github.com/addthis/stream-lib

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/util/DoubleMutable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/DoubleMutable.java b/core-common/src/main/java/org/apache/kylin/common/util/DoubleMutable.java
deleted file mode 100644
index 520cd74..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/util/DoubleMutable.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.common.util;
-
-import java.io.Serializable;
-
-@SuppressWarnings("serial")
-public class DoubleMutable implements Comparable<DoubleMutable>, Serializable {
-
-    private double v;
-
-    public DoubleMutable() {
-        this(0);
-    }
-
-    public DoubleMutable(double v) {
-        set(v);
-    }
-
-    public double get() {
-        return v;
-    }
-
-    public void set(double v) {
-        this.v = v;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof DoubleMutable)) {
-            return false;
-        }
-        DoubleMutable other = (DoubleMutable) o;
-        return this.v == other.v;
-    }
-
-    @Override
-    public int hashCode() {
-        return (int) Double.doubleToLongBits(v);
-    }
-
-    @Override
-    public int compareTo(DoubleMutable o) {
-        return (v < o.v ? -1 : (v == o.v ? 0 : 1));
-    }
-
-    @Override
-    public String toString() {
-        return Double.toString(v);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/main/java/org/apache/kylin/common/util/LongMutable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/LongMutable.java b/core-common/src/main/java/org/apache/kylin/common/util/LongMutable.java
deleted file mode 100644
index 238bb86..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/util/LongMutable.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.common.util;
-
-import java.io.Serializable;
-
-@SuppressWarnings("serial")
-public class LongMutable implements Comparable<LongMutable>, Serializable {
-
-    private long v;
-
-    public LongMutable() {
-        this(0);
-    }
-
-    public LongMutable(long v) {
-        set(v);
-    }
-
-    public long get() {
-        return v;
-    }
-
-    public void set(long v) {
-        this.v = v;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof LongMutable)) {
-            return false;
-        }
-        LongMutable other = (LongMutable) o;
-        return this.v == other.v;
-    }
-
-    @Override
-    public int hashCode() {
-        return (int) v;
-    }
-
-    @Override
-    public int compareTo(LongMutable o) {
-        long thisValue = this.v;
-        long thatValue = o.v;
-        return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
-    }
-
-    @Override
-    public String toString() {
-        return Long.toString(v);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-common/src/test/java/org/apache/kylin/common/datatype/BigDecimalSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/datatype/BigDecimalSerializerTest.java b/core-common/src/test/java/org/apache/kylin/common/datatype/BigDecimalSerializerTest.java
new file mode 100644
index 0000000..cba6795
--- /dev/null
+++ b/core-common/src/test/java/org/apache/kylin/common/datatype/BigDecimalSerializerTest.java
@@ -0,0 +1,51 @@
+package org.apache.kylin.common.datatype;
+
+import static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ */
+public class BigDecimalSerializerTest {
+
+    private static BigDecimalSerializer bigDecimalSerializer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        bigDecimalSerializer = new BigDecimalSerializer(DataType.getType("decimal"));
+    }
+
+    @Test
+    public void testNormal() {
+        BigDecimal input = new BigDecimal("1234.1234");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        buffer.mark();
+        bigDecimalSerializer.serialize(input, buffer);
+        buffer.reset();
+        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
+        assertEquals(input, output);
+    }
+
+    @Test
+    public void testScaleOutOfRange() {
+        BigDecimal input = new BigDecimal("1234.1234567890");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        buffer.mark();
+        bigDecimalSerializer.serialize(input, buffer);
+        buffer.reset();
+        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
+        assertEquals(input.setScale(bigDecimalSerializer.type.getScale(), BigDecimal.ROUND_HALF_EVEN), output);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testOutOfPrecision() {
+        BigDecimal input = new BigDecimal("66855344214907231736.4924");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        bigDecimalSerializer.serialize(input, buffer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/Aggregation.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/Aggregation.java b/core-cube/src/main/java/org/apache/kylin/aggregation/Aggregation.java
deleted file mode 100644
index 193c5de..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/Aggregation.java
+++ /dev/null
@@ -1,42 +0,0 @@
-package org.apache.kylin.aggregation;
-
-import java.util.List;
-
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-
-abstract public class Aggregation {
-    
-    /* ============================================================================
-     * Define
-     * ---------------------------------------------------------------------------- */
-    
-    abstract public DataType getAggregationDataType();
-    
-    abstract public DataType getResultDataType();
-    
-    abstract public void validate(MeasureDesc measureDesc) throws IllegalArgumentException;
-    
-    /* ============================================================================
-     * Build
-     * ---------------------------------------------------------------------------- */
-    
-    abstract public DataTypeSerializer<?> getSeralizer();
-    
-    abstract public MeasureAggregator<?> getAggregator();
- 
-    abstract public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc);
-    
-    abstract public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts);
-
-    /* ============================================================================
-     * Cube Selection
-     * ---------------------------------------------------------------------------- */
-    
-    /* ============================================================================
-     * Query
-     * ---------------------------------------------------------------------------- */
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/AggregationType.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/AggregationType.java b/core-cube/src/main/java/org/apache/kylin/aggregation/AggregationType.java
new file mode 100644
index 0000000..81345b7
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/AggregationType.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.aggregation;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.aggregation.basic.BasicAggregationFactory;
+import org.apache.kylin.aggregation.hllc.HLLCAggregationFactory;
+import org.apache.kylin.aggregation.topn.TopNAggregationFactory;
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.DataTypeSerializer;
+import org.apache.kylin.dict.Dictionary;
+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.Maps;
+
+abstract public class AggregationType {
+    
+    private static final Map<String, IAggregationFactory> factoryRegistry = Maps.newConcurrentMap();
+    private static final IAggregationFactory defaultFactory = new BasicAggregationFactory();
+    
+    static {
+        factoryRegistry.put(FunctionDesc.FUNC_COUNT_DISTINCT, new HLLCAggregationFactory());
+        factoryRegistry.put(FunctionDesc.FUNC_TOP_N, new TopNAggregationFactory());
+    }
+    
+    public static AggregationType create(String funcName, String dataType) {
+        funcName = funcName.toUpperCase();
+        dataType = dataType.toLowerCase();
+        
+        IAggregationFactory factory = factoryRegistry.get(funcName);
+        if (factory == null)
+            factory = defaultFactory;
+        
+        AggregationType result = factory.createAggregationType(funcName, dataType);
+        
+        // register serializer for aggr data type
+        DataType aggregationDataType = result.getAggregationDataType();
+        if (DataTypeSerializer.hasRegistered(aggregationDataType.getName()) == false) {
+            DataTypeSerializer.register(aggregationDataType.getName(), result.getAggregationDataSeralizer());
+        }
+        
+        return result;
+    }
+    
+    /* ============================================================================
+     * Define
+     * ---------------------------------------------------------------------------- */
+    
+    abstract public DataType getAggregationDataType();
+    
+    abstract public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer();
+    
+    abstract public void validate(MeasureDesc measureDesc) throws IllegalArgumentException;
+    
+    /* ============================================================================
+     * Build
+     * ---------------------------------------------------------------------------- */
+    
+    abstract public MeasureAggregator<?> newAggregator();
+ 
+    abstract public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc);
+    
+    abstract public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts);
+
+    /* ============================================================================
+     * Cube Selection
+     * ---------------------------------------------------------------------------- */
+    
+    /* ============================================================================
+     * Query
+     * ---------------------------------------------------------------------------- */
+    
+    /* ============================================================================
+     * Storage
+     * ---------------------------------------------------------------------------- */
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/DataTypeSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/DataTypeSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/DataTypeSerializer.java
deleted file mode 100644
index df6833c..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/DataTypeSerializer.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.aggregation;
-
-import java.io.UnsupportedEncodingException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.kylin.aggregation.basic.BigDecimalSerializer;
-import org.apache.kylin.aggregation.basic.DateTimeSerializer;
-import org.apache.kylin.aggregation.basic.DoubleSerializer;
-import org.apache.kylin.aggregation.basic.LongSerializer;
-import org.apache.kylin.aggregation.basic.StringSerializer;
-import org.apache.kylin.aggregation.hllc.HLLCSerializer;
-import org.apache.kylin.aggregation.topn.TopNCounterSerializer;
-import org.apache.kylin.common.util.BytesSerializer;
-import org.apache.kylin.metadata.model.DataType;
-
-import com.google.common.collect.Maps;
-
-/**
- * @author yangli9
- * 
- * Note: the implementations MUST be thread-safe.
- * 
- */
-abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
-
-    final static Map<String, Class<?>> implementations;
-    static {
-        HashMap<String, Class<?>> impl = Maps.newHashMap();
-        impl.put("varchar", StringSerializer.class);
-        impl.put("decimal", BigDecimalSerializer.class);
-        impl.put("double", DoubleSerializer.class);
-        impl.put("float", DoubleSerializer.class);
-        impl.put("bigint", LongSerializer.class);
-        impl.put("long", LongSerializer.class);
-        impl.put("integer", LongSerializer.class);
-        impl.put("int", LongSerializer.class);
-        impl.put("smallint", LongSerializer.class);
-        impl.put("date", DateTimeSerializer.class);
-        impl.put("datetime", DateTimeSerializer.class);
-        impl.put("timestamp", DateTimeSerializer.class);
-        implementations = Collections.unmodifiableMap(impl);
-
-    }
-
-    public static DataTypeSerializer<?> create(String dataType) {
-        return create(DataType.getInstance(dataType));
-    }
-
-    public static DataTypeSerializer<?> create(DataType type) {
-        if (type.isHLLC()) {
-            return new HLLCSerializer(type);
-        }
-
-        if (type.isTopN()) {
-            return new TopNCounterSerializer(type);
-        }
-
-        Class<?> clz = implementations.get(type.getName());
-        if (clz == null)
-            throw new RuntimeException("No MeasureSerializer for type " + type);
-
-        try {
-            return (DataTypeSerializer<?>) clz.getConstructor(DataType.class).newInstance(type);
-        } catch (Exception e) {
-            throw new RuntimeException(e); // never happen
-        }
-    }
-    
-    /** peek into buffer and return the length of serialization */
-    abstract public int peekLength(ByteBuffer in);
-
-    /** return the max number of bytes to the longest serialization */
-    abstract public int maxLength();
-
-    /** get an estimate of size in bytes of the serialized data */
-    abstract public int getStorageBytesEstimate();
-
-    /** convert from String to obj (string often come as byte[] in mapred) */
-    abstract public T valueOf(byte[] value);
-
-    /** convert from String to obj */
-    public T valueOf(String value) {
-        try {
-            return valueOf(value.getBytes("UTF-8"));
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException(e); // never happen
-        }
-    }
-
-    /** convert from obj to string */
-    public String toString(T value) {
-        if (value == null)
-            return "NULL";
-        else
-            return value.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/IAggregationFactory.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/IAggregationFactory.java b/core-cube/src/main/java/org/apache/kylin/aggregation/IAggregationFactory.java
new file mode 100644
index 0000000..cbdd9da
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/IAggregationFactory.java
@@ -0,0 +1,6 @@
+package org.apache.kylin.aggregation;
+
+public interface IAggregationFactory {
+
+    public AggregationType createAggregationType(String funcName, String dataType);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
index 9c8945d..6968a9d 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureAggregator.java
@@ -18,21 +18,6 @@
 
 package org.apache.kylin.aggregation;
 
-import org.apache.kylin.aggregation.basic.BigDecimalMaxAggregator;
-import org.apache.kylin.aggregation.basic.BigDecimalMinAggregator;
-import org.apache.kylin.aggregation.basic.BigDecimalSumAggregator;
-import org.apache.kylin.aggregation.basic.DoubleMaxAggregator;
-import org.apache.kylin.aggregation.basic.DoubleMinAggregator;
-import org.apache.kylin.aggregation.basic.DoubleSumAggregator;
-import org.apache.kylin.aggregation.basic.LongMaxAggregator;
-import org.apache.kylin.aggregation.basic.LongMinAggregator;
-import org.apache.kylin.aggregation.basic.LongSumAggregator;
-import org.apache.kylin.aggregation.hllc.HLLCAggregator;
-import org.apache.kylin.aggregation.hllc.LDCAggregator;
-import org.apache.kylin.aggregation.topn.TopNAggregator;
-import org.apache.kylin.metadata.model.DataType;
-import org.apache.kylin.metadata.model.FunctionDesc;
-
 import java.io.Serializable;
 
 /**
@@ -40,50 +25,8 @@ import java.io.Serializable;
 @SuppressWarnings("serial")
 abstract public class MeasureAggregator<V> implements Serializable {
 
-    public static MeasureAggregator<?> create(String funcName, String returnType) {
-        if (FunctionDesc.FUNC_SUM.equalsIgnoreCase(funcName) || FunctionDesc.FUNC_COUNT.equalsIgnoreCase(funcName)) {
-            if (isInteger(returnType))
-                return new LongSumAggregator();
-            else if (isBigDecimal(returnType))
-                return new BigDecimalSumAggregator();
-            else if (isDouble(returnType))
-                return new DoubleSumAggregator();
-        } else if (FunctionDesc.FUNC_COUNT_DISTINCT.equalsIgnoreCase(funcName)) {
-            DataType hllcType = DataType.getInstance(returnType);
-            if (hllcType.isHLLC())
-                return new HLLCAggregator(hllcType.getPrecision());
-            else
-                return new LDCAggregator();
-        } else if (FunctionDesc.FUNC_MAX.equalsIgnoreCase(funcName)) {
-            if (isInteger(returnType))
-                return new LongMaxAggregator();
-            else if (isBigDecimal(returnType))
-                return new BigDecimalMaxAggregator();
-            else if (isDouble(returnType))
-                return new DoubleMaxAggregator();
-        } else if (FunctionDesc.FUNC_MIN.equalsIgnoreCase(funcName)) {
-            if (isInteger(returnType))
-                return new LongMinAggregator();
-            else if (isBigDecimal(returnType))
-                return new BigDecimalMinAggregator();
-            else if (isDouble(returnType))
-                return new DoubleMinAggregator();
-        } else if (FunctionDesc.FUNC_TOP_N.equalsIgnoreCase(funcName)) {
-            return new TopNAggregator();
-        }
-        throw new IllegalArgumentException("No aggregator for func '" + funcName + "' and return type '" + returnType + "'");
-    }
-
-    public static boolean isBigDecimal(String type) {
-        return type.startsWith("decimal");
-    }
-
-    public static boolean isDouble(String type) {
-        return "double".equalsIgnoreCase(type) || "float".equalsIgnoreCase(type) || "real".equalsIgnoreCase(type);
-    }
-
-    public static boolean isInteger(String type) {
-        return "long".equalsIgnoreCase(type) || "bigint".equalsIgnoreCase(type) || "int".equalsIgnoreCase(type) || "integer".equalsIgnoreCase(type);
+    public static MeasureAggregator<?> create(String funcName, String dataType) {
+        return AggregationType.create(funcName, dataType).newAggregator();
     }
 
     public static int guessBigDecimalMemBytes() {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java
index cbcb3a8..8721954 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/MeasureCodec.java
@@ -21,6 +21,7 @@ package org.apache.kylin.aggregation;
 import java.nio.ByteBuffer;
 import java.util.Collection;
 
+import org.apache.kylin.common.datatype.DataTypeSerializer;
 import org.apache.kylin.metadata.model.MeasureDesc;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregation.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregation.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregation.java
new file mode 100644
index 0000000..7ea88bd
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregation.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.aggregation.basic;
+
+import java.util.List;
+
+import org.apache.kylin.aggregation.AggregationType;
+import org.apache.kylin.aggregation.MeasureAggregator;
+import org.apache.kylin.common.datatype.BigDecimalSerializer;
+import org.apache.kylin.common.datatype.DataType;
+import org.apache.kylin.common.datatype.DataTypeSerializer;
+import org.apache.kylin.common.datatype.DateTimeSerializer;
+import org.apache.kylin.common.datatype.DoubleSerializer;
+import org.apache.kylin.common.datatype.LongSerializer;
+import org.apache.kylin.common.datatype.StringSerializer;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public class BasicAggregation extends AggregationType {
+    
+    private final String funcName;
+    private final DataType dataType;
+
+    public BasicAggregation(String funcName, String dataType) {
+        this.funcName = funcName;
+        this.dataType = DataType.getType(dataType);
+    }
+
+    @Override
+    public DataType getAggregationDataType() {
+        return dataType;
+    }
+
+    public Class<? extends DataTypeSerializer<?>> getAggregationDataSeralizer() {
+        if (dataType.isStringFamily())
+            return StringSerializer.class;
+        else if (dataType.isIntegerFamily())
+            return LongSerializer.class;
+        else if (dataType.isDecimal())
+            return BigDecimalSerializer.class;
+        else if (dataType.isNumberFamily())
+            return DoubleSerializer.class;
+        else if (dataType.isDateTimeFamily())
+            return DateTimeSerializer.class;
+        else
+            throw new IllegalArgumentException("No default serializer for type " + dataType);
+    }
+    
+    @Override
+    public void validate(MeasureDesc measureDesc) throws IllegalArgumentException {
+        // TODO Auto-generated method stub
+        
+    }
+
+    @Override
+    public MeasureAggregator<?> newAggregator() {
+        if (isSum() || isCount()) {
+            if (dataType.isDecimal())
+                return new BigDecimalSumAggregator();
+            else if (dataType.isIntegerFamily())
+                return new LongSumAggregator();
+            else if (dataType.isNumberFamily())
+                return new DoubleSumAggregator();
+        } else if (isMax()) {
+            if (dataType.isDecimal())
+                return new BigDecimalMaxAggregator();
+            else if (dataType.isIntegerFamily())
+                return new LongMaxAggregator();
+            else if (dataType.isNumberFamily())
+                return new DoubleMaxAggregator();
+        } else if (isMin()) {
+            if (dataType.isDecimal())
+                return new BigDecimalMinAggregator();
+            else if (dataType.isIntegerFamily())
+                return new LongMinAggregator();
+            else if (dataType.isNumberFamily())
+                return new DoubleMinAggregator();
+        }
+        throw new IllegalArgumentException("No aggregator for func '" + funcName + "' and return type '" + dataType + "'");
+    }
+    
+    private boolean isSum() {
+        return FunctionDesc.FUNC_SUM.equalsIgnoreCase(funcName);
+    }
+
+    private boolean isCount() {
+        return FunctionDesc.FUNC_COUNT.equalsIgnoreCase(funcName);
+    }
+    
+    private boolean isMax() {
+        return FunctionDesc.FUNC_MAX.equalsIgnoreCase(funcName);
+    }
+    
+    private boolean isMin() {
+        return FunctionDesc.FUNC_MIN.equalsIgnoreCase(funcName);
+    }
+    
+    @Override
+    public List<TblColRef> getColumnsNeedDictionary(MeasureDesc measureDesc) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Object reEncodeDictionary(Object value, List<Dictionary<?>> oldDicts, List<Dictionary<?>> newDicts) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregationFactory.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregationFactory.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregationFactory.java
new file mode 100644
index 0000000..547d45f
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BasicAggregationFactory.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.aggregation.basic;
+
+import org.apache.kylin.aggregation.AggregationType;
+import org.apache.kylin.aggregation.IAggregationFactory;
+
+public class BasicAggregationFactory implements IAggregationFactory {
+
+    @Override
+    public AggregationType createAggregationType(String funcName, String dataType) {
+        return new BasicAggregation(funcName, dataType);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSerializer.java
deleted file mode 100644
index 9f7c3cf..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/BigDecimalSerializer.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.aggregation.basic;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.aggregation.DataTypeSerializer;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.model.DataType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author yangli9
- * 
- */
-public class BigDecimalSerializer extends DataTypeSerializer<BigDecimal> {
-
-    private static final Logger logger = LoggerFactory.getLogger(BigDecimalSerializer.class);
-
-    final DataType type;
-    final int maxLength;
-
-    int avoidVerbose = 0;
-
-    public BigDecimalSerializer(DataType type) {
-        this.type = type;
-        // see serialize(): 1 byte scale, 1 byte length, assume every 2 digits takes 1 byte
-        this.maxLength = 1 + 1 + (type.getPrecision() + 1) / 2;
-    }
-
-    @Override
-    public void serialize(BigDecimal value, ByteBuffer out) {
-        if (value.scale() > type.getScale()) {
-            if (avoidVerbose % 10000 == 0) {
-                logger.warn("value's scale has exceeded the " + type.getScale() + ", cut it off, to ensure encoded value do not exceed maxLength " + maxLength + " times:" + (avoidVerbose++));
-            }
-            value = value.setScale(type.getScale(), BigDecimal.ROUND_HALF_EVEN);
-        }
-        byte[] bytes = value.unscaledValue().toByteArray();
-        if (bytes.length + 2 > maxLength) {
-            throw new IllegalArgumentException("'" + value + "' exceeds the expected length for type " + type);
-        }
-
-        BytesUtil.writeVInt(value.scale(), out);
-        BytesUtil.writeVInt(bytes.length, out);
-        out.put(bytes);
-    }
-
-    @Override
-    public BigDecimal deserialize(ByteBuffer in) {
-        int scale = BytesUtil.readVInt(in);
-        int n = BytesUtil.readVInt(in);
-
-        byte[] bytes = new byte[n];
-        in.get(bytes);
-
-        return new BigDecimal(new BigInteger(bytes), scale);
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-
-        @SuppressWarnings("unused")
-        int scale = BytesUtil.readVInt(in);
-        int n = BytesUtil.readVInt(in);
-        int len = in.position() - mark + n;
-
-        in.position(mark);
-        return len;
-    }
-
-    @Override
-    public int maxLength() {
-        return maxLength;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 8;
-    }
-
-    @Override
-    public BigDecimal valueOf(byte[] value) {
-        if (value == null)
-            return new BigDecimal(0);
-        else
-            return new BigDecimal(Bytes.toString(value));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DateTimeSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DateTimeSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DateTimeSerializer.java
deleted file mode 100644
index 0bf4aba..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DateTimeSerializer.java
+++ /dev/null
@@ -1,65 +0,0 @@
-package org.apache.kylin.aggregation.basic;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.aggregation.DataTypeSerializer;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.common.util.LongMutable;
-import org.apache.kylin.metadata.model.DataType;
-
-public class DateTimeSerializer extends DataTypeSerializer<LongMutable> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
-
-    public DateTimeSerializer(DataType type) {
-    }
-
-    @Override
-    public void serialize(LongMutable value, ByteBuffer out) {
-        out.putLong(value.get());
-    }
-
-    private LongMutable current() {
-        LongMutable l = current.get();
-        if (l == null) {
-            l = new LongMutable();
-            current.set(l);
-        }
-        return l;
-    }
-
-    @Override
-    public LongMutable deserialize(ByteBuffer in) {
-        LongMutable l = current();
-        l.set(in.getLong());
-        return l;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return 8;
-    }
-
-    @Override
-    public int maxLength() {
-        return 8;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 8;
-    }
-
-    @Override
-    public LongMutable valueOf(byte[] value) {
-        LongMutable l = current();
-        if (value == null)
-            l.set(0L);
-        else
-            l.set(DateFormat.stringToMillis(Bytes.toString(value)));
-        return l;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java
index f09614d..99896a6 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMaxAggregator.java
@@ -19,7 +19,7 @@
 package org.apache.kylin.aggregation.basic;
 
 import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.util.DoubleMutable;
+import org.apache.kylin.common.datatype.DoubleMutable;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java
index b93c15c..7430c4e 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleMinAggregator.java
@@ -19,7 +19,7 @@
 package org.apache.kylin.aggregation.basic;
 
 import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.util.DoubleMutable;
+import org.apache.kylin.common.datatype.DoubleMutable;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSerializer.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSerializer.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSerializer.java
deleted file mode 100644
index f207054..0000000
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSerializer.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.aggregation.basic;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.aggregation.DataTypeSerializer;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.DoubleMutable;
-import org.apache.kylin.metadata.model.DataType;
-
-/**
- */
-public class DoubleSerializer extends DataTypeSerializer<DoubleMutable> {
-
-    // be thread-safe and avoid repeated obj creation
-    private ThreadLocal<DoubleMutable> current = new ThreadLocal<DoubleMutable>();
-
-    public DoubleSerializer(DataType type) {
-    }
-
-    @Override
-    public void serialize(DoubleMutable value, ByteBuffer out) {
-        out.putDouble(value.get());
-    }
-
-    private DoubleMutable current() {
-        DoubleMutable d = current.get();
-        if (d == null) {
-            d = new DoubleMutable();
-            current.set(d);
-        }
-        return d;
-    }
-
-    @Override
-    public DoubleMutable deserialize(ByteBuffer in) {
-        DoubleMutable d = current();
-        d.set(in.getDouble());
-        return d;
-    }
-
-    @Override
-    public int peekLength(ByteBuffer in) {
-        return 8;
-    }
-
-    @Override
-    public int maxLength() {
-        return 8;
-    }
-
-    @Override
-    public int getStorageBytesEstimate() {
-        return 8;
-    }
-
-    @Override
-    public DoubleMutable valueOf(byte[] value) {
-        DoubleMutable d = current();
-        if (value == null)
-            d.set(0d);
-        else
-            d.set(Double.parseDouble(Bytes.toString(value)));
-        return d;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java
index 298cec6..6e66c1b 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/DoubleSumAggregator.java
@@ -19,7 +19,7 @@
 package org.apache.kylin.aggregation.basic;
 
 import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.util.DoubleMutable;
+import org.apache.kylin.common.datatype.DoubleMutable;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java
index 71d95f2..7fdf3d8 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMaxAggregator.java
@@ -19,7 +19,7 @@
 package org.apache.kylin.aggregation.basic;
 
 import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.util.LongMutable;
+import org.apache.kylin.common.datatype.LongMutable;
 
 /**
  */

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8f2a56cf/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java
index d1e93f2..22ae865 100644
--- a/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java
+++ b/core-cube/src/main/java/org/apache/kylin/aggregation/basic/LongMinAggregator.java
@@ -19,7 +19,7 @@
 package org.apache.kylin.aggregation.basic;
 
 import org.apache.kylin.aggregation.MeasureAggregator;
-import org.apache.kylin.common.util.LongMutable;
+import org.apache.kylin.common.datatype.LongMutable;
 
 /**
  */