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/12/23 03:26:40 UTC

[36/50] [abbrv] kylin git commit: KYLIN-976 Support Custom Aggregation Types

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
index 7a0ab15..8587075 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.invertedindex.index.TableRecord;
 import org.apache.kylin.invertedindex.index.TableRecordInfo;
+import org.apache.kylin.measure.hllc.HLLCMeasureType;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.model.FunctionDesc;
@@ -45,8 +46,8 @@ import org.apache.kylin.storage.hbase.coprocessor.CoprocessorFilter;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorProjector;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorRowType;
 import org.apache.kylin.storage.hbase.coprocessor.endpoint.generated.IIProtos;
-import org.apache.kylin.storage.tuple.Tuple;
-import org.apache.kylin.storage.tuple.TupleInfo;
+import org.apache.kylin.metadata.tuple.Tuple;
+import org.apache.kylin.metadata.tuple.TupleInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -148,7 +149,7 @@ public class EndpointTupleIterator implements ITupleIterator {
                 boolean updated = false;
                 for (TblColRef column : columns) {
                     if (column.isSameAs(factTableName, functionDesc.getParameter().getValue())) {
-                        if (functionDesc.isCountDistinct()) {
+                        if (HLLCMeasureType.isCountDistinct(functionDesc)) {
                             //TODO: default precision might need be configurable
                             String iiDefaultHLLC = "hllc10";
                             functionDesc.setReturnType(iiDefaultHLLC);
@@ -243,7 +244,7 @@ public class EndpointTupleIterator implements ITupleIterator {
         }
 
         for (FunctionDesc measure : measures) {
-            info.setField(measure.getRewriteFieldName(), null, measure.getSQLType().getName(), index++);
+            info.setField(measure.getRewriteFieldName(), null, measure.getRewriteFieldType().getName(), index++);
         }
 
         return info;

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
index d55361b..a770f55 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
@@ -38,7 +38,7 @@ 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.measure.MeasureAggregator;
 import org.apache.kylin.storage.filter.BitMapFilterEvaluator;
 import org.apache.kylin.storage.hbase.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorConstants;

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
index 65b616f..8075bc3 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/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.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorFilter;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorProjector;

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
index f609a5a..f5fb497 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/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.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.coprocessor.AggregationCache;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java
index b0cd0f2..cac37f4 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregators.java
@@ -31,8 +31,11 @@ import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.kv.RowValueDecoder;
 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.measure.MeasureAggregator;
+import org.apache.kylin.measure.MeasureCodec;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.measure.MeasureTypeFactory;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorConstants;
 
@@ -123,6 +126,8 @@ public class ObserverAggregators {
     final ByteBuffer[] hColValues;
     final int nTotalMeasures;
 
+    MeasureType measureTypes[];
+
     public ObserverAggregators(HCol[] _hcols) {
         this.hcols = sort(_hcols);
         this.nHCols = hcols.length;
@@ -150,11 +155,18 @@ public class ObserverAggregators {
     }
 
     public MeasureAggregator[] createBuffer() {
+        if (measureTypes == null) {
+            measureTypes = new MeasureType[nTotalMeasures];
+            int i = 0;
+            for (HCol col : hcols) {
+                for (int j = 0; j < col.nMeasures; j++)
+                    measureTypes[i++] = MeasureTypeFactory.create(col.funcNames[j], DataType.getInstance(col.dataTypes[j]));
+            }
+        }
+
         MeasureAggregator[] aggrs = new MeasureAggregator[nTotalMeasures];
-        int i = 0;
-        for (HCol col : hcols) {
-            for (int j = 0; j < col.nMeasures; j++)
-                aggrs[i++] = MeasureAggregator.create(col.funcNames[j], col.dataTypes[j]);
+        for (int i = 0; i < nTotalMeasures; i++) {
+            aggrs[i] = measureTypes[i].newAggregator();
         }
         return aggrs;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
index f0f7ed5..5278326 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
@@ -117,7 +117,7 @@ public class ObserverEnabler {
             return r;
         }
 
-        if (RowValueDecoder.hasMemHungryCountDistinct(rowValueDecoders)) {
+        if (RowValueDecoder.hasMemHungryMeasures(rowValueDecoders)) {
             logger.info("Coprocessor is disabled because there is memory hungry count distinct");
             return false;
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverTuple.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverTuple.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverTuple.java
index 2a8f20b..fd6ba8b 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverTuple.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverTuple.java
@@ -21,7 +21,7 @@ package org.apache.kylin.storage.hbase.coprocessor.observer;
 import java.util.List;
 
 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.ITuple;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorRowType;

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java b/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
index 9d61fd0..42ae77d 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
@@ -28,6 +28,7 @@ import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.project.RealizationEntry;
+import org.apache.kylin.metadata.realization.CapabilityResult;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.metadata.realization.RealizationRegistry;
 import org.apache.kylin.metadata.realization.RealizationType;
@@ -90,7 +91,7 @@ public class HybridInstance extends RootPersistentEntity implements IRealization
             for (int i = 0; i < realizationEntries.size(); i++) {
                 IRealization realization = registry.getRealization(realizationEntries.get(i).getType(), realizationEntries.get(i).getRealization());
                 if (realization == null) {
-                    logger.error("Realization '" + realization.getName() + " is not found, remove from Hybrid '" + this.getName() + "'");
+                    logger.error("Realization '" + realizationEntries.get(i) + " is not found, remove from Hybrid '" + this.getName() + "'");
                     continue;
                 }
                 if (realization.isReady() == false) {
@@ -150,26 +151,24 @@ public class HybridInstance extends RootPersistentEntity implements IRealization
             initiated = true;
         }
     }
-
     @Override
-    public boolean isCapable(SQLDigest digest) {
+    public CapabilityResult isCapable(SQLDigest digest) {
+        CapabilityResult result = new CapabilityResult();
+        result.cost = Integer.MAX_VALUE;
+
         for (IRealization realization : getRealizations()) {
-            if (realization.isCapable(digest))
-                return true;
+            CapabilityResult child = realization.isCapable(digest);
+            if (child.capable) {
+                result.capable = true;
+                result.cost = Math.min(result.cost, child.cost);
+                result.influences.addAll(child.influences);
+            }
         }
-        return false;
-    }
 
-    @Override
-    public int getCost(SQLDigest digest) {
-        cost = 100;
-        for (IRealization realization : this.getRealizations()) {
-            if (realization.isCapable(digest))
-                cost = Math.min(cost, realization.getCost(digest));
-        }
+        if (result.cost > 0)
+            result.cost--; // let hybrid win its children
 
-        // Make hybrid always win its children
-        return cost - 1;
+        return result;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageEngine.java
index 45d27cb..5c2e84c 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageEngine.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageEngine.java
@@ -48,7 +48,7 @@ public class HybridStorageEngine implements IStorageEngine {
     public ITupleIterator search(final StorageContext context, final SQLDigest sqlDigest) {
         List<ITupleIterator> tupleIterators = Lists.newArrayList();
         for (int i = 0; i < realizations.length; i++) {
-            if (realizations[i].isReady() && realizations[i].isCapable(sqlDigest)) {
+            if (realizations[i].isReady() && realizations[i].isCapable(sqlDigest).capable) {
                 ITupleIterator dataIterator = storageEngines[i].search(context, sqlDigest);
                 tupleIterators.add(dataIterator);
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java b/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
deleted file mode 100644
index 556ddca..0000000
--- a/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
+++ /dev/null
@@ -1,252 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.storage.tuple;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.List;
-
-import org.apache.kylin.common.util.Array;
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
-import org.apache.kylin.dict.lookup.LookupStringTable;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.tuple.ITuple;
-
-/**
- * @author xjiang
- */
-public class Tuple implements ITuple {
-
-    private final TupleInfo info;
-    private final Object[] values;
-
-    public Tuple(TupleInfo info) {
-        this.info = info;
-        this.values = new Object[info.size()];
-    }
-
-    public List<String> getAllFields() {
-        return info.getAllFields();
-    }
-
-    public List<TblColRef> getAllColumns() {
-        return info.getAllColumns();
-    }
-
-    public Object[] getAllValues() {
-        return values;
-    }
-
-    public TupleInfo getInfo() {
-        return info;
-    }
-
-    public String getFieldName(TblColRef col) {
-        return info.getFieldName(col);
-    }
-
-    public TblColRef getFieldColumn(String fieldName) {
-        return info.getColumn(fieldName);
-    }
-
-    public Object getValue(String fieldName) {
-        int index = info.getFieldIndex(fieldName);
-        return values[index];
-    }
-
-    public Object getValue(TblColRef col) {
-        int index = info.getColumnIndex(col);
-        return values[index];
-    }
-
-    public String getDataType(String fieldName) {
-        return info.getDataType(fieldName);
-    }
-
-    public void setFieldObjectValue(String fieldName, Object fieldValue) {
-        int index = info.getFieldIndex(fieldName);
-        values[index] = fieldValue;
-    }
-
-    public void setDimensionValue(String fieldName, String fieldValue) {
-        Object objectValue = convertOptiqCellValue(fieldValue, getDataType(fieldName));
-        setFieldObjectValue(fieldName, objectValue);
-    }
-
-    public void setMeasureValue(String fieldName, Object fieldValue) {
-        String dataType = info.getDataType(fieldName);
-        // special handling for BigDecimal, allow double be aggregated as
-        // BigDecimal during cube build for best precision
-        if ("double".equals(dataType) && fieldValue instanceof BigDecimal) {
-            fieldValue = ((BigDecimal) fieldValue).doubleValue();
-        } else if ("integer".equals(dataType) && !(fieldValue instanceof Integer)) {
-            fieldValue = ((Number) fieldValue).intValue();
-        } else if ("float".equals(dataType) && fieldValue instanceof BigDecimal) {
-            fieldValue = ((BigDecimal) fieldValue).floatValue();
-        }
-
-        setFieldObjectValue(fieldName, fieldValue);
-    }
-
-    public boolean hasColumn(TblColRef column) {
-        return info.hasColumn(column);
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        for (String field : info.getAllFields()) {
-            sb.append(field);
-            sb.append("=");
-            sb.append(getValue(field));
-            sb.append(",");
-        }
-        return sb.toString();
-    }
-
-    public static Object convertOptiqCellValue(String strValue, String dataType) {
-        if (strValue == null)
-            return null;
-
-        if ((strValue.equals("") || strValue.equals("\\N")) && !dataType.equals("string"))
-            return null;
-
-        // TODO use data type enum instead of string comparison
-        if ("date".equals(dataType)) {
-            // convert epoch time
-            Date dateValue = DateFormat.stringToDate(strValue); // NOTE: forces GMT timezone
-            long millis = dateValue.getTime();
-            long days = millis / (1000 * 3600 * 24);
-            return Integer.valueOf((int) days); // Optiq expects Integer instead of Long. by honma
-        } else if ("tinyint".equals(dataType)) {
-            return Byte.valueOf(strValue);
-        } else if ("short".equals(dataType) || "smallint".equals(dataType)) {
-            return Short.valueOf(strValue);
-        } else if ("integer".equals(dataType)) {
-            return Integer.valueOf(strValue);
-        } else if ("long".equals(dataType) || "bigint".equals(dataType)) {
-            return Long.valueOf(strValue);
-        } else if ("double".equals(dataType)) {
-            return Double.valueOf(strValue);
-        } else if ("decimal".equals(dataType)) {
-            return new BigDecimal(strValue);
-        } else if ("timestamp".equals(dataType)) {
-            return Long.valueOf(DateFormat.stringToMillis(strValue));
-        } else if ("float".equals(dataType)) {
-            return Float.valueOf(strValue);
-        } else if ("boolean".equals(dataType)) {
-            return Boolean.valueOf(strValue);
-        } else {
-            return strValue;
-        }
-    }
-
-    // ============================================================================
-
-    public static IDerivedColumnFiller newDerivedColumnFiller(List<TblColRef> rowColumns, TblColRef[] hostCols, DeriveInfo deriveInfo, TupleInfo tupleInfo, CubeManager cubeMgr, CubeSegment cubeSegment) {
-
-        int[] hostIndex = new int[hostCols.length];
-        for (int i = 0; i < hostCols.length; i++) {
-            hostIndex[i] = rowColumns.indexOf(hostCols[i]);
-        }
-        String[] derivedFieldNames = new String[deriveInfo.columns.length];
-        for (int i = 0; i < deriveInfo.columns.length; i++) {
-            derivedFieldNames[i] = tupleInfo.getFieldName(deriveInfo.columns[i]);
-        }
-
-        switch (deriveInfo.type) {
-        case LOOKUP:
-            LookupStringTable lookupTable = cubeMgr.getLookupTable(cubeSegment, deriveInfo.dimension);
-            return new LookupFiller(hostIndex, lookupTable, deriveInfo, derivedFieldNames);
-        case PK_FK:
-            // composite key are split, see CubeDesc.initDimensionColumns()
-            return new PKFKFiller(hostIndex[0], derivedFieldNames[0]);
-        default:
-            throw new IllegalArgumentException();
-        }
-    }
-
-    public interface IDerivedColumnFiller {
-        public void fillDerivedColumns(List<String> rowValues, Tuple tuple);
-    }
-
-    static class PKFKFiller implements IDerivedColumnFiller {
-        final int hostIndex;
-        final String derivedFieldName;
-
-        public PKFKFiller(int hostIndex, String derivedFieldName) {
-            this.hostIndex = hostIndex;
-            this.derivedFieldName = derivedFieldName;
-        }
-
-        @Override
-        public void fillDerivedColumns(List<String> rowValues, Tuple tuple) {
-            String value = rowValues.get(hostIndex);
-            tuple.setDimensionValue(derivedFieldName, value);
-        }
-    }
-
-    static class LookupFiller implements IDerivedColumnFiller {
-
-        final int[] hostIndex;
-        final int hostLen;
-        final Array<String> lookupKey;
-        final LookupStringTable lookupTable;
-        final int[] derivedIndex;
-        final int derivedLen;
-        final String[] derivedFieldNames;
-
-        public LookupFiller(int[] hostIndex, LookupStringTable lookupTable, DeriveInfo deriveInfo, String[] derivedFieldNames) {
-            this.hostIndex = hostIndex;
-            this.hostLen = hostIndex.length;
-            this.lookupKey = new Array<String>(new String[hostLen]);
-            this.lookupTable = lookupTable;
-            this.derivedIndex = new int[deriveInfo.columns.length];
-            this.derivedLen = derivedIndex.length;
-            this.derivedFieldNames = derivedFieldNames;
-
-            for (int i = 0; i < derivedLen; i++) {
-                derivedIndex[i] = deriveInfo.columns[i].getColumn().getZeroBasedIndex();
-            }
-        }
-
-        @Override
-        public void fillDerivedColumns(List<String> rowValues, Tuple tuple) {
-            for (int i = 0; i < hostLen; i++) {
-                lookupKey.data[i] = rowValues.get(hostIndex[i]);
-            }
-
-            String[] lookupRow = lookupTable.getRow(lookupKey);
-
-            if (lookupRow != null) {
-                for (int i = 0; i < derivedLen; i++) {
-                    String value = lookupRow[derivedIndex[i]];
-                    tuple.setDimensionValue(derivedFieldNames[i], value);
-                }
-            } else {
-                for (int i = 0; i < derivedLen; i++) {
-                    tuple.setDimensionValue(derivedFieldNames[i], null);
-                }
-            }
-        }
-    }
-}

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

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java b/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java
index 5a3041d..c915912 100644
--- a/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.assertEquals;
 import java.util.ArrayList;
 import java.util.Iterator;
 
-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;

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/test/java/org/apache/kylin/storage/filter/FilterBaseTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/filter/FilterBaseTest.java b/storage/src/test/java/org/apache/kylin/storage/filter/FilterBaseTest.java
index 05405fb..6631111 100644
--- a/storage/src/test/java/org/apache/kylin/storage/filter/FilterBaseTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/filter/FilterBaseTest.java
@@ -33,8 +33,8 @@ import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.tuple.Tuple;
-import org.apache.kylin.storage.tuple.TupleInfo;
+import org.apache.kylin.metadata.tuple.Tuple;
+import org.apache.kylin.metadata.tuple.TupleInfo;
 
 /**
  * @author xjiang

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/test/java/org/apache/kylin/storage/filter/FilterEvaluateTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/filter/FilterEvaluateTest.java b/storage/src/test/java/org/apache/kylin/storage/filter/FilterEvaluateTest.java
index dd5d142..bf29655 100644
--- a/storage/src/test/java/org/apache/kylin/storage/filter/FilterEvaluateTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/filter/FilterEvaluateTest.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilterSerializer;
 import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.tuple.Tuple;
+import org.apache.kylin.metadata.tuple.Tuple;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/test/java/org/apache/kylin/storage/filter/FilterPerfTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/filter/FilterPerfTest.java b/storage/src/test/java/org/apache/kylin/storage/filter/FilterPerfTest.java
index 7098c01..54b16e4 100644
--- a/storage/src/test/java/org/apache/kylin/storage/filter/FilterPerfTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/filter/FilterPerfTest.java
@@ -34,8 +34,8 @@ import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.tuple.Tuple;
-import org.apache.kylin.storage.tuple.TupleInfo;
+import org.apache.kylin.metadata.tuple.Tuple;
+import org.apache.kylin.metadata.tuple.TupleInfo;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/kylin/blob/c721d679/storage/src/test/java/org/apache/kylin/storage/hbase/ColumnValueRangeTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/hbase/ColumnValueRangeTest.java b/storage/src/test/java/org/apache/kylin/storage/hbase/ColumnValueRangeTest.java
index 769b790..2c4dd51 100644
--- a/storage/src/test/java/org/apache/kylin/storage/hbase/ColumnValueRangeTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/hbase/ColumnValueRangeTest.java
@@ -24,7 +24,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/kylin/blob/c721d679/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointAggregationTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointAggregationTest.java b/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointAggregationTest.java
index 09ef85d..9a50945 100644
--- a/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointAggregationTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointAggregationTest.java
@@ -33,12 +33,12 @@ import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.index.TableRecord;
 import org.apache.kylin.invertedindex.index.TableRecordInfo;
+import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.measure.MeasureAggregator;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;